diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java index c42c237ee36..a7b4ba21203 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java @@ -37,6 +37,8 @@ import org.apache.calcite.rel.core.RelFactories; import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.rel.rules.DateRangeRules; +import org.apache.calcite.rel.rules.FilterJoinRule.FilterIntoJoinRule.FilterIntoJoinRuleConfig; +import org.apache.calcite.rel.rules.JoinExtractFilterRule; import org.apache.calcite.rel.rules.JoinPushThroughJoinRule; import org.apache.calcite.rel.rules.ProjectMergeRule; import org.apache.calcite.rel.rules.PruneEmptyRules; @@ -67,6 +69,7 @@ import org.apache.druid.sql.calcite.rule.ReverseLookupRule; import org.apache.druid.sql.calcite.rule.RewriteFirstValueLastValueRule; import org.apache.druid.sql.calcite.rule.SortCollapseRule; import org.apache.druid.sql.calcite.rule.logical.DruidAggregateRemoveRedundancyRule; +import org.apache.druid.sql.calcite.rule.logical.DruidJoinRule; import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules; import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.hook.DruidHook; @@ -273,10 +276,11 @@ public class CalciteRulesManager { final HepProgramBuilder builder = HepProgram.builder(); builder.addMatchLimit(CalciteRulesManager.HEP_DEFAULT_MATCH_LIMIT); - builder.addGroupBegin(); builder.addRuleCollection(baseRuleSet(plannerContext)); builder.addRuleInstance(CoreRules.UNION_MERGE); - builder.addGroupEnd(); + builder.addRuleInstance(JoinExtractFilterRule.Config.DEFAULT.toRule()); + builder.addRuleInstance(FilterIntoJoinRuleConfig.DEFAULT.withPredicate(DruidJoinRule::isSupportedPredicate).toRule()); + return Programs.of(builder.build(), true, DefaultRelMetadataProvider.INSTANCE); } @@ -494,6 +498,8 @@ public class CalciteRulesManager rules.addAll(FANCY_JOIN_RULES); } + rules.add(DruidAggregateRemoveRedundancyRule.instance()); + if (!plannerConfig.isUseApproximateCountDistinct()) { if (plannerConfig.isUseGroupingSetForExactDistinct() && plannerContext.featureAvailable(EngineFeature.GROUPING_SETS)) { @@ -507,7 +513,6 @@ public class CalciteRulesManager rules.add(FilterJoinExcludePushToChildRule.FILTER_ON_JOIN_EXCLUDE_PUSH_TO_CHILD); rules.add(SortCollapseRule.instance()); rules.add(ProjectAggregatePruneUnusedCallRule.instance()); - rules.add(DruidAggregateRemoveRedundancyRule.instance()); return rules.build(); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java index 61f6d13755b..28de4e8c255 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java @@ -259,6 +259,7 @@ public class DruidQueryGenerator final PartialDruidQuery partialDruidQuery; final List inputs; final JoinSupportTweaks jst; + private SourceDesc source; public PDQVertex(PartialDruidQuery partialDruidQuery, List inputs, JoinSupportTweaks jst) { @@ -280,10 +281,18 @@ public class DruidQueryGenerator ); } + private SourceDesc getSource() + { + if (source == null) { + source = realGetSource(); + } + return source; + } + /** * Creates the {@link SourceDesc} for the current {@link Vertex}. */ - private SourceDesc getSource() + private SourceDesc realGetSource() { List sourceDescs = new ArrayList<>(); for (Vertex inputVertex : inputs) { 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 1a50ae1fbcd..faf459f658c 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 @@ -24,6 +24,8 @@ import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rex.RexNode; import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.sql.calcite.rel.logical.DruidJoin; import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; @@ -72,4 +74,13 @@ public class DruidJoinRule extends ConverterRule ); } + public static boolean isSupportedPredicate(Join join, JoinRelType joinType, RexNode exp) + { + ConditionAnalysis analysis = org.apache.druid.sql.calcite.rule.DruidJoinRule.analyzeCondition( + exp, + join.getLeft().getRowType(), + join.getCluster().getRexBuilder() + ); + return analysis.errorStr == null; + } } 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 04cea5e2c13..026261b566b 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 @@ -678,9 +678,9 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") - @NotYetSupported(Modes.CANNOT_JOIN_LOOKUP_NON_KEY) public void testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing(Map queryContext) { @@ -760,9 +760,9 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) public void testFilterAndGroupByLookupUsingJoinOperatorBackwards(Map queryContext) { // Like "testFilterAndGroupByLookupUsingJoinOperator", but with the table and lookup reversed. @@ -816,7 +816,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter(Map queryContext) @@ -864,7 +864,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testJoinUnionTablesOnLookup(Map queryContext) @@ -917,7 +917,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.CANNOT_JOIN_LOOKUP_NON_KEY) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testFilterAndGroupByLookupUsingJoinOperator(Map queryContext) @@ -1127,7 +1127,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_UNSUPORTED_OPERAND) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinTableLookupLookupWithFilterWithOuterLimit(Map queryContext) @@ -1171,7 +1171,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_UNSUPORTED_OPERAND) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinTableLookupLookupWithFilterWithoutLimit(Map queryContext) @@ -1213,7 +1213,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_UNSUPORTED_OPERAND) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns(Map queryContext) @@ -1258,7 +1258,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_UNSUPORTED_OPERAND) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns(Map queryContext) @@ -1300,7 +1300,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_UNSUPORTED_OPERAND) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testManyManyInnerJoinOnManyManyLookup(Map queryContext) @@ -2010,7 +2010,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest // This SQL currently does not result in an optimum plan. // Unfortunately, we have disabled pushing down predicates (conditions and filters) due to https://github.com/apache/druid/pull/9773 // Hence, comma join will result in a cross join with filter on outermost - @NotYetSupported(Modes.JOIN_CONDITION_UNSUPORTED_OPERAND) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IRRELEVANT_SCANQUERY) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testCommaJoinTableLookupTableMismatchedTypes(Map queryContext) @@ -2502,6 +2502,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testSelectOnLookupUsingRightJoinOperator(Map queryContext) @@ -2791,7 +2792,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinWithIsNullFilter(Map queryContext) @@ -3672,7 +3673,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") - @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN, separateDefaultModeTest = true) public void testLeftJoinSubqueryWithSelectorFilter(Map queryContext) { // Cannot vectorize due to 'concat' expression. @@ -3873,6 +3874,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinSubqueryWithSelectorFilter(Map queryContext) @@ -3928,8 +3930,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @Test - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) public void testSemiJoinWithOuterTimeExtractScan() { testQuery( @@ -3976,7 +3978,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testTwoSemiJoinsSimultaneously(Map queryContext) @@ -4040,6 +4042,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery(Map queryContext) @@ -4146,6 +4149,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins(Map queryContext) @@ -4214,8 +4218,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @Test - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) public void testSemiJoinWithOuterTimeExtractAggregateWithOrderBy() { testQuery( @@ -4681,7 +4685,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testJoinWithNonEquiCondition(Map queryContext) @@ -4743,7 +4746,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_UNSUPORTED_OPERAND) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testJoinWithEquiAndNonEquiCondition(Map queryContext) @@ -4788,7 +4790,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testUsingSubqueryAsPartOfAndFilter(Map queryContext) @@ -4946,8 +4948,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - - @NotYetSupported(Modes.JOIN_CONDITION_UNSUPORTED_OPERAND) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IRRELEVANT_SCANQUERY, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testNestedGroupByOnInlineDataSourceWithFilter(Map queryContext) @@ -5222,7 +5223,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_PLAN) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testVirtualColumnOnMVFilterJoinExpression(Map queryContext) @@ -5359,7 +5360,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults(Map queryContext) @@ -5467,7 +5468,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, separateDefaultModeTest = true) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults(Map queryContext) @@ -6138,7 +6139,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest } @Test - @NotYetSupported(Modes.CORRELATE_CONVERSION) + @NotYetSupported(Modes.UNNEST_NOT_SUPPORTED_CORRELATE_CONVERSION) public void testJoinsWithUnnestOnLeft() { // Segment map function of MSQ needs some work @@ -6193,7 +6194,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest } @Test - @NotYetSupported(Modes.CORRELATE_CONVERSION) + @NotYetSupported(Modes.UNNEST_NOT_SUPPORTED_CORRELATE_CONVERSION) public void testJoinsWithUnnestOverFilteredDSOnLeft() { // Segment map function of MSQ needs some work @@ -6251,7 +6252,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest } @Test - @NotYetSupported(Modes.CORRELATE_CONVERSION) + @NotYetSupported(Modes.UNNEST_NOT_SUPPORTED_CORRELATE_CONVERSION) public void testJoinsWithUnnestOverJoin() { // Segment map function of MSQ needs some work @@ -6336,7 +6337,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest } @Test - @NotYetSupported(Modes.CORRELATE_CONVERSION) + @NotYetSupported(Modes.UNNEST_NOT_SUPPORTED_CORRELATE_CONVERSION) public void testSelfJoinsWithUnnestOnLeftAndRight() { // Segment map function of MSQ needs some work @@ -6405,7 +6406,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest } @Test - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @NotYetSupported(Modes.UNNEST_NOT_SUPPORTED_CORRELATE_CONVERSION) public void testJoinsOverUnnestOverFilterDSOverJoin() { // Segment map function of MSQ needs some work 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 55da3f41c23..5af5ec3097c 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 @@ -6388,8 +6388,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testUnplannableJoinQueriesInNonSQLCompatibleMode() { + assumeFalse(testBuilder().isDecoupledMode(), "only valid in non-decoupled mode"); msqIncompatible(); - Assumptions.assumeFalse(NullHandling.sqlCompatible()); assertQueryIsUnplannable( @@ -6398,6 +6398,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "FROM foo INNER JOIN lookup.lookyloo l ON foo.dim2 <> l.k", "SQL requires a join with 'NOT_EQUALS' condition that is not supported." ); + } + + @Test + public void testUnplannableJoinQueriesInNonSQLCompatibleMode2() + { + assumeFalse(testBuilder().isDecoupledMode(), "only valid in non-decoupled mode"); + msqIncompatible(); + Assumptions.assumeFalse(NullHandling.sqlCompatible()); assertQueryIsUnplannable( // JOIN condition with a function of both sides. @@ -6407,6 +6415,36 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testPlannableJoinQueriesInNonSQLCompatibleMode() + { + assumeTrue(testBuilder().isDecoupledMode(), "only in decoupled mode"); + msqIncompatible(); + Assumptions.assumeFalse(NullHandling.sqlCompatible()); + + testBuilder() + .sql( + "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + + "FROM foo INNER JOIN lookup.lookyloo l ON foo.dim2 <> l.k" + ) + .run(); + } + + @Test + public void testPlannableJoinQueriesInNonSQLCompatibleMode2() + { + assumeTrue(testBuilder().isDecoupledMode(), "only in decoupled mode"); + msqIncompatible(); + Assumptions.assumeFalse(NullHandling.sqlCompatible()); + + testBuilder() + .sql( + "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + + "FROM foo INNER JOIN lookup.lookyloo l ON CHARACTER_LENGTH(foo.dim2 || l.k) > 3\n" + ) + .run(); + } + @Test public void testCountStarWithBoundFilterSimplifyOnMetric() { @@ -9341,7 +9379,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } - @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_PLAN, separateDefaultModeTest = true) @SqlTestFrameworkConfig.NumMergeBuffers(3) @Test public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename() @@ -12962,7 +12999,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest } // __time >= x remains in the join condition - @NotYetSupported(Modes.JOIN_CONDITION_NOT_PUSHED_CONDITION) + @NotYetSupported(Modes.REQUIRE_TIME_CONDITION) @Test public void testRequireTimeConditionPositive3() { 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 14ae52def5f..9bc4fe0fee4 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 @@ -96,7 +96,25 @@ public @interface DecoupledTestConfig * instead of joining on condition (CAST("j0.k", 'DOUBLE') == "_j0.m1") * a vc was computed for CAST("j0.k", 'DOUBLE') */ - EQUIV_PLAN_CAST_MATERIALIZED_EARLIER; + EQUIV_PLAN_CAST_MATERIALIZED_EARLIER, + /** + * Filter pushed down. + * + * Instead: + * Filter -> Join -> Table + * Join -> Filter -> Table + */ + SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND, + /** + * Instead: + * Join (l=r && lCol='a') -> Gby + * Join (l=r) -> Gby[lCol='a] + */ + FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE, + /** + * Strange things; needs more investigation + */ + IRRELEVANT_SCANQUERY; public boolean isPresent() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index 50dd2e43773..1a83a708e21 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite; import com.google.common.base.Throwables; import org.apache.druid.error.DruidException; +import org.apache.druid.sql.calcite.rel.CannotBuildQueryException; import org.junit.AssumptionViolatedException; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.InvocationInterceptor; @@ -91,12 +92,10 @@ public @interface NotYetSupported UNSUPPORTED_DATASOURCE(DruidException.class, "WindowOperatorQuery must run on top of a query or inline data source"), UNION_WITH_COMPLEX_OPERAND(DruidException.class, "Only Table and Values are supported as inputs for Union"), UNION_MORE_STRICT_ROWTYPE_CHECK(DruidException.class, "Row signature mismatch in Union inputs"), - JOIN_CONDITION_NOT_PUSHED_CONDITION(DruidException.class, "SQL requires a join with '.*' condition"), - JOIN_CONDITION_UNSUPORTED_OPERAND(DruidException.class, "SQL .* unsupported operand type"), - CORRELATE_CONVERSION(DruidException.class, "Missing conversion( is|s are) LogicalCorrelate"), + UNNEST_NOT_SUPPORTED_CORRELATE_CONVERSION(DruidException.class, "Missing conversion( is|s are) LogicalCorrelate"), SORT_REMOVE_TROUBLE(DruidException.class, "Calcite assertion violated.*Sort\\."), - CANNOT_JOIN_LOOKUP_NON_KEY(RuntimeException.class, "Cannot join lookup with condition referring to non-key"), - SORT_REMOVE_CONSTANT_KEYS_CONFLICT(DruidException.class, "not enough rules"); + SORT_REMOVE_CONSTANT_KEYS_CONFLICT(DruidException.class, "not enough rules"), + REQUIRE_TIME_CONDITION(CannotBuildQueryException.class, "requireTimeCondition is enabled"); // @formatter:on public Class throwableClass; diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@all_disabled.iq new file mode 100644 index 00000000000..89f9a54998d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@all_disabled.iq @@ -0,0 +1,151 @@ +# testCommaJoinTableLookupTableMismatchedTypes@all_disabled case-crc:fa64f472 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo, lookup.lookyloo l, numfoo +WHERE foo.cnt = l.k AND l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[=($0, CAST($1):BIGINT)]) + LogicalProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[=($0, CAST($1):BIGINT)]) + DruidProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[true], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "expression", + "expression" : "(\"cnt\" == CAST(\"j0.k\", 'LONG'))" + }, + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "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/testCommaJoinTableLookupTableMismatchedTypes@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@all_enabled.iq new file mode 100644 index 00000000000..94b7f73e131 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@all_enabled.iq @@ -0,0 +1,151 @@ +# testCommaJoinTableLookupTableMismatchedTypes@all_enabled case-crc:e2bcc2a9 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo, lookup.lookyloo l, numfoo +WHERE foo.cnt = l.k AND l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[=($0, CAST($1):BIGINT)]) + LogicalProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[=($0, CAST($1):BIGINT)]) + DruidProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[true], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "expression", + "expression" : "(\"cnt\" == CAST(\"j0.k\", 'LONG'))" + }, + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "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/testCommaJoinTableLookupTableMismatchedTypes@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@default.iq new file mode 100644 index 00000000000..d633b774a5f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@default.iq @@ -0,0 +1,148 @@ +# testCommaJoinTableLookupTableMismatchedTypes@default case-crc:14308da2 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo, lookup.lookyloo l, numfoo +WHERE foo.cnt = l.k AND l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[=($0, CAST($1):BIGINT)]) + LogicalProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[=($0, CAST($1):BIGINT)]) + DruidProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[true], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "expression", + "expression" : "(\"cnt\" == CAST(\"j0.k\", 'LONG'))" + }, + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "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/testCommaJoinTableLookupTableMismatchedTypes@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@filter-on-value-column_disabled.iq new file mode 100644 index 00000000000..b35cc7db31b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@filter-on-value-column_disabled.iq @@ -0,0 +1,151 @@ +# testCommaJoinTableLookupTableMismatchedTypes@filter-on-value-column_disabled case-crc:98c3c89c +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo, lookup.lookyloo l, numfoo +WHERE foo.cnt = l.k AND l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[=($0, CAST($1):BIGINT)]) + LogicalProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[=($0, CAST($1):BIGINT)]) + DruidProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[true], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "expression", + "expression" : "(\"cnt\" == CAST(\"j0.k\", 'LONG'))" + }, + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "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/testCommaJoinTableLookupTableMismatchedTypes@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@filter-rewrites-disabled.iq new file mode 100644 index 00000000000..c8a0f41b1dc --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@filter-rewrites-disabled.iq @@ -0,0 +1,151 @@ +# testCommaJoinTableLookupTableMismatchedTypes@filter-rewrites-disabled case-crc:b0c29cbc +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo, lookup.lookyloo l, numfoo +WHERE foo.cnt = l.k AND l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[=($0, CAST($1):BIGINT)]) + LogicalProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[=($0, CAST($1):BIGINT)]) + DruidProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[true], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "expression", + "expression" : "(\"cnt\" == CAST(\"j0.k\", 'LONG'))" + }, + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "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/testCommaJoinTableLookupTableMismatchedTypes@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@filter-rewrites.iq new file mode 100644 index 00000000000..7dbdb9d245c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@filter-rewrites.iq @@ -0,0 +1,151 @@ +# testCommaJoinTableLookupTableMismatchedTypes@filter-rewrites case-crc:148021fd +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo, lookup.lookyloo l, numfoo +WHERE foo.cnt = l.k AND l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[=($0, CAST($1):BIGINT)]) + LogicalProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[=($0, CAST($1):BIGINT)]) + DruidProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[true], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "expression", + "expression" : "(\"cnt\" == CAST(\"j0.k\", 'LONG'))" + }, + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "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/testCommaJoinTableLookupTableMismatchedTypes@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@join-to-filter.iq new file mode 100644 index 00000000000..4f3f7571a22 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinTableLookupTableMismatchedTypes@join-to-filter.iq @@ -0,0 +1,151 @@ +# testCommaJoinTableLookupTableMismatchedTypes@join-to-filter case-crc:3eadb819 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo, lookup.lookyloo l, numfoo +WHERE foo.cnt = l.k AND l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[=($0, CAST($1):BIGINT)]) + LogicalProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[=($0, CAST($1):BIGINT)]) + DruidProject(cnt=[$0], k=[$1], k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[true], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "expression", + "expression" : "(\"cnt\" == CAST(\"j0.k\", 'LONG'))" + }, + "columns" : [ "cnt", "j0.k", "v0" ], + "columnTypes" : [ "LONG", "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "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/testFilterAndGroupByLookupUsingJoinOperator@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..f74626ec64e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@all_disabled@NullHandling=default.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperator@all_disabled@NullHandling=default case-crc:6ad9dbcf +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..48f7793b1e8 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@all_disabled@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testFilterAndGroupByLookupUsingJoinOperator@all_disabled@NullHandling=sql case-crc:6ad9dbcf +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..7d763dd4b63 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@all_enabled@NullHandling=default.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperator@all_enabled@NullHandling=default case-crc:b2f0e3d2 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..72386cc966d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@all_enabled@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testFilterAndGroupByLookupUsingJoinOperator@all_enabled@NullHandling=sql case-crc:b2f0e3d2 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@default@NullHandling=default.iq new file mode 100644 index 00000000000..2f1458b4d94 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@default@NullHandling=default.iq @@ -0,0 +1,98 @@ +# testFilterAndGroupByLookupUsingJoinOperator@default@NullHandling=default case-crc:ab2fdc58 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@default@NullHandling=sql.iq new file mode 100644 index 00000000000..e7882c737f6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@default@NullHandling=sql.iq @@ -0,0 +1,99 @@ +# testFilterAndGroupByLookupUsingJoinOperator@default@NullHandling=sql case-crc:ab2fdc58 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..d51f3f4c239 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperator@filter-on-value-column_disabled@NullHandling=default case-crc:f363558d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..9e87aa3f991 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testFilterAndGroupByLookupUsingJoinOperator@filter-on-value-column_disabled@NullHandling=sql case-crc:f363558d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..f486a02f040 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites-disabled@NullHandling=default case-crc:4e59d27f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..86587e2d85f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites-disabled@NullHandling=sql case-crc:4e59d27f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..03b77931754 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites@NullHandling=default case-crc:33065cd9 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..0e1160d4d54 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testFilterAndGroupByLookupUsingJoinOperator@filter-rewrites@NullHandling=sql case-crc:33065cd9 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..9716b7ab6d2 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@join-to-filter@NullHandling=default.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperator@join-to-filter@NullHandling=default case-crc:6decf339 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperator@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..410d0a5707e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperator@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testFilterAndGroupByLookupUsingJoinOperator@join-to-filter@NullHandling=sql case-crc:6decf339 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = 'xa' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ +| a | 2 | ++---+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..62f7d1a5be6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_disabled@NullHandling=default.iq @@ -0,0 +1,121 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_disabled@NullHandling=default case-crc:969035d8 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..0a766c31034 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_disabled@NullHandling=sql.iq @@ -0,0 +1,122 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_disabled@NullHandling=sql case-crc:969035d8 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..f7695328f30 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_enabled@NullHandling=default.iq @@ -0,0 +1,121 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_enabled@NullHandling=default case-crc:d55a66d3 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..157ffa39e6e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_enabled@NullHandling=sql.iq @@ -0,0 +1,122 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@all_enabled@NullHandling=sql case-crc:d55a66d3 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@default@NullHandling=default.iq new file mode 100644 index 00000000000..2dbf990aeae --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@default@NullHandling=default.iq @@ -0,0 +1,118 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@default@NullHandling=default case-crc:b7b60a5e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@default@NullHandling=sql.iq new file mode 100644 index 00000000000..15e03e9e967 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@default@NullHandling=sql.iq @@ -0,0 +1,119 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@default@NullHandling=sql case-crc:b7b60a5e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..c84f1a640cd --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,121 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-on-value-column_disabled@NullHandling=default case-crc:d9b90253 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..76db2a82cc5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,122 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-on-value-column_disabled@NullHandling=sql case-crc:d9b90253 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..d9c9900d9a3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,121 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites-disabled@NullHandling=default case-crc:194265c4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..6e73fa0cbef --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,122 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites-disabled@NullHandling=sql case-crc:194265c4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..1e53d7e699d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,121 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites@NullHandling=default case-crc:49af20a5 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..761bf019293 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,122 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@filter-rewrites@NullHandling=sql case-crc:49af20a5 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..1e34768d340 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@join-to-filter@NullHandling=default.iq @@ -0,0 +1,121 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@join-to-filter@NullHandling=default case-crc:5a34c7c1 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorBackwards@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..c029a14c277 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorBackwards@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,122 @@ +# testFilterAndGroupByLookupUsingJoinOperatorBackwards@join-to-filter@NullHandling=sql case-crc:5a34c7c1 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "dim2" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.dim2\")", + "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" : "v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..1532fe93ef1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_disabled@NullHandling=default.iq @@ -0,0 +1,104 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_disabled@NullHandling=default case-crc:99a0fb62 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..05005055e0c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_disabled@NullHandling=sql.iq @@ -0,0 +1,105 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_disabled@NullHandling=sql case-crc:99a0fb62 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..8a6b05916ac --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_enabled@NullHandling=default.iq @@ -0,0 +1,104 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_enabled@NullHandling=default case-crc:56c2eeea +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..bd3115e360e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_enabled@NullHandling=sql.iq @@ -0,0 +1,105 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@all_enabled@NullHandling=sql case-crc:56c2eeea +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@default@NullHandling=default.iq new file mode 100644 index 00000000000..ccdba478dbe --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@default@NullHandling=default.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@default@NullHandling=default case-crc:49550f8e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@default@NullHandling=sql.iq new file mode 100644 index 00000000000..24a981c6d7f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@default@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@default@NullHandling=sql case-crc:49550f8e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..e152ce672f3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,104 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-on-value-column_disabled@NullHandling=default case-crc:c3056aeb +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..0db5b7d1876 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,105 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-on-value-column_disabled@NullHandling=sql case-crc:c3056aeb +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..c97ba352aef --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,104 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites-disabled@NullHandling=default case-crc:c84d485e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..7f287172364 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,105 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites-disabled@NullHandling=sql case-crc:c84d485e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..b5cd6a9b7cf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,104 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites@NullHandling=default case-crc:16ce5a70 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..7aa073faaf9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,105 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@filter-rewrites@NullHandling=sql case-crc:16ce5a70 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..33b491c2f14 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@join-to-filter@NullHandling=default.iq @@ -0,0 +1,104 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@join-to-filter@NullHandling=default case-crc:0f1d69fc +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..59db4614b47 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,105 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter@join-to-filter@NullHandling=sql case-crc:0f1d69fc +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..7e399655709 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_disabled@NullHandling=default.iq @@ -0,0 +1,100 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_disabled@NullHandling=default case-crc:b506c194 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..0fea9c0817a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_disabled@NullHandling=sql.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_disabled@NullHandling=sql case-crc:b506c194 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..2eb9e3dd342 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_enabled@NullHandling=default.iq @@ -0,0 +1,100 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_enabled@NullHandling=default case-crc:7a64d41c +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..15ad5587a40 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_enabled@NullHandling=sql.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@all_enabled@NullHandling=sql case-crc:7a64d41c +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@default@NullHandling=default.iq new file mode 100644 index 00000000000..7acd6a610a1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@default@NullHandling=default.iq @@ -0,0 +1,97 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@default@NullHandling=default case-crc:65f33578 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@default@NullHandling=sql.iq new file mode 100644 index 00000000000..49f864a5ec8 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@default@NullHandling=sql.iq @@ -0,0 +1,98 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@default@NullHandling=sql case-crc:65f33578 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..97429f762d0 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,100 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-on-value-column_disabled@NullHandling=default case-crc:efa3501d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..f054f1fcb10 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-on-value-column_disabled@NullHandling=sql case-crc:efa3501d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..4b75484d921 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,100 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites-disabled@NullHandling=default case-crc:e4eb72a8 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..93a5d135023 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites-disabled@NullHandling=sql case-crc:e4eb72a8 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..820d83af355 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,100 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites@NullHandling=default case-crc:3a686086 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..2d63d026361 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@filter-rewrites@NullHandling=sql case-crc:3a686086 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..5bed173887f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@join-to-filter@NullHandling=default.iq @@ -0,0 +1,100 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@join-to-filter@NullHandling=default case-crc:23bb530a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..f4a5ca3cd0f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,101 @@ +# testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothing@join-to-filter@NullHandling=sql case-crc:23bb530a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.k, COUNT(*) +FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k +WHERE lookyloo.v = '123' +GROUP BY lookyloo.k; ++---+--------+ +| k | EXPR$1 | ++---+--------+ ++---+--------+ +(0 rows) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, '123')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, '123')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "123" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.k", + "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/testInnerJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..6217c2a18e9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=default.iq @@ -0,0 +1,106 @@ +# testInnerJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=default case-crc:ab8e838b +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..a8d6bb7ebdd --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=sql.iq @@ -0,0 +1,107 @@ +# testInnerJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=sql case-crc:ab8e838b +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..23a5053dcb8 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=default.iq @@ -0,0 +1,106 @@ +# testInnerJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=default case-crc:77939a13 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..0ba46744a82 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=sql.iq @@ -0,0 +1,107 @@ +# testInnerJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=sql case-crc:77939a13 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@default@NullHandling=default.iq new file mode 100644 index 00000000000..0dfb9013551 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@default@NullHandling=default.iq @@ -0,0 +1,103 @@ +# testInnerJoinSubqueryWithSelectorFilter@default@NullHandling=default case-crc:8fb61cd8 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@default@NullHandling=sql.iq new file mode 100644 index 00000000000..70d30179c9f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@default@NullHandling=sql.iq @@ -0,0 +1,104 @@ +# testInnerJoinSubqueryWithSelectorFilter@default@NullHandling=sql case-crc:8fb61cd8 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..ba94c15f690 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,106 @@ +# testInnerJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=default case-crc:cf03d9f0 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..17c679a8095 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,107 @@ +# testInnerJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=sql case-crc:cf03d9f0 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..a9a953ea7fa --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,106 @@ +# testInnerJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=default case-crc:9cf0e315 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..80f1859cd24 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,107 @@ +# testInnerJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=sql case-crc:9cf0e315 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..99f56f4cea9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,106 @@ +# testInnerJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=default case-crc:39eaeb27 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..c0f92aa3630 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,107 @@ +# testInnerJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=sql case-crc:39eaeb27 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..634c03424a9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=default.iq @@ -0,0 +1,106 @@ +# testInnerJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=default case-crc:5c4107c7 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..1cb08d2f91f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,107 @@ +# testInnerJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=sql case-crc:5c4107c7 +# quidem testcase reason: FILTER_PUSHED_DOWN_FROM_JOIN_CAN_BE_MORE +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l1.k FROM foo INNER JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k and l1.k = 'abc'; ++------+-----+ +| dim1 | k | ++------+-----+ +| abc | abc | ++------+-----+ +(1 row) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..e877cf70afb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_disabled@NullHandling=default.iq @@ -0,0 +1,115 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_disabled@NullHandling=default case-crc:c0bbdf34 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..dd6bc7df891 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_disabled@NullHandling=sql.iq @@ -0,0 +1,116 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_disabled@NullHandling=sql case-crc:c0bbdf34 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..d32b7d9edd6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_enabled@NullHandling=default.iq @@ -0,0 +1,115 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_enabled@NullHandling=default case-crc:a33b5d02 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..998268eebef --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_enabled@NullHandling=sql.iq @@ -0,0 +1,116 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@all_enabled@NullHandling=sql case-crc:a33b5d02 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@default@NullHandling=default.iq new file mode 100644 index 00000000000..4fc3cf4d651 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@default@NullHandling=default.iq @@ -0,0 +1,112 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@default@NullHandling=default case-crc:1e1b5255 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@default@NullHandling=sql.iq new file mode 100644 index 00000000000..7dae799497a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@default@NullHandling=sql.iq @@ -0,0 +1,113 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@default@NullHandling=sql case-crc:1e1b5255 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..34401a59615 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,115 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-on-value-column_disabled@NullHandling=default case-crc:644a2686 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..5bd1e509cd1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,116 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-on-value-column_disabled@NullHandling=sql case-crc:644a2686 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..224c51f36d4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,115 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites-disabled@NullHandling=default case-crc:ff4064d2 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..b399c29618c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,116 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites-disabled@NullHandling=sql case-crc:ff4064d2 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..e17a0911fec --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,115 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites@NullHandling=default case-crc:77fd82e0 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..0457d233041 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,116 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@filter-rewrites@NullHandling=sql case-crc:77fd82e0 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..cf3cda46c82 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@join-to-filter@NullHandling=default.iq @@ -0,0 +1,115 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@join-to-filter@NullHandling=default case-crc:c06e2be4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..a42ad1c52dc --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimit@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,116 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimit@join-to-filter@NullHandling=sql case-crc:c06e2be4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..5b29f09d9cf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_disabled@NullHandling=default.iq @@ -0,0 +1,113 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_disabled@NullHandling=default case-crc:42c7f2c6 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..bcacc5f1566 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_disabled@NullHandling=sql.iq @@ -0,0 +1,114 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_disabled@NullHandling=sql case-crc:42c7f2c6 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..2a9913cde1c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_enabled@NullHandling=default.iq @@ -0,0 +1,113 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_enabled@NullHandling=default case-crc:250bb174 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..125a27b5d74 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_enabled@NullHandling=sql.iq @@ -0,0 +1,114 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@all_enabled@NullHandling=sql case-crc:250bb174 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@default@NullHandling=default.iq new file mode 100644 index 00000000000..1784d49f2de --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@default@NullHandling=default.iq @@ -0,0 +1,110 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@default@NullHandling=default case-crc:f4ae4910 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@default@NullHandling=sql.iq new file mode 100644 index 00000000000..64960daffdd --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@default@NullHandling=sql.iq @@ -0,0 +1,111 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@default@NullHandling=sql case-crc:f4ae4910 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..0fc76ae56bc --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,113 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=default case-crc:492f69c4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..dff0889fc20 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,114 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=sql case-crc:492f69c4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..7a65f384aa0 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,113 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites-disabled@NullHandling=default case-crc:c0ea913a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..2fb3483a658 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,114 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites-disabled@NullHandling=sql case-crc:c0ea913a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..89f63abf345 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,113 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites@NullHandling=default case-crc:72a94276 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..43c5077875f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,114 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@filter-rewrites@NullHandling=sql case-crc:72a94276 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..a6e56829354 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@join-to-filter@NullHandling=default.iq @@ -0,0 +1,113 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@join-to-filter@NullHandling=default case-crc:5f194714 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..7c1afeb6e2d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,114 @@ +# testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns@join-to-filter@NullHandling=sql case-crc:5f194714 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +LIMIT 100 +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalSort(fetch=[100]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidSort(fetch=[100], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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", + "limit" : 100, + "columns" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..5a1dd26cf99 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_disabled@NullHandling=default.iq @@ -0,0 +1,111 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_disabled@NullHandling=default case-crc:3cccedd2 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..4de06488b7c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_disabled@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_disabled@NullHandling=sql case-crc:3cccedd2 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..d1be2e043f5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_enabled@NullHandling=default.iq @@ -0,0 +1,111 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_enabled@NullHandling=default case-crc:dc8f333d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..1b8503d03f5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_enabled@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@all_enabled@NullHandling=sql case-crc:dc8f333d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@default@NullHandling=default.iq new file mode 100644 index 00000000000..b11e26248ca --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@default@NullHandling=default.iq @@ -0,0 +1,108 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@default@NullHandling=default case-crc:44e3029e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@default@NullHandling=sql.iq new file mode 100644 index 00000000000..38c3346b1db --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@default@NullHandling=sql.iq @@ -0,0 +1,109 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@default@NullHandling=sql case-crc:44e3029e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..75d246bf4ac --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,111 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-on-value-column_disabled@NullHandling=default case-crc:14a70534 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..1ea11b33874 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-on-value-column_disabled@NullHandling=sql case-crc:14a70534 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..6597bd7ebc5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,111 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites-disabled@NullHandling=default case-crc:5f02f6af +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..7a2cd28df4d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites-disabled@NullHandling=sql case-crc:5f02f6af +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..4a25441359e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,111 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites@NullHandling=default case-crc:4616fcc4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..41907829d62 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@filter-rewrites@NullHandling=sql case-crc:4616fcc4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..ba892f05ec1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@join-to-filter@NullHandling=default.iq @@ -0,0 +1,111 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@join-to-filter@NullHandling=default case-crc:03b02640 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..ac41eb795d8 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimit@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimit@join-to-filter@NullHandling=sql case-crc:03b02640 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..a582ab52ecc --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_disabled@NullHandling=default.iq @@ -0,0 +1,109 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_disabled@NullHandling=default case-crc:bd0bd8bf +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..2e8b50eae78 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_disabled@NullHandling=sql.iq @@ -0,0 +1,110 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_disabled@NullHandling=sql case-crc:bd0bd8bf +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..1355671848c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_enabled@NullHandling=default.iq @@ -0,0 +1,109 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_enabled@NullHandling=default case-crc:5b16ab5b +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..7092bd37643 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_enabled@NullHandling=sql.iq @@ -0,0 +1,110 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@all_enabled@NullHandling=sql case-crc:5b16ab5b +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@default@NullHandling=default.iq new file mode 100644 index 00000000000..0c3545233a9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@default@NullHandling=default.iq @@ -0,0 +1,106 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@default@NullHandling=default case-crc:f2c3960e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@default@NullHandling=sql.iq new file mode 100644 index 00000000000..0fa857e8918 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@default@NullHandling=sql.iq @@ -0,0 +1,107 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@default@NullHandling=sql case-crc:f2c3960e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..ff35fbe2cfa --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,109 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=default case-crc:8706fb61 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..4cd0c859123 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,110 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-on-value-column_disabled@NullHandling=sql case-crc:8706fb61 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..457e26b259f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,109 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites-disabled@NullHandling=default case-crc:efab2e71 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..e8620a6b747 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,110 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites-disabled@NullHandling=sql case-crc:efab2e71 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..cfdb7dcc062 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,109 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites@NullHandling=default case-crc:5a4bfe05 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..1a5340f1f01 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,110 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@filter-rewrites@NullHandling=sql case-crc:5a4bfe05 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..47afb97d104 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@join-to-filter@NullHandling=default.iq @@ -0,0 +1,109 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@join-to-filter@NullHandling=default case-crc:aff4ddad +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..cae65ad23fb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,110 @@ +# testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns@join-to-filter@NullHandling=sql case-crc:aff4ddad +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +WHERE l.v = 'xa' +; ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| __time | cnt | dim1 | dim2 | dim3 | m1 | m2 | unique_dim1 | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +| 2000-01-01 00:00:00.000 | 1 | | a | ["a","b"] | 1.0 | 1.0 | "AQAAAEAAAA==" | +| 2001-01-01 00:00:00.000 | 1 | 1 | a | | 4.0 | 4.0 | "AQAAAQAAAAFREA==" | ++-------------------------+-----+------+------+-----------+-----+-----+--------------------+ +(2 rows) + +!ok +LogicalProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7]) + LogicalJoin(condition=[=($2, $10)], joinType=[inner]) + LogicalJoin(condition=[=($2, $8)], joinType=[inner]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(__time=[$0], cnt=[$4], dim1=[$1], dim2=[$2], dim3=[$3], m1=[$5], m2=[$6], unique_dim1=[$7], druid=[logical]) + DruidJoin(condition=[=($2, $10)], joinType=[inner]) + DruidJoin(condition=[=($2, $8)], joinType=[inner]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" : [ "__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1" ], + "columnTypes" : [ "LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..910efe62429 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_disabled@NullHandling=default.iq @@ -0,0 +1,275 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_disabled@NullHandling=default case-crc:41547876 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "A" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..01ad3ca130e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_disabled@NullHandling=sql.iq @@ -0,0 +1,305 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_disabled@NullHandling=sql case-crc:41547876 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "A" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..541186def46 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_enabled@NullHandling=default.iq @@ -0,0 +1,275 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_enabled@NullHandling=default case-crc:edf72f8f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "A" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..30d480efebb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_enabled@NullHandling=sql.iq @@ -0,0 +1,305 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@all_enabled@NullHandling=sql case-crc:edf72f8f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "A" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@default@NullHandling=default.iq new file mode 100644 index 00000000000..2b24b70d779 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@default@NullHandling=default.iq @@ -0,0 +1,272 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@default@NullHandling=default case-crc:580b0963 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "A" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@default@NullHandling=sql.iq new file mode 100644 index 00000000000..c5df737ee4b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@default@NullHandling=sql.iq @@ -0,0 +1,302 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@default@NullHandling=sql case-crc:580b0963 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "A" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..a1554d1216e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,275 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-on-value-column_disabled@NullHandling=default case-crc:cabec9b7 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "A" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..ad0ff57854a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,305 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-on-value-column_disabled@NullHandling=sql case-crc:cabec9b7 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "A" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..875f6397d10 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,275 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites-disabled@NullHandling=default case-crc:32c73cb4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "A" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..6005240994a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,305 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites-disabled@NullHandling=sql case-crc:32c73cb4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "A" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..2c24f9fd43a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,275 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites@NullHandling=default case-crc:0cd533f3 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "A" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..2c0f5acd75d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,305 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@filter-rewrites@NullHandling=sql case-crc:0cd533f3 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "A" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..95f337e744d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@join-to-filter@NullHandling=default.iq @@ -0,0 +1,275 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@join-to-filter@NullHandling=default case-crc:bf629ea1 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "A" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..33b928001a9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,305 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults@join-to-filter@NullHandling=sql case-crc:bf629ea1 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('A', 'B'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++----+----+ +| m1 | m1 | ++----+----+ ++----+----+ +(0 rows) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, 'A'), =($1, 'B')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "A" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "B" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..27c694d3761 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_disabled@NullHandling=default.iq @@ -0,0 +1,276 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_disabled@NullHandling=default case-crc:01ca7eac +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "1" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..6ad76f4adfe --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_disabled@NullHandling=sql.iq @@ -0,0 +1,306 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_disabled@NullHandling=sql case-crc:01ca7eac +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "1" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..f178bb77913 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_enabled@NullHandling=default.iq @@ -0,0 +1,276 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_enabled@NullHandling=default case-crc:ad692955 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "1" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..16631807596 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_enabled@NullHandling=sql.iq @@ -0,0 +1,306 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@all_enabled@NullHandling=sql case-crc:ad692955 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "1" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@default@NullHandling=default.iq new file mode 100644 index 00000000000..8bcef0971a3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@default@NullHandling=default.iq @@ -0,0 +1,273 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@default@NullHandling=default case-crc:18950fb9 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "1" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@default@NullHandling=sql.iq new file mode 100644 index 00000000000..5e06a06055a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@default@NullHandling=sql.iq @@ -0,0 +1,303 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@default@NullHandling=sql case-crc:18950fb9 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "1" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..c9ec614dd6c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,276 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-on-value-column_disabled@NullHandling=default case-crc:8a20cf6d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "1" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..2b0a91f52ee --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,306 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-on-value-column_disabled@NullHandling=sql case-crc:8a20cf6d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "1" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..244271ad3f5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,276 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites-disabled@NullHandling=default case-crc:72593a6e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "1" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..b317aac1707 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,306 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites-disabled@NullHandling=sql case-crc:72593a6e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "1" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..5e79404a0b5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,276 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites@NullHandling=default case-crc:4c4b3529 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "1" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..79dc2bbc12e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,306 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@filter-rewrites@NullHandling=sql case-crc:4c4b3529 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "1" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..b905026fcc5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@join-to-filter@NullHandling=default.iq @@ -0,0 +1,276 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@join-to-filter@NullHandling=default case-crc:fffc987b +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "D" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "C" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "E" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim2", + "value" : "Q" + }, { + "type" : "in", + "dimension" : "dim1", + "values" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "1" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "D" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "I" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "J" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Q" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "Z" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "U" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "dim1", + "value" : "X" + }, { + "type" : "selector", + "dimension" : "dim2", + "value" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..f1249ac37bc --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,306 @@ +# testInnerJoinWithFilterPushdownAndManyFiltersNonEmptyResults@join-to-filter@NullHandling=sql case-crc:fffc987b +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT f1.m1, f2.m1 +FROM foo f1 +INNER JOIN foo f2 ON f1.m1 = f2.m1 where (f1.dim1, f1.dim2) in (('1', 'a'), ('C', 'D'), ('A', 'C'), ('C', 'E'), ('D', 'H'), ('A', 'D'), ('B', 'C'), +('H', 'E'), ('I', 'J'), ('I', 'K'), ('J', 'I'), ('Q', 'R'), ('Q', 'S'), ('S', 'Q'), ('X', 'Y'), ('Z', 'U'), ('U', 'Z'), ('P', 'Q'), ('X', 'A')) +; ++-----+-----+ +| m1 | m1 | ++-----+-----+ +| 4.0 | 4.0 | ++-----+-----+ +(1 row) + +!ok +LogicalProject(m1=[$2], m10=[$3]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + LogicalProject(dim1=[$1], dim2=[$2], m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(m1=[$2], m10=[$3], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[OR(AND(=($0, '1'), =($1, 'a')), AND(=($0, 'C'), =($1, 'D')), AND(=($0, 'A'), =($1, 'C')), AND(=($0, 'C'), =($1, 'E')), AND(=($0, 'D'), =($1, 'H')), AND(=($0, 'A'), =($1, 'D')), AND(=($0, 'B'), =($1, 'C')), AND(=($0, 'H'), =($1, 'E')), AND(=($0, 'I'), =($1, 'J')), AND(=($0, 'I'), =($1, 'K')), AND(=($0, 'J'), =($1, 'I')), AND(=($0, 'Q'), =($1, 'R')), AND(=($0, 'Q'), =($1, 'S')), AND(=($0, 'S'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'Y')), AND(=($0, 'Z'), =($1, 'U')), AND(=($0, 'U'), =($1, 'Z')), AND(=($0, 'P'), =($1, 'Q')), AND(=($0, 'X'), =($1, 'A')))]) + DruidProject(dim1=[$1], dim2=[$2], m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "C" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "C" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "A", "B" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "E" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "C", "H" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "P", "S" ] + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "1" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "a" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "D" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "H" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "J" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "I" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "K" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "J" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "I" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "R" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Q" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "S" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Y" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "Z" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "U" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "U" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "Z" + } ] + }, { + "type" : "and", + "fields" : [ { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "X" + }, { + "type" : "equals", + "column" : "dim2", + "matchValueType" : "STRING", + "matchValue" : "A" + } ] + } ] + }, + "columns" : [ "dim1", "dim2", "m1" ], + "columnTypes" : [ "STRING", "STRING", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "j0.m1", "m1" ], + "columnTypes" : [ "FLOAT", "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..91325d00f58 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_disabled@NullHandling=default.iq @@ -0,0 +1,89 @@ +# testInnerJoinWithIsNullFilter@all_disabled@NullHandling=default case-crc:d168560a +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim2", + "value" : null + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..5494c70717d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_disabled@NullHandling=sql.iq @@ -0,0 +1,88 @@ +# testInnerJoinWithIsNullFilter@all_disabled@NullHandling=sql case-crc:d168560a +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "null", + "column" : "dim2" + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..b786ccdd1b9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_enabled@NullHandling=default.iq @@ -0,0 +1,89 @@ +# testInnerJoinWithIsNullFilter@all_enabled@NullHandling=default case-crc:0ff7d829 +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim2", + "value" : null + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..71ff3c51315 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@all_enabled@NullHandling=sql.iq @@ -0,0 +1,88 @@ +# testInnerJoinWithIsNullFilter@all_enabled@NullHandling=sql case-crc:0ff7d829 +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "null", + "column" : "dim2" + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@default@NullHandling=default.iq new file mode 100644 index 00000000000..1e16acec1d9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@default@NullHandling=default.iq @@ -0,0 +1,86 @@ +# testInnerJoinWithIsNullFilter@default@NullHandling=default case-crc:a18273fc +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim2", + "value" : null + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@default@NullHandling=sql.iq new file mode 100644 index 00000000000..ac808026cd4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@default@NullHandling=sql.iq @@ -0,0 +1,85 @@ +# testInnerJoinWithIsNullFilter@default@NullHandling=sql case-crc:a18273fc +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "null", + "column" : "dim2" + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..cb40c7f8f5f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,89 @@ +# testInnerJoinWithIsNullFilter@filter-on-value-column_disabled@NullHandling=default case-crc:4376e2fd +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim2", + "value" : null + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..de1730371b6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,88 @@ +# testInnerJoinWithIsNullFilter@filter-on-value-column_disabled@NullHandling=sql case-crc:4376e2fd +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "null", + "column" : "dim2" + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..10746f2a112 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,89 @@ +# testInnerJoinWithIsNullFilter@filter-rewrites-disabled@NullHandling=default case-crc:bd065f57 +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim2", + "value" : null + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..5258e3f9c9a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,88 @@ +# testInnerJoinWithIsNullFilter@filter-rewrites-disabled@NullHandling=sql case-crc:bd065f57 +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "null", + "column" : "dim2" + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..e3ba40f56d1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,89 @@ +# testInnerJoinWithIsNullFilter@filter-rewrites@NullHandling=default case-crc:dc2fa169 +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim2", + "value" : null + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..0cd43017a00 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,88 @@ +# testInnerJoinWithIsNullFilter@filter-rewrites@NullHandling=sql case-crc:dc2fa169 +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "null", + "column" : "dim2" + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..c0b7ca271a3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@join-to-filter@NullHandling=default.iq @@ -0,0 +1,89 @@ +# testInnerJoinWithIsNullFilter@join-to-filter@NullHandling=default case-crc:1223f3d8 +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim2", + "value" : null + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..07a954368ef --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinWithIsNullFilter@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,88 @@ +# testInnerJoinWithIsNullFilter@join-to-filter@NullHandling=sql case-crc:1223f3d8 +# quidem testcase reason: EQUIV_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null; ++------+------+ +| dim1 | v | ++------+------+ +| abc | xabc | ++------+------+ +(1 row) + +!ok +LogicalProject(dim1=[$0], v=[$3]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[IS NULL($1)]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], v=[$3], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[IS NULL($1)]) + DruidProject(dim1=[$1], dim2=[$2], 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "null", + "column" : "dim2" + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..9db056cde3a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_disabled@NullHandling=default.iq @@ -0,0 +1,118 @@ +# testJoinUnionTablesOnLookup@all_disabled@NullHandling=default case-crc:0f858af9 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..f79216b870a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_disabled@NullHandling=sql.iq @@ -0,0 +1,119 @@ +# testJoinUnionTablesOnLookup@all_disabled@NullHandling=sql case-crc:0f858af9 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..d8920354d5f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_enabled@NullHandling=default.iq @@ -0,0 +1,118 @@ +# testJoinUnionTablesOnLookup@all_enabled@NullHandling=default case-crc:b6ca4d07 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..96aaf377741 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@all_enabled@NullHandling=sql.iq @@ -0,0 +1,119 @@ +# testJoinUnionTablesOnLookup@all_enabled@NullHandling=sql case-crc:b6ca4d07 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@default@NullHandling=default.iq new file mode 100644 index 00000000000..749b75cc5a3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@default@NullHandling=default.iq @@ -0,0 +1,115 @@ +# testJoinUnionTablesOnLookup@default@NullHandling=default case-crc:afdb4094 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@default@NullHandling=sql.iq new file mode 100644 index 00000000000..aca438fdfd4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@default@NullHandling=sql.iq @@ -0,0 +1,116 @@ +# testJoinUnionTablesOnLookup@default@NullHandling=sql case-crc:afdb4094 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..31e6ccdc5be --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,118 @@ +# testJoinUnionTablesOnLookup@filter-on-value-column_disabled@NullHandling=default case-crc:cc84f10f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..9cbca800f56 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,119 @@ +# testJoinUnionTablesOnLookup@filter-on-value-column_disabled@NullHandling=sql case-crc:cc84f10f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..35800f37074 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,118 @@ +# testJoinUnionTablesOnLookup@filter-rewrites-disabled@NullHandling=default case-crc:547b2f23 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..61a740559d9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,119 @@ +# testJoinUnionTablesOnLookup@filter-rewrites-disabled@NullHandling=sql case-crc:547b2f23 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..f0a6386c8a6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,118 @@ +# testJoinUnionTablesOnLookup@filter-rewrites@NullHandling=default case-crc:75977c99 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..c84c653bd99 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,119 @@ +# testJoinUnionTablesOnLookup@filter-rewrites@NullHandling=sql case-crc:75977c99 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..1b2b5b1db9c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@join-to-filter@NullHandling=default.iq @@ -0,0 +1,118 @@ +# testJoinUnionTablesOnLookup@join-to-filter@NullHandling=default case-crc:2965d141 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testJoinUnionTablesOnLookup@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..912b0521ddd --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinUnionTablesOnLookup@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,119 @@ +# testJoinUnionTablesOnLookup@join-to-filter@NullHandling=sql case-crc:2965d141 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT lookyloo.v, COUNT(*) +FROM + (SELECT dim2 FROM foo UNION ALL SELECT dim2 FROM numfoo) u + LEFT JOIN lookup.lookyloo ON u.dim2 = lookyloo.k +WHERE lookyloo.v <> 'xa' +GROUP BY lookyloo.v; ++------+--------+ +| v | EXPR$1 | ++------+--------+ +| xabc | 2 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{2}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalUnion(all=[true]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim2=[$2]) + LogicalTableScan(table=[[druid, numfoo]]) + LogicalFilter(condition=[<>($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidAggregate(group=[{2}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidUnion(all=[true]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + DruidFilter(condition=[<>($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "union", + "dataSources" : [ { + "type" : "table", + "name" : "foo" + }, { + "type" : "table", + "name" : "numfoo" + } ] + }, + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + } + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "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" : "j0.v", + "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/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=default.iq similarity index 71% rename from sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq rename to sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=default.iq index 2fb49c488d8..44ab3d35751 100644 --- 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@NullHandling=default.iq @@ -1,5 +1,5 @@ -# testLeftJoinSubqueryWithSelectorFilter@all_disabled case-crc:6d5ce04a -# quidem testcase reason: EQUIV_PLAN +# testLeftJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=default case-crc:69cf0894 +# quidem testcase reason: IMPROVED_PLAN !set computeInnerJoinCostAsFilter false !set debug true !set defaultTimeout 300000 @@ -25,20 +25,22 @@ WHERE l1.k = 'abc' (1 row) !ok -LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(dim1=[$1]) LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}]) - LogicalProject(k=[||($0, '')]) - LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) !logicalPlan -DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) +DruidJoin(condition=[=($0, $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]) + DruidFilter(condition=[=($0, 'abc')]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) !druidPlan { @@ -76,13 +78,22 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "outputName" : "d0", "outputType" : "STRING" } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, "limitSpec" : { "type" : "NoopLimitSpec" } } }, "rightPrefix" : "j0.", - "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "condition" : "(\"dim1\" == \"j0.d0\")", "joinType" : "INNER" }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..948f9d6ccc0 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testLeftJoinSubqueryWithSelectorFilter@all_disabled@NullHandling=sql case-crc:69cf0894 +# quidem testcase reason: IMPROVED_PLAN +!set computeInnerJoinCostAsFilter false +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!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=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!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@NullHandling=default.iq similarity index 71% rename from sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq rename to sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=default.iq index f8c20e42d6a..f0010104810 100644 --- 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@NullHandling=default.iq @@ -1,5 +1,5 @@ -# testLeftJoinSubqueryWithSelectorFilter@all_enabled case-crc:74a61554 -# quidem testcase reason: EQUIV_PLAN +# testLeftJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=default case-crc:0644d80d +# quidem testcase reason: IMPROVED_PLAN !set computeInnerJoinCostAsFilter false !set debug true !set defaultTimeout 300000 @@ -25,20 +25,22 @@ WHERE l1.k = 'abc' (1 row) !ok -LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(dim1=[$1]) LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}]) - LogicalProject(k=[||($0, '')]) - LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) !logicalPlan -DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) +DruidJoin(condition=[=($0, $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]) + DruidFilter(condition=[=($0, 'abc')]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) !druidPlan { @@ -76,13 +78,22 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "outputName" : "d0", "outputType" : "STRING" } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, "limitSpec" : { "type" : "NoopLimitSpec" } } }, "rightPrefix" : "j0.", - "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "condition" : "(\"dim1\" == \"j0.d0\")", "joinType" : "INNER" }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..4d11fcca977 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testLeftJoinSubqueryWithSelectorFilter@all_enabled@NullHandling=sql case-crc:0644d80d +# quidem testcase reason: IMPROVED_PLAN +!set computeInnerJoinCostAsFilter false +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!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=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!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@NullHandling=default.iq similarity index 70% rename from sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq rename to sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default@NullHandling=default.iq index a365f6899bf..506d51e7424 100644 --- 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@NullHandling=default.iq @@ -1,5 +1,5 @@ -# testLeftJoinSubqueryWithSelectorFilter@default case-crc:cbdfab50 -# quidem testcase reason: EQUIV_PLAN +# testLeftJoinSubqueryWithSelectorFilter@default@NullHandling=default case-crc:8c5a723c +# quidem testcase reason: IMPROVED_PLAN !set computeInnerJoinCostAsFilter false !set debug true !set defaultTimeout 300000 @@ -22,20 +22,22 @@ WHERE l1.k = 'abc' (1 row) !ok -LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(dim1=[$1]) LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}]) - LogicalProject(k=[||($0, '')]) - LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) !logicalPlan -DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) +DruidJoin(condition=[=($0, $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]) + DruidFilter(condition=[=($0, 'abc')]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) !druidPlan { @@ -73,13 +75,22 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "outputName" : "d0", "outputType" : "STRING" } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, "limitSpec" : { "type" : "NoopLimitSpec" } } }, "rightPrefix" : "j0.", - "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "condition" : "(\"dim1\" == \"j0.d0\")", "joinType" : "INNER" }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default@NullHandling=sql.iq new file mode 100644 index 00000000000..d52362bb637 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default@NullHandling=sql.iq @@ -0,0 +1,109 @@ +# testLeftJoinSubqueryWithSelectorFilter@default@NullHandling=sql case-crc:8c5a723c +# quidem testcase reason: IMPROVED_PLAN +!set computeInnerJoinCostAsFilter false +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!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=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!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@NullHandling=default.iq similarity index 72% rename from sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq rename to sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=default.iq index 398b3cc3a85..8e3703bf9c0 100644 --- 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@NullHandling=default.iq @@ -1,5 +1,5 @@ -# testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled case-crc:ca23bb81 -# quidem testcase reason: EQUIV_PLAN +# testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=default case-crc:38fee93f +# quidem testcase reason: IMPROVED_PLAN !set computeInnerJoinCostAsFilter false !set debug true !set defaultTimeout 300000 @@ -25,20 +25,22 @@ WHERE l1.k = 'abc' (1 row) !ok -LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(dim1=[$1]) LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}]) - LogicalProject(k=[||($0, '')]) - LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) !logicalPlan -DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) +DruidJoin(condition=[=($0, $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]) + DruidFilter(condition=[=($0, 'abc')]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) !druidPlan { @@ -76,13 +78,22 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "outputName" : "d0", "outputType" : "STRING" } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, "limitSpec" : { "type" : "NoopLimitSpec" } } }, "rightPrefix" : "j0.", - "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "condition" : "(\"dim1\" == \"j0.d0\")", "joinType" : "INNER" }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..f4dc5ee0c02 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled@NullHandling=sql case-crc:38fee93f +# quidem testcase reason: IMPROVED_PLAN +!set computeInnerJoinCostAsFilter false +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!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=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!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@NullHandling=default.iq similarity index 73% rename from sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq rename to sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=default.iq index 5c86e793f30..68477e65d17 100644 --- 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@NullHandling=default.iq @@ -1,5 +1,5 @@ -# testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled case-crc:c16b9934 -# quidem testcase reason: EQUIV_PLAN +# testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=default case-crc:33b6cb8a +# quidem testcase reason: IMPROVED_PLAN !set computeInnerJoinCostAsFilter false !set debug true !set defaultTimeout 300000 @@ -25,20 +25,22 @@ WHERE l1.k = 'abc' (1 row) !ok -LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(dim1=[$1]) LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}]) - LogicalProject(k=[||($0, '')]) - LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) !logicalPlan -DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) +DruidJoin(condition=[=($0, $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]) + DruidFilter(condition=[=($0, 'abc')]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) !druidPlan { @@ -76,13 +78,22 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "outputName" : "d0", "outputType" : "STRING" } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, "limitSpec" : { "type" : "NoopLimitSpec" } } }, "rightPrefix" : "j0.", - "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "condition" : "(\"dim1\" == \"j0.d0\")", "joinType" : "INNER" }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..6ed1d182124 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled@NullHandling=sql case-crc:33b6cb8a +# quidem testcase reason: IMPROVED_PLAN +!set computeInnerJoinCostAsFilter false +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!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=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!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@NullHandling=default.iq similarity index 71% rename from sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq rename to sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=default.iq index e1be3203710..1d531332385 100644 --- 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@NullHandling=default.iq @@ -1,5 +1,5 @@ -# testLeftJoinSubqueryWithSelectorFilter@filter-rewrites case-crc:1fe88b1a -# quidem testcase reason: EQUIV_PLAN +# testLeftJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=default case-crc:ed35d9a4 +# quidem testcase reason: IMPROVED_PLAN !set computeInnerJoinCostAsFilter false !set debug true !set defaultTimeout 300000 @@ -25,20 +25,22 @@ WHERE l1.k = 'abc' (1 row) !ok -LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(dim1=[$1]) LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}]) - LogicalProject(k=[||($0, '')]) - LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) !logicalPlan -DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) +DruidJoin(condition=[=($0, $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]) + DruidFilter(condition=[=($0, 'abc')]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) !druidPlan { @@ -76,13 +78,22 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "outputName" : "d0", "outputType" : "STRING" } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, "limitSpec" : { "type" : "NoopLimitSpec" } } }, "rightPrefix" : "j0.", - "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "condition" : "(\"dim1\" == \"j0.d0\")", "joinType" : "INNER" }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..0fea9905f6a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testLeftJoinSubqueryWithSelectorFilter@filter-rewrites@NullHandling=sql case-crc:ed35d9a4 +# quidem testcase reason: IMPROVED_PLAN +!set computeInnerJoinCostAsFilter false +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!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=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!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@NullHandling=default.iq similarity index 71% rename from sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq rename to sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=default.iq index 590296d4904..e00b99d6023 100644 --- 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@NullHandling=default.iq @@ -1,5 +1,5 @@ -# testLeftJoinSubqueryWithSelectorFilter@join-to-filter case-crc:04e8d03a -# quidem testcase reason: EQUIV_PLAN +# testLeftJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=default case-crc:e929dd69 +# quidem testcase reason: IMPROVED_PLAN !set computeInnerJoinCostAsFilter false !set debug true !set defaultTimeout 300000 @@ -25,20 +25,22 @@ WHERE l1.k = 'abc' (1 row) !ok -LogicalJoin(condition=[AND(=($0, $1), =($1, 'abc'))], joinType=[inner]) +LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(dim1=[$1]) LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}]) - LogicalProject(k=[||($0, '')]) - LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) !logicalPlan -DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) +DruidJoin(condition=[=($0, $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]) + DruidFilter(condition=[=($0, 'abc')]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(k=[||($0, '')], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) !druidPlan { @@ -76,13 +78,22 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "outputName" : "d0", "outputType" : "STRING" } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "selector", + "dimension" : "d0", + "value" : "abc" + }, + "finalize" : true + }, "limitSpec" : { "type" : "NoopLimitSpec" } } }, "rightPrefix" : "j0.", - "condition" : "((\"dim1\" == \"j0.d0\") && ('abc' == \"j0.d0\"))", + "condition" : "(\"dim1\" == \"j0.d0\")", "joinType" : "INNER" }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..941c40c6264 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,112 @@ +# testLeftJoinSubqueryWithSelectorFilter@join-to-filter@NullHandling=sql case-crc:e929dd69 +# quidem testcase reason: IMPROVED_PLAN +!set computeInnerJoinCostAsFilter false +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!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=[=($0, $1)], joinType=[inner]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($0, 'abc')]) + LogicalAggregate(group=[{0}]) + LogicalProject(k=[||($0, '')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($0, 'abc')]) + 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" + } ], + "having" : { + "type" : "filter", + "filter" : { + "type" : "equals", + "column" : "d0", + "matchValueType" : "STRING", + "matchValue" : "abc" + }, + "finalize" : true + }, + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..224cb623af4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_disabled@NullHandling=default.iq @@ -0,0 +1,400 @@ +# testManyManyInnerJoinOnManyManyLookup@all_disabled@NullHandling=default case-crc:90406045 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..d488371257f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_disabled@NullHandling=sql.iq @@ -0,0 +1,401 @@ +# testManyManyInnerJoinOnManyManyLookup@all_disabled@NullHandling=sql case-crc:90406045 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..47198190548 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_enabled@NullHandling=default.iq @@ -0,0 +1,400 @@ +# testManyManyInnerJoinOnManyManyLookup@all_enabled@NullHandling=default case-crc:0a05a3f9 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..2bdbad9010a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@all_enabled@NullHandling=sql.iq @@ -0,0 +1,401 @@ +# testManyManyInnerJoinOnManyManyLookup@all_enabled@NullHandling=sql case-crc:0a05a3f9 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@default@NullHandling=default.iq new file mode 100644 index 00000000000..17cdfa0724e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@default@NullHandling=default.iq @@ -0,0 +1,397 @@ +# testManyManyInnerJoinOnManyManyLookup@default@NullHandling=default case-crc:224906b6 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@default@NullHandling=sql.iq new file mode 100644 index 00000000000..418c0e1284e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@default@NullHandling=sql.iq @@ -0,0 +1,398 @@ +# testManyManyInnerJoinOnManyManyLookup@default@NullHandling=sql case-crc:224906b6 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..6733c2ac769 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,400 @@ +# testManyManyInnerJoinOnManyManyLookup@filter-on-value-column_disabled@NullHandling=default case-crc:8cedc32c +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..82771dab5b3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,401 @@ +# testManyManyInnerJoinOnManyManyLookup@filter-on-value-column_disabled@NullHandling=sql case-crc:8cedc32c +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..7c6b9a51056 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,400 @@ +# testManyManyInnerJoinOnManyManyLookup@filter-rewrites-disabled@NullHandling=default case-crc:f53f1b71 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..61fcca711b1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,401 @@ +# testManyManyInnerJoinOnManyManyLookup@filter-rewrites-disabled@NullHandling=sql case-crc:f53f1b71 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..6aa0ab84ef4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,400 @@ +# testManyManyInnerJoinOnManyManyLookup@filter-rewrites@NullHandling=default case-crc:15aa4614 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..98b55395617 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,401 @@ +# testManyManyInnerJoinOnManyManyLookup@filter-rewrites@NullHandling=sql case-crc:15aa4614 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..4fbb4d55fdb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@join-to-filter@NullHandling=default.iq @@ -0,0 +1,400 @@ +# testManyManyInnerJoinOnManyManyLookup@join-to-filter@NullHandling=default case-crc:3bba4901 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "v", + "value" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..b3356f563e9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testManyManyInnerJoinOnManyManyLookup@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,401 @@ +# testManyManyInnerJoinOnManyManyLookup@join-to-filter@NullHandling=sql case-crc:3bba4901 +# quidem testcase reason: IMPROVED_PLAN +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1 +FROM foo +INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k +INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k +INNER JOIN lookup.lookyloo l3 ON foo.dim2 = l3.k +INNER JOIN lookup.lookyloo l4 ON foo.dim2 = l4.k +INNER JOIN lookup.lookyloo l5 ON foo.dim2 = l5.k +INNER JOIN lookup.lookyloo l6 ON foo.dim2 = l6.k +INNER JOIN lookup.lookyloo l7 ON foo.dim2 = l7.k +INNER JOIN lookup.lookyloo l8 ON foo.dim2 = l8.k +INNER JOIN lookup.lookyloo l9 ON foo.dim2 = l9.k +INNER JOIN lookup.lookyloo l10 ON foo.dim2 = l10.k +INNER JOIN lookup.lookyloo l11 ON foo.dim2 = l11.k +INNER JOIN lookup.lookyloo l12 ON foo.dim2 = l12.k +INNER JOIN lookup.lookyloo l13 ON foo.dim2 = l13.k +INNER JOIN lookup.lookyloo l14 ON foo.dim2 = l14.k +INNER JOIN lookup.lookyloo l15 ON foo.dim2 = l15.k +INNER JOIN lookup.lookyloo l16 ON foo.dim2 = l16.k +INNER JOIN lookup.lookyloo l17 ON foo.dim2 = l17.k +INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k +INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k +WHERE l.v = 'xa' +; ++------+ +| dim1 | ++------+ +| | +| 1 | ++------+ +(2 rows) + +!ok +LogicalProject(dim1=[$0]) + LogicalJoin(condition=[=($1, $21)], joinType=[inner]) + LogicalJoin(condition=[=($1, $20)], joinType=[inner]) + LogicalJoin(condition=[=($1, $19)], joinType=[inner]) + LogicalJoin(condition=[=($1, $18)], joinType=[inner]) + LogicalJoin(condition=[=($1, $17)], joinType=[inner]) + LogicalJoin(condition=[=($1, $16)], joinType=[inner]) + LogicalJoin(condition=[=($1, $15)], joinType=[inner]) + LogicalJoin(condition=[=($1, $14)], joinType=[inner]) + LogicalJoin(condition=[=($1, $13)], joinType=[inner]) + LogicalJoin(condition=[=($1, $12)], joinType=[inner]) + LogicalJoin(condition=[=($1, $11)], joinType=[inner]) + LogicalJoin(condition=[=($1, $10)], joinType=[inner]) + LogicalJoin(condition=[=($1, $9)], joinType=[inner]) + LogicalJoin(condition=[=($1, $8)], joinType=[inner]) + LogicalJoin(condition=[=($1, $7)], joinType=[inner]) + LogicalJoin(condition=[=($1, $6)], joinType=[inner]) + LogicalJoin(condition=[=($1, $5)], joinType=[inner]) + LogicalJoin(condition=[=($1, $4)], joinType=[inner]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[=($1, 'xa')]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidProject(dim1=[$0], druid=[logical]) + DruidJoin(condition=[=($1, $21)], joinType=[inner]) + DruidJoin(condition=[=($1, $20)], joinType=[inner]) + DruidJoin(condition=[=($1, $19)], joinType=[inner]) + DruidJoin(condition=[=($1, $18)], joinType=[inner]) + DruidJoin(condition=[=($1, $17)], joinType=[inner]) + DruidJoin(condition=[=($1, $16)], joinType=[inner]) + DruidJoin(condition=[=($1, $15)], joinType=[inner]) + DruidJoin(condition=[=($1, $14)], joinType=[inner]) + DruidJoin(condition=[=($1, $13)], joinType=[inner]) + DruidJoin(condition=[=($1, $12)], joinType=[inner]) + DruidJoin(condition=[=($1, $11)], joinType=[inner]) + DruidJoin(condition=[=($1, $10)], joinType=[inner]) + DruidJoin(condition=[=($1, $9)], joinType=[inner]) + DruidJoin(condition=[=($1, $8)], joinType=[inner]) + DruidJoin(condition=[=($1, $7)], joinType=[inner]) + DruidJoin(condition=[=($1, $6)], joinType=[inner]) + DruidJoin(condition=[=($1, $5)], joinType=[inner]) + DruidJoin(condition=[=($1, $4)], joinType=[inner]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[=($1, 'xa')]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xa" + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim2\" == \"j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_j0.", + "condition" : "(\"dim2\" == \"_j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__j0.", + "condition" : "(\"dim2\" == \"__j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___j0.", + "condition" : "(\"dim2\" == \"___j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____j0.", + "condition" : "(\"dim2\" == \"____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____j0.", + "condition" : "(\"dim2\" == \"_____j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______j0.", + "condition" : "(\"dim2\" == \"______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______j0.", + "condition" : "(\"dim2\" == \"_______j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________j0.", + "condition" : "(\"dim2\" == \"________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________j0.", + "condition" : "(\"dim2\" == \"_________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "__________j0.", + "condition" : "(\"dim2\" == \"__________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "___________j0.", + "condition" : "(\"dim2\" == \"___________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "____________j0.", + "condition" : "(\"dim2\" == \"____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_____________j0.", + "condition" : "(\"dim2\" == \"_____________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "______________j0.", + "condition" : "(\"dim2\" == \"______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_______________j0.", + "condition" : "(\"dim2\" == \"_______________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "________________j0.", + "condition" : "(\"dim2\" == \"________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "_________________j0.", + "condition" : "(\"dim2\" == \"_________________j0.k\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "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" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..d7654db2683 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_disabled@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@all_disabled@NullHandling=default case-crc:4c1847a0 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..099cd78426f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_disabled@NullHandling=sql.iq @@ -0,0 +1,166 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@all_disabled@NullHandling=sql case-crc:4c1847a0 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..7b9e1e8c7b4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_enabled@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@all_enabled@NullHandling=default case-crc:d0070228 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..bb208e492bf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@all_enabled@NullHandling=sql.iq @@ -0,0 +1,166 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@all_enabled@NullHandling=sql case-crc:d0070228 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@default@NullHandling=default.iq new file mode 100644 index 00000000000..1d931c684cf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@default@NullHandling=default.iq @@ -0,0 +1,162 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@default@NullHandling=default case-crc:af596bbc +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@default@NullHandling=sql.iq new file mode 100644 index 00000000000..3e0dd5aa534 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@default@NullHandling=sql.iq @@ -0,0 +1,163 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@default@NullHandling=sql case-crc:af596bbc +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..8a3ac331ad5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@filter-on-value-column_disabled@NullHandling=default case-crc:f03f6096 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..a5fcb7974bf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,166 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@filter-on-value-column_disabled@NullHandling=sql case-crc:f03f6096 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..72706fda5ea --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites-disabled@NullHandling=default case-crc:d155bb65 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..2b1f2d94242 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,166 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites-disabled@NullHandling=sql case-crc:d155bb65 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..38fa8f55680 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites@NullHandling=default case-crc:9c16e907 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..e253673fb06 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,166 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@filter-rewrites@NullHandling=sql case-crc:9c16e907 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..0e6bb9c108f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@join-to-filter@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@join-to-filter@NullHandling=default case-crc:34b62ae1 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..043c4d0a48f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testNestedGroupByOnInlineDataSourceWithFilter@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,166 @@ +# testNestedGroupByOnInlineDataSourceWithFilter@join-to-filter@NullHandling=sql case-crc:34b62ae1 +# quidem testcase reason: IRRELEVANT_SCANQUERY +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +with abc as( SELECT dim1, m2 from druid.foo where "__time" >= '2001-01-02'), def as( SELECT t1.dim1, SUM(t2.m2) as "metricSum" from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1 where t1.dim1='def' group by 1)SELECT count(*) from def; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalAggregate(group=[{0}]) + LogicalProject(dim1=[CAST('def':VARCHAR):VARCHAR]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalFilter(condition=[=($0, 'def')]) + LogicalProject(dim1=[$1]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(dim1=[$1], m2=[$6]) + LogicalFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidAggregate(group=[{0}], druid=[logical]) + DruidProject(dim1=[CAST('def':VARCHAR):VARCHAR], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidFilter(condition=[=($0, 'def')]) + DruidProject(dim1=[$1], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(dim1=[$1], m2=[$6], druid=[logical]) + DruidFilter(condition=[>=($0, 2001-01-02 00:00:00)]) + 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" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "columns" : [ "dim1" ], + "columnTypes" : [ "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "m2" ], + "columnTypes" : [ "STRING", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"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" : "'def'", + "outputType" : "STRING" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "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" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..bc52d4060f8 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_disabled@NullHandling=default.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@all_disabled@NullHandling=default case-crc:d750dd9d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | +| abc | abc | xabc | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "v", + "value" : null + }, { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..2b46bd02a95 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_disabled@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@all_disabled@NullHandling=sql case-crc:d750dd9d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| abc | abc | xabc | +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "null", + "column" : "v" + }, { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..6700f37c42e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_enabled@NullHandling=default.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@all_enabled@NullHandling=default case-crc:1832c815 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | +| abc | abc | xabc | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "v", + "value" : null + }, { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..730a4733d4a --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@all_enabled@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@all_enabled@NullHandling=sql case-crc:1832c815 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| abc | abc | xabc | +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "null", + "column" : "v" + }, { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@default@NullHandling=default.iq new file mode 100644 index 00000000000..4ef9028bf40 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@default@NullHandling=default.iq @@ -0,0 +1,99 @@ +# testSelectOnLookupUsingRightJoinOperator@default@NullHandling=default case-crc:07a52971 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | +| abc | abc | xabc | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "v", + "value" : null + }, { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@default@NullHandling=sql.iq new file mode 100644 index 00000000000..2848eecb0b1 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@default@NullHandling=sql.iq @@ -0,0 +1,99 @@ +# testSelectOnLookupUsingRightJoinOperator@default@NullHandling=sql case-crc:07a52971 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| abc | abc | xabc | +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "null", + "column" : "v" + }, { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..d91b26ffcc3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@filter-on-value-column_disabled@NullHandling=default case-crc:8df54c14 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | +| abc | abc | xabc | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "v", + "value" : null + }, { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..74641f7d7fb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@filter-on-value-column_disabled@NullHandling=sql case-crc:8df54c14 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| abc | abc | xabc | +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "null", + "column" : "v" + }, { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..62b6fb52b4b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@filter-rewrites-disabled@NullHandling=default case-crc:86bd6ea1 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | +| abc | abc | xabc | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "v", + "value" : null + }, { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..ad50dcdbe47 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@filter-rewrites-disabled@NullHandling=sql case-crc:86bd6ea1 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| abc | abc | xabc | +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "null", + "column" : "v" + }, { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..53356639972 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@filter-rewrites@NullHandling=default case-crc:583e7c8f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | +| abc | abc | xabc | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "v", + "value" : null + }, { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..87cc45afce5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@filter-rewrites@NullHandling=sql case-crc:583e7c8f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| abc | abc | xabc | +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "null", + "column" : "v" + }, { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..5ebdb2e2ff9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@join-to-filter@NullHandling=default.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@join-to-filter@NullHandling=default case-crc:41ed4f03 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | +| abc | abc | xabc | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "v", + "value" : null + }, { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "v", + "value" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..2404814f177 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSelectOnLookupUsingRightJoinOperator@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,102 @@ +# testSelectOnLookupUsingRightJoinOperator@join-to-filter@NullHandling=sql case-crc:41ed4f03 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, lookyloo.* +FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k +WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL; ++------+-----------+--------------+ +| dim1 | k | v | ++------+-----------+--------------+ +| abc | abc | xabc | +| | 6 | x6 | +| | a | xa | +| | nosuchkey | mysteryvalue | ++------+-----------+--------------+ +(4 rows) + +!ok +LogicalJoin(condition=[=($0, $1)], joinType=[right]) + LogicalProject(dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + LogicalTableScan(table=[[lookup, lookyloo]]) + +!logicalPlan +DruidJoin(condition=[=($0, $1)], joinType=[right]) + DruidProject(dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidFilter(condition=[SEARCH($1, Sarg[(-∞..'xxx':VARCHAR), ('xxx':VARCHAR..+∞); NULL AS TRUE]:VARCHAR)]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "null", + "column" : "v" + }, { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "v", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + } ] + }, + "columns" : [ "k", "v" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"dim1\" == \"j0.k\")", + "joinType" : "RIGHT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "dim1", "j0.k", "j0.v" ], + "columnTypes" : [ "STRING", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..101a7b6c74d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_disabled@NullHandling=default.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_disabled@NullHandling=default case-crc:51e7cf34 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "_j0.a0", + "value" : null + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..6d0ffd3c3c9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_disabled@NullHandling=sql.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_disabled@NullHandling=sql case-crc:51e7cf34 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "null", + "column" : "_j0.a0" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..f46ed02d137 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_enabled@NullHandling=default.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_enabled@NullHandling=default case-crc:66cfe2c8 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "_j0.a0", + "value" : null + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..b733c4c086b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_enabled@NullHandling=sql.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@all_enabled@NullHandling=sql case-crc:66cfe2c8 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "null", + "column" : "_j0.a0" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@default@NullHandling=default.iq new file mode 100644 index 00000000000..f24615642df --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@default@NullHandling=default.iq @@ -0,0 +1,154 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@default@NullHandling=default case-crc:9311cae7 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "_j0.a0", + "value" : null + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@default@NullHandling=sql.iq new file mode 100644 index 00000000000..f3ecb3bea56 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@default@NullHandling=sql.iq @@ -0,0 +1,154 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@default@NullHandling=sql case-crc:9311cae7 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "null", + "column" : "_j0.a0" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..5b19c4c2563 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-on-value-column_disabled@NullHandling=default case-crc:f2169f67 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "_j0.a0", + "value" : null + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..7701a02bfee --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-on-value-column_disabled@NullHandling=sql case-crc:f2169f67 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "null", + "column" : "_j0.a0" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..37b054a823b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites-disabled@NullHandling=default case-crc:38c4fa5e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "_j0.a0", + "value" : null + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..9760704dac3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites-disabled@NullHandling=sql case-crc:38c4fa5e +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "null", + "column" : "_j0.a0" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..56a68a1d162 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites@NullHandling=default case-crc:9ff2e870 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "_j0.a0", + "value" : null + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..5fbc891a475 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@filter-rewrites@NullHandling=sql case-crc:9ff2e870 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "null", + "column" : "_j0.a0" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..889203c5566 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@join-to-filter@NullHandling=default.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@join-to-filter@NullHandling=default case-crc:0a72daeb +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "selector", + "dimension" : "_j0.a0", + "value" : null + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..937d559604c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,157 @@ +# testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins@join-to-filter@NullHandling=sql case-crc:0a72daeb +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM +foo +INNER JOIN (SELECT MAX(__time) t FROM foo) t0 on t0.t = foo.__time +LEFT JOIN (SELECT MIN(__time) t FROM foo) t1 on t1.t = foo.__time +WHERE dim1 IN ('abc', 'def') AND t1.t is null +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[IS NULL($3)]) + LogicalJoin(condition=[=($3, $0)], joinType=[left]) + LogicalJoin(condition=[=($2, $0)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], t=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[IS NULL($3)]) + DruidJoin(condition=[=($0, $3)], joinType=[left]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], t=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "null", + "column" : "_j0.a0" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..3b4612fe1fa --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_disabled@NullHandling=default.iq @@ -0,0 +1,237 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_disabled@NullHandling=default case-crc:d7ede89b +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "count", + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "_j0._a0", + "value" : "0" + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "__j0.p0", + "value" : null + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..dabbf8c9566 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_disabled@NullHandling=sql.iq @@ -0,0 +1,249 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_disabled@NullHandling=sql case-crc:d7ede89b +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "a0" + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "equals", + "column" : "_j0._a0", + "matchValueType" : "LONG", + "matchValue" : 0 + }, { + "type" : "and", + "fields" : [ { + "type" : "null", + "column" : "__j0.p0" + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..3b1114343cd --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_enabled@NullHandling=default.iq @@ -0,0 +1,237 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_enabled@NullHandling=default case-crc:588e02ac +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "count", + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "_j0._a0", + "value" : "0" + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "__j0.p0", + "value" : null + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..69fbdc2839e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_enabled@NullHandling=sql.iq @@ -0,0 +1,249 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@all_enabled@NullHandling=sql case-crc:588e02ac +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "a0" + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "equals", + "column" : "_j0._a0", + "matchValueType" : "LONG", + "matchValue" : 0 + }, { + "type" : "and", + "fields" : [ { + "type" : "null", + "column" : "__j0.p0" + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@default@NullHandling=default.iq new file mode 100644 index 00000000000..72af14b10af --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@default@NullHandling=default.iq @@ -0,0 +1,234 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@default@NullHandling=default case-crc:380b2789 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "count", + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "_j0._a0", + "value" : "0" + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "__j0.p0", + "value" : null + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@default@NullHandling=sql.iq new file mode 100644 index 00000000000..9e08a5e66bb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@default@NullHandling=sql.iq @@ -0,0 +1,246 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@default@NullHandling=sql case-crc:380b2789 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "a0" + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "equals", + "column" : "_j0._a0", + "matchValueType" : "LONG", + "matchValue" : 0 + }, { + "type" : "and", + "fields" : [ { + "type" : "null", + "column" : "__j0.p0" + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..55a081a200b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,237 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-on-value-column_disabled@NullHandling=default case-crc:13ce8fe9 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "count", + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "_j0._a0", + "value" : "0" + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "__j0.p0", + "value" : null + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..c3153057a2d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,249 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-on-value-column_disabled@NullHandling=sql case-crc:13ce8fe9 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "a0" + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "equals", + "column" : "_j0._a0", + "matchValueType" : "LONG", + "matchValue" : 0 + }, { + "type" : "and", + "fields" : [ { + "type" : "null", + "column" : "__j0.p0" + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..1669befcb8b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,237 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites-disabled@NullHandling=default case-crc:833d3669 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "count", + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "_j0._a0", + "value" : "0" + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "__j0.p0", + "value" : null + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..25c11967aa4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,249 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites-disabled@NullHandling=sql case-crc:833d3669 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "a0" + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "equals", + "column" : "_j0._a0", + "matchValueType" : "LONG", + "matchValue" : 0 + }, { + "type" : "and", + "fields" : [ { + "type" : "null", + "column" : "__j0.p0" + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..c1d251bc1e9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,237 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites@NullHandling=default case-crc:59c3074a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "count", + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "_j0._a0", + "value" : "0" + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "__j0.p0", + "value" : null + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..07f8519ba40 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,249 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@filter-rewrites@NullHandling=sql case-crc:59c3074a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "a0" + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "equals", + "column" : "_j0._a0", + "matchValueType" : "LONG", + "matchValue" : 0 + }, { + "type" : "and", + "fields" : [ { + "type" : "null", + "column" : "__j0.p0" + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..79ee3f8a6db --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@join-to-filter@NullHandling=default.iq @@ -0,0 +1,237 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@join-to-filter@NullHandling=default case-crc:fcf14bf8 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "count", + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "selector", + "dimension" : "_j0._a0", + "value" : "0" + }, { + "type" : "and", + "fields" : [ { + "type" : "selector", + "dimension" : "__j0.p0", + "value" : null + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..2b1c95734af --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,249 @@ +# testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery@join-to-filter@NullHandling=sql case-crc:fcf14bf8 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def') +AND __time IN (SELECT MAX(__time) FROM foo) +AND __time NOT IN (SELECT MIN(__time) FROM foo) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + LogicalJoin(condition=[=($0, $5)], joinType=[left]) + LogicalJoin(condition=[true], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(EXPR$0=[$0], i=[true]) + LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidFilter(condition=[OR(=($3, 0), AND(IS NULL($6), >=($4, $3)))]) + DruidJoin(condition=[=($0, $5)], joinType=[left]) + DruidJoin(condition=[true], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(EXPR$0=[$0], i=[true], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MIN($0)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "minTime", + "granularity" : { + "type" : "all" + } + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ ], + "aggregations" : [ { + "type" : "count", + "name" : "_a0" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "_a1" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "a0" + } + }, + "name" : "_a1" + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "1", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeseries", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "longMin", + "name" : "a0", + "fieldName" : "__time" + } ], + "postAggregations" : [ { + "type" : "expression", + "name" : "p0", + "expression" : "1", + "outputType" : "LONG" + } ] + } + }, + "rightPrefix" : "__j0.", + "condition" : "(\"__time\" == \"__j0.a0\")", + "joinType" : "LEFT" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "filter" : { + "type" : "or", + "fields" : [ { + "type" : "equals", + "column" : "_j0._a0", + "matchValueType" : "LONG", + "matchValue" : 0 + }, { + "type" : "and", + "fields" : [ { + "type" : "null", + "column" : "__j0.p0" + }, { + "type" : "expression", + "expression" : "(\"_j0._a1\" >= \"_j0._a0\")" + } ] + } ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "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/testSemiJoinWithOuterTimeExtractAggregateWithOrderBy@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractAggregateWithOrderBy@NullHandling=default.iq new file mode 100644 index 00000000000..396d37ca2e2 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractAggregateWithOrderBy@NullHandling=default.iq @@ -0,0 +1,154 @@ +# testSemiJoinWithOuterTimeExtractAggregateWithOrderBy@NullHandling=default case-crc:a25a8d86 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(DISTINCT dim1), EXTRACT(MONTH FROM __time) FROM druid.foo + WHERE dim2 IN ( + SELECT dim2 + FROM druid.foo + WHERE dim1 = 'def' + ) AND dim1 <> ''GROUP BY EXTRACT(MONTH FROM __time) +ORDER BY EXTRACT(MONTH FROM __time); ++--------+--------+ +| EXPR$0 | EXPR$1 | ++--------+--------+ +| 1 | 1 | ++--------+--------+ +(1 row) + +!ok +LogicalProject(EXPR$0=[$1], EXPR$1=[$0]) + LogicalSort(sort0=[$0], dir0=[ASC]) + LogicalAggregate(group=[{0}], EXPR$0=[COUNT(DISTINCT $1)]) + LogicalProject(EXPR$1=[EXTRACT(FLAG(MONTH), $0)], dim1=[$1]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[<>($1, '')]) + LogicalProject(__time=[$0], dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{2}]) + LogicalFilter(condition=[=($1, 'def')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(EXPR$0=[$1], EXPR$1=[$0], druid=[logical]) + DruidAggregate(group=[{0}], EXPR$0=[COUNT(DISTINCT $1)], druid=[logical]) + DruidProject(EXPR$1=[EXTRACT(FLAG(MONTH), $0)], dim1=[$1], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[<>($1, '')]) + DruidProject(__time=[$0], dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{2}], druid=[logical]) + DruidFilter(condition=[=($1, 'def')]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "columns" : [ "__time", "dim1", "dim2" ], + "columnTypes" : [ "LONG", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_extract(\"__time\",'MONTH','UTC')", + "outputType" : "LONG" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "aggregations" : [ { + "type" : "cardinality", + "name" : "a0", + "fields" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "dim1", + "outputType" : "STRING" + } ], + "byRow" : false, + "round" : true + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractAggregateWithOrderBy@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractAggregateWithOrderBy@NullHandling=sql.iq new file mode 100644 index 00000000000..2ac2cbd2830 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractAggregateWithOrderBy@NullHandling=sql.iq @@ -0,0 +1,156 @@ +# testSemiJoinWithOuterTimeExtractAggregateWithOrderBy@NullHandling=sql case-crc:a25a8d86 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(DISTINCT dim1), EXTRACT(MONTH FROM __time) FROM druid.foo + WHERE dim2 IN ( + SELECT dim2 + FROM druid.foo + WHERE dim1 = 'def' + ) AND dim1 <> ''GROUP BY EXTRACT(MONTH FROM __time) +ORDER BY EXTRACT(MONTH FROM __time); ++--------+--------+ +| EXPR$0 | EXPR$1 | ++--------+--------+ +| 1 | 1 | ++--------+--------+ +(1 row) + +!ok +LogicalProject(EXPR$0=[$1], EXPR$1=[$0]) + LogicalSort(sort0=[$0], dir0=[ASC]) + LogicalAggregate(group=[{0}], EXPR$0=[COUNT(DISTINCT $1)]) + LogicalProject(EXPR$1=[EXTRACT(FLAG(MONTH), $0)], dim1=[$1]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[<>($1, '')]) + LogicalProject(__time=[$0], dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{2}]) + LogicalFilter(condition=[=($1, 'def')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(EXPR$0=[$1], EXPR$1=[$0], druid=[logical]) + DruidAggregate(group=[{0}], EXPR$0=[COUNT(DISTINCT $1)], druid=[logical]) + DruidProject(EXPR$1=[EXTRACT(FLAG(MONTH), $0)], dim1=[$1], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[<>($1, '')]) + DruidProject(__time=[$0], dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{2}], druid=[logical]) + DruidFilter(condition=[=($1, 'def')]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "columns" : [ "__time", "dim1", "dim2" ], + "columnTypes" : [ "LONG", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_extract(\"__time\",'MONTH','UTC')", + "outputType" : "LONG" + } ], + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "v0", + "outputName" : "d0", + "outputType" : "LONG" + } ], + "aggregations" : [ { + "type" : "cardinality", + "name" : "a0", + "fields" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "dim1", + "outputType" : "STRING" + } ], + "byRow" : false, + "round" : true + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractScan@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractScan@NullHandling=default.iq new file mode 100644 index 00000000000..675d6a757be --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractScan@NullHandling=default.iq @@ -0,0 +1,131 @@ +# testSemiJoinWithOuterTimeExtractScan@NullHandling=default case-crc:f91c4c1d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, EXTRACT(MONTH FROM __time) FROM druid.foo + WHERE dim2 IN ( + SELECT dim2 + FROM druid.foo + WHERE dim1 = 'def' + ) AND dim1 <> ''; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| def | 1 | ++------+--------+ +(1 row) + +!ok +LogicalProject(dim1=[$1], EXPR$1=[EXTRACT(FLAG(MONTH), $0)]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[<>($1, '')]) + LogicalProject(__time=[$0], dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{2}]) + LogicalFilter(condition=[=($1, 'def')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[$1], EXPR$1=[EXTRACT(FLAG(MONTH), $0)], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[<>($1, '')]) + DruidProject(__time=[$0], dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{2}], druid=[logical]) + DruidFilter(condition=[=($1, 'def')]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : null + } + }, + "columns" : [ "__time", "dim1", "dim2" ], + "columnTypes" : [ "LONG", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "selector", + "dimension" : "dim1", + "value" : "def" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_extract(\"__time\",'MONTH','UTC')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractScan@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractScan@NullHandling=sql.iq new file mode 100644 index 00000000000..a76916e1296 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testSemiJoinWithOuterTimeExtractScan@NullHandling=sql.iq @@ -0,0 +1,133 @@ +# testSemiJoinWithOuterTimeExtractScan@NullHandling=sql case-crc:f91c4c1d +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, EXTRACT(MONTH FROM __time) FROM druid.foo + WHERE dim2 IN ( + SELECT dim2 + FROM druid.foo + WHERE dim1 = 'def' + ) AND dim1 <> ''; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| def | 1 | ++------+--------+ +(1 row) + +!ok +LogicalProject(dim1=[$1], EXPR$1=[EXTRACT(FLAG(MONTH), $0)]) + LogicalJoin(condition=[=($2, $3)], joinType=[inner]) + LogicalFilter(condition=[<>($1, '')]) + LogicalProject(__time=[$0], dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{2}]) + LogicalFilter(condition=[=($1, 'def')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidProject(dim1=[$1], EXPR$1=[EXTRACT(FLAG(MONTH), $0)], druid=[logical]) + DruidJoin(condition=[=($2, $3)], joinType=[inner]) + DruidFilter(condition=[<>($1, '')]) + DruidProject(__time=[$0], dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{2}], druid=[logical]) + DruidFilter(condition=[=($1, 'def')]) + 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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "columns" : [ "__time", "dim1", "dim2" ], + "columnTypes" : [ "LONG", "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "def" + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "timestamp_extract(\"__time\",'MONTH','UTC')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "dim1", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..31676bf6107 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_disabled@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testTwoSemiJoinsSimultaneously@all_disabled@NullHandling=default case-crc:34b41cbf +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "1" + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "2" + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..6912ce74409 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_disabled@NullHandling=sql.iq @@ -0,0 +1,168 @@ +# testTwoSemiJoinsSimultaneously@all_disabled@NullHandling=sql case-crc:34b41cbf +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 1 + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 2 + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..e7a7c8a63d4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_enabled@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testTwoSemiJoinsSimultaneously@all_enabled@NullHandling=default case-crc:25c89e4f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "1" + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "2" + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..6ebb59c96d2 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@all_enabled@NullHandling=sql.iq @@ -0,0 +1,168 @@ +# testTwoSemiJoinsSimultaneously@all_enabled@NullHandling=sql case-crc:25c89e4f +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 1 + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 2 + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@default@NullHandling=default.iq new file mode 100644 index 00000000000..9251aac1602 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@default@NullHandling=default.iq @@ -0,0 +1,162 @@ +# testTwoSemiJoinsSimultaneously@default@NullHandling=default case-crc:3372d2de +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "1" + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "2" + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@default@NullHandling=sql.iq new file mode 100644 index 00000000000..bb2d3b4a591 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@default@NullHandling=sql.iq @@ -0,0 +1,165 @@ +# testTwoSemiJoinsSimultaneously@default@NullHandling=sql case-crc:3372d2de +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 1 + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 2 + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..f92d14a0a08 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testTwoSemiJoinsSimultaneously@filter-on-value-column_disabled@NullHandling=default case-crc:ede47e22 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "1" + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "2" + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..081e03cf048 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,168 @@ +# testTwoSemiJoinsSimultaneously@filter-on-value-column_disabled@NullHandling=sql case-crc:ede47e22 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 1 + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 2 + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..1ac6b640dd6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testTwoSemiJoinsSimultaneously@filter-rewrites-disabled@NullHandling=default case-crc:2db0dd99 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "1" + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "2" + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..b8a128c8c83 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,168 @@ +# testTwoSemiJoinsSimultaneously@filter-rewrites-disabled@NullHandling=sql case-crc:2db0dd99 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 1 + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 2 + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..48e3f064fb5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testTwoSemiJoinsSimultaneously@filter-rewrites@NullHandling=default case-crc:dfbbc9bc +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "1" + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "2" + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..aeb265a2e2f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,168 @@ +# testTwoSemiJoinsSimultaneously@filter-rewrites@NullHandling=sql case-crc:dfbbc9bc +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 1 + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 2 + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..acbdbb3e40f --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@join-to-filter@NullHandling=default.iq @@ -0,0 +1,165 @@ +# testTwoSemiJoinsSimultaneously@join-to-filter@NullHandling=default case-crc:a72f732c +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "in", + "dimension" : "dim1", + "values" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "1" + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "cnt", + "value" : "2" + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testTwoSemiJoinsSimultaneously@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..c457e64c123 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testTwoSemiJoinsSimultaneously@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,168 @@ +# testTwoSemiJoinsSimultaneously@join-to-filter@NullHandling=sql case-crc:a72f732c +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set enableTimeBoundaryPlanning true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, COUNT(*) FROM foo +WHERE dim1 IN ('abc', 'def')AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt = 1) +AND __time IN (SELECT MAX(__time) FROM foo WHERE cnt <> 2) +GROUP BY 1; ++------+--------+ +| dim1 | EXPR$1 | ++------+--------+ +| abc | 1 | ++------+--------+ +(1 row) + +!ok +LogicalAggregate(group=[{1}], EXPR$1=[COUNT()]) + LogicalJoin(condition=[=($0, $3)], joinType=[inner]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + LogicalProject(__time=[$0], dim1=[$1]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[=($4, 1)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) + LogicalFilter(condition=[<>($4, 2)]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{1}], EXPR$1=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $3)], joinType=[inner]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidFilter(condition=[SEARCH($1, Sarg['abc':VARCHAR, 'def':VARCHAR]:VARCHAR)]) + DruidProject(__time=[$0], dim1=[$1], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[=($4, 1)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[MAX($0)], druid=[logical]) + DruidFilter(condition=[<>($4, 2)]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "join", + "left" : { + "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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "inType", + "column" : "dim1", + "matchValueType" : "STRING", + "sortedValues" : [ "abc", "def" ] + }, + "columns" : [ "__time", "dim1" ], + "columnTypes" : [ "LONG", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 1 + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"__time\" == \"j0.a0\")", + "joinType" : "INNER" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "timeBoundary", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "bound" : "maxTime", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "cnt", + "matchValueType" : "LONG", + "matchValue" : 2 + } + }, + "granularity" : { + "type" : "all" + } + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"__time\" == \"_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" : "dim1", + "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/testUsingSubqueryAsPartOfAndFilter@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..8398bd47433 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_disabled@NullHandling=default.iq @@ -0,0 +1,152 @@ +# testUsingSubqueryAsPartOfAndFilter@all_disabled@NullHandling=default case-crc:5416b44a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..5bc8df831e2 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_disabled@NullHandling=sql.iq @@ -0,0 +1,154 @@ +# testUsingSubqueryAsPartOfAndFilter@all_disabled@NullHandling=sql case-crc:5416b44a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_enabled@NullHandling=default.iq new file mode 100644 index 00000000000..d00e48a310e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_enabled@NullHandling=default.iq @@ -0,0 +1,152 @@ +# testUsingSubqueryAsPartOfAndFilter@all_enabled@NullHandling=default case-crc:a7bb9ccc +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_enabled@NullHandling=sql.iq new file mode 100644 index 00000000000..392a11cecfb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@all_enabled@NullHandling=sql.iq @@ -0,0 +1,154 @@ +# testUsingSubqueryAsPartOfAndFilter@all_enabled@NullHandling=sql case-crc:a7bb9ccc +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@default@NullHandling=default.iq new file mode 100644 index 00000000000..2821adc1c6e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@default@NullHandling=default.iq @@ -0,0 +1,149 @@ +# testUsingSubqueryAsPartOfAndFilter@default@NullHandling=default case-crc:18fc370a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@default@NullHandling=sql.iq new file mode 100644 index 00000000000..7b0882069eb --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@default@NullHandling=sql.iq @@ -0,0 +1,151 @@ +# testUsingSubqueryAsPartOfAndFilter@default@NullHandling=sql case-crc:18fc370a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-on-value-column_disabled@NullHandling=default.iq new file mode 100644 index 00000000000..1987ab4c04b --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-on-value-column_disabled@NullHandling=default.iq @@ -0,0 +1,152 @@ +# testUsingSubqueryAsPartOfAndFilter@filter-on-value-column_disabled@NullHandling=default case-crc:ce3a443c +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-on-value-column_disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..f6e3943305d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-on-value-column_disabled@NullHandling=sql.iq @@ -0,0 +1,154 @@ +# testUsingSubqueryAsPartOfAndFilter@filter-on-value-column_disabled@NullHandling=sql case-crc:ce3a443c +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites-disabled@NullHandling=default.iq new file mode 100644 index 00000000000..706bec28edf --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites-disabled@NullHandling=default.iq @@ -0,0 +1,152 @@ +# testUsingSubqueryAsPartOfAndFilter@filter-rewrites-disabled@NullHandling=default case-crc:7acb032c +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites-disabled@NullHandling=sql.iq new file mode 100644 index 00000000000..208b080e226 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites-disabled@NullHandling=sql.iq @@ -0,0 +1,154 @@ +# testUsingSubqueryAsPartOfAndFilter@filter-rewrites-disabled@NullHandling=sql case-crc:7acb032c +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites@NullHandling=default.iq new file mode 100644 index 00000000000..f5c84712ef5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites@NullHandling=default.iq @@ -0,0 +1,152 @@ +# testUsingSubqueryAsPartOfAndFilter@filter-rewrites@NullHandling=default case-crc:d83db91a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites@NullHandling=sql.iq new file mode 100644 index 00000000000..759d73986dd --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@filter-rewrites@NullHandling=sql.iq @@ -0,0 +1,154 @@ +# testUsingSubqueryAsPartOfAndFilter@filter-rewrites@NullHandling=sql case-crc:d83db91a +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@join-to-filter@NullHandling=default.iq new file mode 100644 index 00000000000..7fca495c2b4 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@join-to-filter@NullHandling=default.iq @@ -0,0 +1,152 @@ +# testUsingSubqueryAsPartOfAndFilter@join-to-filter@NullHandling=default case-crc:d70bc6c4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "selector", + "dimension" : "dim1", + "value" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@join-to-filter@NullHandling=sql.iq new file mode 100644 index 00000000000..cb4dd3a2e99 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryAsPartOfAndFilter@join-to-filter@NullHandling=sql.iq @@ -0,0 +1,154 @@ +# testUsingSubqueryAsPartOfAndFilter@join-to-filter@NullHandling=sql case-crc:d70bc6c4 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim1, dim2, COUNT(*) FROM druid.foo +WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '') +AND dim1 <> 'xxx' +group by dim1, dim2 ORDER BY dim2; ++------+------+--------+ +| dim1 | dim2 | EXPR$2 | ++------+------+--------+ +| def | abc | 1 | ++------+------+--------+ +(1 row) + +!ok +LogicalSort(sort0=[$1], dir0=[ASC]) + LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) + LogicalJoin(condition=[=($1, $2)], joinType=[inner]) + LogicalFilter(condition=[<>($0, 'xxx')]) + LogicalProject(dim1=[$1], dim2=[$2]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{1}]) + LogicalFilter(condition=[<>($1, '')]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(sort0=[$1], dir0=[ASC], druid=[logical]) + DruidAggregate(group=[{0, 1}], EXPR$2=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($1, $2)], joinType=[inner]) + DruidFilter(condition=[<>($0, 'xxx')]) + DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{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" ] + }, + "resultFormat" : "compactedList", + "filter" : { + "type" : "not", + "field" : { + "type" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "xxx" + } + }, + "columns" : [ "dim1", "dim2" ], + "columnTypes" : [ "STRING", "STRING" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "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" : "equals", + "column" : "dim1", + "matchValueType" : "STRING", + "matchValue" : "" + } + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "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" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim1", + "outputName" : "d0", + "outputType" : "STRING" + }, { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d1", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ], + "limitSpec" : { + "type" : "default", + "columns" : [ { + "dimension" : "d1", + "direction" : "ascending", + "dimensionOrder" : { + "type" : "lexicographic" + } + } ] + } +} +!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 index 8c460cad242..4ab31f91d51 100644 --- 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 @@ -1,5 +1,5 @@ -# testVirtualColumnOnMVFilterJoinExpression@all_disabled case-crc:1d9875df -# quidem testcase reason: SLIGHTLY_WORSE_PLAN +# testVirtualColumnOnMVFilterJoinExpression@all_disabled case-crc:7b2fde84 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND !set debug true !set defaultTimeout 300000 !set enableJoinFilterRewrite false 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 index ed9c27dc2d5..3b22f5f38f8 100644 --- 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 @@ -1,5 +1,5 @@ -# testVirtualColumnOnMVFilterJoinExpression@all_enabled case-crc:83f9a216 -# quidem testcase reason: SLIGHTLY_WORSE_PLAN +# testVirtualColumnOnMVFilterJoinExpression@all_enabled case-crc:04b89712 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND !set debug true !set defaultTimeout 300000 !set enableJoinFilterRewrite true 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 index 4a3b1ddb5cf..e8b07156d63 100644 --- 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 @@ -1,5 +1,5 @@ -# testVirtualColumnOnMVFilterJoinExpression@default case-crc:d6301c80 -# quidem testcase reason: SLIGHTLY_WORSE_PLAN +# testVirtualColumnOnMVFilterJoinExpression@default case-crc:9cb8f271 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND !set debug true !set defaultTimeout 300000 !set maxScatterGatherBytes 9223372036854775807 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 index 5bc156562d7..c70baae11f5 100644 --- 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 @@ -1,5 +1,5 @@ -# testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled case-crc:98ed9bb6 -# quidem testcase reason: SLIGHTLY_WORSE_PLAN +# testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled case-crc:5ace43c5 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND !set debug true !set defaultTimeout 300000 !set enableJoinFilterRewrite true 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 index 8b2cffb58c2..5024bedbfd4 100644 --- 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 @@ -1,5 +1,5 @@ -# testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled case-crc:c960c4d6 -# quidem testcase reason: SLIGHTLY_WORSE_PLAN +# testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled case-crc:0b431ca5 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND !set debug true !set defaultTimeout 300000 !set enableJoinFilterRewrite false 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 index d196685bbe4..b333d65720f 100644 --- 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 @@ -1,5 +1,5 @@ -# testVirtualColumnOnMVFilterJoinExpression@filter-rewrites case-crc:ec24f5d6 -# quidem testcase reason: SLIGHTLY_WORSE_PLAN +# testVirtualColumnOnMVFilterJoinExpression@filter-rewrites case-crc:2e072da5 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND !set debug true !set defaultTimeout 300000 !set enableJoinFilterRewrite true 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 index d29ec916a0f..2ea206a46df 100644 --- 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 @@ -1,5 +1,5 @@ -# testVirtualColumnOnMVFilterJoinExpression@join-to-filter case-crc:8f235220 -# quidem testcase reason: SLIGHTLY_WORSE_PLAN +# testVirtualColumnOnMVFilterJoinExpression@join-to-filter case-crc:30584b55 +# quidem testcase reason: SLIGHTLY_WORSE_FILTER_PUSHED_TO_JOIN_OPERAND !set debug true !set defaultTimeout 300000 !set enableJoinFilterRewrite false 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 deleted file mode 100644 index 8dff354a6cb..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=default.iq +++ /dev/null @@ -1,144 +0,0 @@ -# testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=default case-crc:5a66a3be -# quidem testcase reason: SLIGHTLY_WORSE_PLAN -!set debug true -!set defaultTimeout 300000 -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!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" - }, { - "type" : "expression", - "name" : "v1", - "expression" : "case_searched(((\"__time\" >= 947005200000) && (\"__time\" < 1641402000000)),1,0)", - "outputType" : "LONG" - } ], - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "default", - "dimension" : "v0", - "outputName" : "d0", - "outputType" : "STRING" - } ], - "aggregations" : [ { - "type" : "longSum", - "name" : "a0", - "fieldName" : "v1" - }, { - "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 deleted file mode 100644 index 0a1c54da595..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=sql.iq +++ /dev/null @@ -1,145 +0,0 @@ -# testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=sql case-crc:5a66a3be -# quidem testcase reason: SLIGHTLY_WORSE_PLAN -!set debug true -!set defaultTimeout 300000 -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!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" - }, { - "type" : "expression", - "name" : "v1", - "expression" : "case_searched(((\"__time\" >= 947005200000) && (\"__time\" < 1641402000000)),1,0)", - "outputType" : "LONG" - } ], - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "default", - "dimension" : "v0", - "outputName" : "d0", - "outputType" : "STRING" - } ], - "aggregations" : [ { - "type" : "longSum", - "name" : "a0", - "fieldName" : "v1" - }, { - "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