Decoupled planning: improve join predicate handling (#17105)

* enforces to only allow supported predicates in join conditions
* fixed a recursive query building issue by caching the `source` in `DruidQueryGenerator`
* moved `DruidAggregateRemoveRedundancyRule.instance` higher up; as if `CoreRules.AGGREGATE_EXPAND_DISTINCT_AGGREGATES` runs earlier the resulting `GROUPING` might become invalid
This commit is contained in:
Zoltan Haindrich 2024-09-25 16:00:25 +02:00 committed by GitHub
parent 9132a65a48
commit 6f7e8ca74a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
321 changed files with 46819 additions and 422 deletions

View File

@ -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();
}

View File

@ -259,6 +259,7 @@ public class DruidQueryGenerator
final PartialDruidQuery partialDruidQuery;
final List<Vertex> inputs;
final JoinSupportTweaks jst;
private SourceDesc source;
public PDQVertex(PartialDruidQuery partialDruidQuery, List<Vertex> 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<SourceDesc> sourceDescs = new ArrayList<>();
for (Vertex inputVertex : inputs) {

View File

@ -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;
}
}

View File

@ -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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> 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

View File

@ -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()
{

View File

@ -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()
{

View File

@ -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\\.<init>"),
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<? extends Throwable> throwableClass;

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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<hyperUnique>" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -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<hyperUnique>" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

Some files were not shown because too many files have changed in this diff Show More