Enable quidem shadowing for decoupled testcases (#16431)

* Altered `QueryTestBuilder` to be able to switch to a backing quidem test
* added a small crc to ensure that the shadow testcase does not deviate from the original one
* Packaged all decoupled related things into a a single `DecoupledExtension` to reduce copy-paste
* `DecoupledTestConfig#quidemReason` must describe why its being used
* `DecoupledTestConfig#separateDefaultModeTest` can be used to make multiple case files based on `NullHandling` state
* fixed a cosmetic bug during decoupled join translation
* enhanced `!druidPlan` to report the final logical plan in non-decoupled mode as well
* add check to ensure that only supported params are present in a druidtest uri
* enabled shadow testcases for previously disabled testcases
This commit is contained in:
Zoltan Haindrich 2024-05-23 07:03:16 +02:00 committed by GitHub
parent 10ea88e5bf
commit 12f79acc7e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
216 changed files with 25612 additions and 88 deletions

View File

@ -20,10 +20,12 @@
package org.apache.druid.sql.calcite.planner;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.QueryContexts;
import org.joda.time.DateTimeZone;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
@ -428,4 +430,37 @@ public class PlannerConfig
return config;
}
}
/**
* Translates {@link PlannerConfig} settings into its equivalent QueryContext map.
*
* @throws DruidException if the translation is not possible.
*/
public Map<String, Object> getNonDefaultAsQueryContext()
{
Map<String, Object> overrides = new HashMap<>();
PlannerConfig def = new PlannerConfig();
if (def.useApproximateCountDistinct != useApproximateCountDistinct) {
overrides.put(
CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT,
String.valueOf(useApproximateCountDistinct)
);
}
if (def.useGroupingSetForExactDistinct != useGroupingSetForExactDistinct) {
overrides.put(
CTX_KEY_USE_GROUPING_SET_FOR_EXACT_DISTINCT,
String.valueOf(useGroupingSetForExactDistinct)
);
}
PlannerConfig newConfig = PlannerConfig.builder().withOverrides(overrides).build();
if (!equals(newConfig)) {
throw DruidException.defensive(
"Not all PlannerConfig options are not persistable as QueryContext keys!\nold: %s\nnew: %s",
this,
newConfig
);
}
return overrides;
}
}

View File

@ -588,7 +588,11 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
.plus(rootQueryRel.collation),
parameterized
);
handlerContext.hook().captureDruidRel(druidRel);
Hook.JAVA_PLAN.run(druidRel);
if (explain != null) {
return planExplanation(possiblyLimitedRoot, druidRel, true);
} else {

View File

@ -19,14 +19,19 @@
package org.apache.druid.sql.calcite.rel;
import com.google.common.collect.Iterables;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.plan.volcano.RelSubset;
import org.apache.calcite.rel.AbstractRelNode;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelShuttleImpl;
import org.apache.calcite.rel.RelWriter;
import org.apache.druid.server.QueryResponse;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import javax.annotation.Nullable;
import java.util.Set;
public abstract class DruidRel<T extends DruidRel<?>> extends AbstractRelNode
@ -122,4 +127,31 @@ public abstract class DruidRel<T extends DruidRel<?>> extends AbstractRelNode
* Get the set of names of table datasources read by this DruidRel
*/
public abstract Set<String> getDataSourceNames();
public final RelNode unwrapLogicalPlan()
{
return accept(new LogicalPlanUnwrapperShuttle());
}
private static class LogicalPlanUnwrapperShuttle extends RelShuttleImpl
{
@Override
public RelNode visit(RelNode other)
{
return super.visit(visitNode(other));
}
private RelNode visitNode(RelNode other)
{
if (other instanceof RelSubset) {
final RelSubset subset = (RelSubset) other;
return visitNode(Iterables.getFirst(subset.getRels(), null));
}
if (other instanceof DruidRel<?>) {
DruidRel<?> druidRel = (DruidRel<?>) other;
return druidRel.getPartialDruidQuery().leafRel();
}
return other;
}
}
}

View File

@ -54,7 +54,6 @@ public class DruidJoinRule extends ConverterRule
// reject the query in case the anaysis detected any issues
throw InvalidSqlInput.exception(analysis.errorStr);
}
return new DruidJoin(
join.getCluster(),
newTrait,
@ -67,7 +66,7 @@ public class DruidJoinRule extends ConverterRule
join.getRight(),
DruidLogicalConvention.instance()
),
join.getCondition(),
analysis.getConditionWithUnsupportedSubConditionsIgnored(join.getCluster().getRexBuilder()),
join.getVariablesSet(),
join.getJoinType()
);

View File

@ -0,0 +1,41 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.quidem;
import java.io.File;
public class DruidQTestInfo
{
public final File caseDir;
public final String testName;
public final String comment;
public DruidQTestInfo(File caseDir, String testName, String comment)
{
this.caseDir = caseDir;
this.testName = testName;
this.comment = comment;
}
public File getIQFile()
{
return new File(caseDir, testName + ".iq");
}
}

View File

@ -33,6 +33,7 @@ import org.apache.calcite.sql.SqlExplainLevel;
import org.apache.calcite.util.Util;
import org.apache.druid.query.Query;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.rel.DruidRel;
import org.apache.druid.sql.calcite.util.QueryLogHook;
import java.sql.ResultSet;
@ -171,6 +172,9 @@ public class DruidQuidemCommandHandler implements CommandHandler
}
for (RelNode node : logged) {
if (node instanceof DruidRel<?>) {
node = ((DruidRel) node).unwrapLogicalPlan();
}
String str = RelOptUtil.dumpPlan("", node, SqlExplainFormat.TEXT, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
x.echo(ImmutableList.of(str));
}

View File

@ -106,6 +106,7 @@ import org.joda.time.chrono.ISOChronology;
import org.junit.Assert;
import org.junit.internal.matchers.ThrowableMessageMatcher;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Named;
import org.junit.jupiter.api.extension.RegisterExtension;
import javax.annotation.Nullable;
@ -933,6 +934,12 @@ public class BaseCalciteQueryTest extends CalciteTestBase
{
return baseQueryContext;
}
@Override
public SqlTestFramework queryFramework()
{
return BaseCalciteQueryTest.this.queryFramework();
}
}
public enum ResultMatchMode
@ -1268,51 +1275,51 @@ public class BaseCalciteQueryTest extends CalciteTestBase
{
return new Object[] {
// default behavior
QUERY_CONTEXT_DEFAULT,
Named.of("default", QUERY_CONTEXT_DEFAULT),
// all rewrites enabled
new ImmutableMap.Builder<String, Object>()
Named.of("all_enabled", new ImmutableMap.Builder<String, Object>()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true)
.put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, true)
.put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, true)
.build(),
.build()),
// filter-on-value-column rewrites disabled, everything else enabled
new ImmutableMap.Builder<String, Object>()
Named.of("filter-on-value-column_disabled", new ImmutableMap.Builder<String, Object>()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, false)
.put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, true)
.put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, true)
.build(),
.build()),
// filter rewrites fully disabled, join-to-filter enabled
new ImmutableMap.Builder<String, Object>()
Named.of("join-to-filter", new ImmutableMap.Builder<String, Object>()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, false)
.put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, false)
.put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, true)
.build(),
.build()),
// filter rewrites disabled, but value column filters still set to true
// (it should be ignored and this should
// behave the same as the previous context)
new ImmutableMap.Builder<String, Object>()
Named.of("filter-rewrites-disabled", new ImmutableMap.Builder<String, Object>()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true)
.put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, false)
.put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, true)
.build(),
.build()),
// filter rewrites fully enabled, join-to-filter disabled
new ImmutableMap.Builder<String, Object>()
Named.of("filter-rewrites", new ImmutableMap.Builder<String, Object>()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true)
.put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, true)
.put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, false)
.build(),
.build()),
// all rewrites disabled
new ImmutableMap.Builder<String, Object>()
Named.of("all_disabled", new ImmutableMap.Builder<String, Object>()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, false)
.put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, false)
.put(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, false)
.build(),
.build()),
};
}

View File

@ -83,7 +83,7 @@ import org.apache.druid.segment.join.JoinType;
import org.apache.druid.segment.virtual.ListFilteredVirtualColumn;
import org.apache.druid.server.QueryLifecycle;
import org.apache.druid.server.security.Access;
import org.apache.druid.sql.calcite.DecoupledTestConfig.NativeQueryIgnore;
import org.apache.druid.sql.calcite.DecoupledTestConfig.QuidemTestCaseReason;
import org.apache.druid.sql.calcite.NotYetSupported.Modes;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.MinTopNThreshold;
import org.apache.druid.sql.calcite.expression.DruidExpression;
@ -94,7 +94,6 @@ import org.apache.druid.sql.calcite.util.CalciteTests;
import org.joda.time.DateTimeZone;
import org.joda.time.Period;
import org.junit.Assert;
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
@ -108,6 +107,7 @@ import java.util.Map;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assumptions.assumeFalse;
public class CalciteJoinQueryTest extends BaseCalciteQueryTest
{
@ -179,12 +179,11 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
// Adjust topN threshold, so that the topN engine keeps only 1 slot for aggregates, which should be enough
// to compute the query with limit 1.
@SqlTestFrameworkConfig.MinTopNThreshold(1)
@Test
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN, separateDefaultModeTest = true)
public void testExactTopNOnInnerJoinWithLimit()
{
Map<String, Object> context = new HashMap<>(QUERY_CONTEXT_DEFAULT);
@ -492,7 +491,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
}
@Test
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true)
public void testJoinOnTimeseriesWithFloorOnTime()
{
// Cannot vectorize JOIN operator.
@ -547,7 +546,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
}
@Test
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true)
public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime()
{
// Cannot vectorize JOIN operator.
@ -614,7 +613,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
}
@Test
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true)
public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues()
{
// Cannot vectorize JOIN operator.
@ -1529,7 +1528,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.FINALIZING_FIELD_ACCESS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.FINALIZING_FIELD_ACCESS)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testInnerJoinQueryOfLookup(Map<String, Object> queryContext)
@ -1609,7 +1608,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
}
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.DEFINETLY_WORSE_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.DEFINETLY_WORSE_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testInnerJoinQueryOfLookupRemovable(Map<String, Object> queryContext)
@ -1648,7 +1647,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testInnerJoinTwoLookupsToTableUsingNumericColumn(Map<String, Object> queryContext)
@ -1976,7 +1974,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
}
@Test
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS)
public void testCommaJoinLeftFunction()
{
testQuery(
@ -2151,7 +2149,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testInnerJoinCastLeft(Map<String, Object> queryContext)
@ -2280,7 +2278,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testInnerJoinLeftFunction(Map<String, Object> queryContext)
@ -2733,7 +2731,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testUsingSubqueryWithExtractionFns(Map<String, Object> queryContext)
@ -2937,7 +2935,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_LEFT_DIRECT_ACCESS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_LEFT_DIRECT_ACCESS, separateDefaultModeTest = true)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess(Map<String, Object> queryContext)
@ -3049,7 +3047,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_LEFT_DIRECT_ACCESS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_LEFT_DIRECT_ACCESS, separateDefaultModeTest = true)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess(Map<String, Object> queryContext)
@ -3151,7 +3149,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_LEFT_DIRECT_ACCESS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_LEFT_DIRECT_ACCESS, separateDefaultModeTest = true)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess(Map<String, Object> queryContext)
@ -3253,7 +3251,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_LEFT_DIRECT_ACCESS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_LEFT_DIRECT_ACCESS, separateDefaultModeTest = true)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess(Map<String, Object> queryContext)
@ -3355,9 +3353,9 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN, separateDefaultModeTest = true)
public void testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources(Map<String, Object> queryContext)
{
cannotVectorize();
@ -3440,7 +3438,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_LEFT_DIRECT_ACCESS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_LEFT_DIRECT_ACCESS, separateDefaultModeTest = true)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess(Map<String, Object> queryContext)
@ -3589,7 +3587,12 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
{
// JoinFilterAnalyzer bug causes incorrect results on this test in replace-with-default mode.
// This test case was originally added in https://github.com/apache/druid/pull/11434 with a note about this.
Assumptions.assumeFalse(NullHandling.replaceWithDefault() && QueryContext.of(queryContext).getEnableJoinFilterRewrite());
assumeFalse(NullHandling.replaceWithDefault() && QueryContext.of(queryContext).getEnableJoinFilterRewrite());
assumeFalse(
testBuilder().isDecoupledMode() && NullHandling.replaceWithDefault(),
"join condition not support in decoupled mode"
);
// Cannot vectorize due to 'concat' expression.
cannotVectorize();
@ -3670,9 +3673,9 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN)
public void testLeftJoinSubqueryWithSelectorFilter(Map<String, Object> queryContext)
{
// Cannot vectorize due to 'concat' expression.
@ -3876,7 +3879,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testInnerJoinSubqueryWithSelectorFilter(Map<String, Object> queryContext)
@ -4152,7 +4154,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins(Map<String, Object> queryContext)
@ -4701,7 +4702,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
cannotVectorize();
// We don't handle non-equi join conditions for non-sql compatible mode.
Assumptions.assumeFalse(NullHandling.replaceWithDefault());
assumeFalse(NullHandling.replaceWithDefault());
testQuery(
"SELECT x.m1, y.m1 FROM foo x INNER JOIN foo y ON x.m1 > y.m1",
@ -4764,7 +4765,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
cannotVectorize();
// We don't handle non-equi join conditions for non-sql compatible mode.
Assumptions.assumeFalse(NullHandling.replaceWithDefault());
assumeFalse(NullHandling.replaceWithDefault());
testQuery(
"SELECT x.m1, y.m1 FROM foo x INNER JOIN foo y ON x.m1 = y.m1 AND x.m1 + y.m1 = 6.0",
@ -5152,9 +5153,9 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN)
public void testTopNOnStringWithNonSortedOrUniqueDictionary(Map<String, Object> queryContext)
{
testQuery(
@ -5193,9 +5194,9 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EQUIV_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN)
public void testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim(Map<String, Object> queryContext)
{
@ -5234,7 +5235,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.SLIGHTLY_WORSE_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testVirtualColumnOnMVFilterJoinExpression(Map<String, Object> queryContext)
@ -5291,7 +5292,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.DEFINETLY_WORSE_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.DEFINETLY_WORSE_PLAN)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testVirtualColumnOnMVFilterMultiJoinExpression(Map<String, Object> queryContext)
@ -5652,6 +5653,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
@ParameterizedTest(name = "{0}")
public void testRegressionFilteredAggregatorsSubqueryJoins(Map<String, Object> queryContext)
{
assumeFalse(testBuilder().isDecoupledMode() && NullHandling.replaceWithDefault(), "not support in decoupled mode");
cannotVectorize();
testQuery(
"select\n" +
@ -6027,7 +6030,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
}
@Test
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.JOIN_FILTER_LOCATIONS)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.JOIN_FILTER_LOCATIONS, separateDefaultModeTest = true)
public void testJoinWithInputRefCondition()
{
cannotVectorize();

View File

@ -114,7 +114,7 @@ import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.join.JoinType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.sql.calcite.DecoupledTestConfig.NativeQueryIgnore;
import org.apache.druid.sql.calcite.DecoupledTestConfig.QuidemTestCaseReason;
import org.apache.druid.sql.calcite.NotYetSupported.Modes;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.filtration.Filtration;
@ -7495,7 +7495,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.AGG_COL_EXCHANGE)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.AGG_COL_EXCHANGE, separateDefaultModeTest = true)
@Test
public void testExactCountDistinctWithGroupingAndOtherAggregators()
{
@ -7550,7 +7550,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.AGG_COL_EXCHANGE)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.AGG_COL_EXCHANGE, separateDefaultModeTest = true)
@Test
public void testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin()
{
@ -8749,7 +8749,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.AGG_COL_EXCHANGE)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.AGG_COL_EXCHANGE)
@Test
public void testGroupBySortPushDown()
{
@ -8845,7 +8845,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN, separateDefaultModeTest = true)
@Test
public void testGroupByLimitPushdownExtraction()
{
@ -9292,7 +9292,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.SLIGHTLY_WORSE_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_PLAN, separateDefaultModeTest = true)
@SqlTestFrameworkConfig.NumMergeBuffers(3)
@Test
public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename()
@ -11134,7 +11134,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN, separateDefaultModeTest = true)
@Test
public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim()
{
@ -13269,7 +13269,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN)
@Test
public void testGroupByWithLiteralInSubqueryGrouping()
{
@ -13458,7 +13458,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN)
@Test
public void testRepeatedIdenticalVirtualExpressionGrouping()
{
@ -15032,7 +15032,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.AGGREGATE_REMOVE_NOT_FIRED)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.AGGREGATE_REMOVE_NOT_FIRED, separateDefaultModeTest = true)
@Test
public void testSubqueryTypeMismatchWithLiterals()
{
@ -15752,7 +15752,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.run();
}
@DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.SLIGHTLY_WORSE_PLAN)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.SLIGHTLY_WORSE_PLAN, separateDefaultModeTest = true)
@Test
public void testWindowingWithScanAndSort()
{

View File

@ -20,14 +20,20 @@
package org.apache.druid.sql.calcite;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.quidem.DruidQTestInfo;
import org.apache.druid.quidem.ProjectPathUtils;
import org.apache.druid.server.security.AuthConfig;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest.CalciteTestConfig;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.util.SqlTestFramework;
import org.junit.jupiter.api.extension.Extension;
import org.junit.jupiter.api.extension.BeforeEachCallback;
import org.junit.jupiter.api.extension.ExtensionContext;
public class DecoupledExtension implements Extension
import java.io.File;
public class DecoupledExtension implements BeforeEachCallback
{
private BaseCalciteQueryTest baseTest;
@ -36,6 +42,15 @@ public class DecoupledExtension implements Extension
this.baseTest = baseTest;
}
private File qCaseDir;
@Override
public void beforeEach(ExtensionContext context)
{
Class<?> testClass = context.getTestClass().get();
qCaseDir = ProjectPathUtils.getPathFromProjectRoot("sql/src/test/quidem/" + testClass.getName());
}
private static final ImmutableMap<String, Object> CONTEXT_OVERRIDES = ImmutableMap.<String, Object>builder()
.putAll(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
.put(PlannerConfig.CTX_NATIVE_QUERY_SQL_PLANNING_MODE, PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED)
@ -47,8 +62,11 @@ public class DecoupledExtension implements Extension
DecoupledTestConfig decTestConfig = BaseCalciteQueryTest.queryFrameworkRule
.getAnnotation(DecoupledTestConfig.class);
boolean runQuidem = (decTestConfig != null && decTestConfig.quidemReason().isPresent());
CalciteTestConfig testConfig = baseTest.new CalciteTestConfig(CONTEXT_OVERRIDES)
{
@Override
public SqlTestFramework.PlannerFixture plannerFixture(PlannerConfig plannerConfig, AuthConfig authConfig)
{
@ -56,16 +74,36 @@ public class DecoupledExtension implements Extension
return baseTest.queryFramework().plannerFixture(plannerConfig, authConfig);
}
@Override
public DruidQTestInfo getQTestInfo()
{
if (runQuidem) {
final String testName;
if (decTestConfig.separateDefaultModeTest()) {
if (NullHandling.sqlCompatible()) {
testName = BaseCalciteQueryTest.queryFrameworkRule.testName() + "@NullHandling=sql";
} else {
testName = BaseCalciteQueryTest.queryFrameworkRule.testName() + "@NullHandling=default";
}
} else {
testName = BaseCalciteQueryTest.queryFrameworkRule.testName();
}
return new DruidQTestInfo(
qCaseDir,
testName,
"quidem testcase reason: " + decTestConfig.quidemReason()
);
} else {
return null;
}
}
};
QueryTestBuilder builder = new QueryTestBuilder(testConfig)
.cannotVectorize(baseTest.cannotVectorize)
.skipVectorize(baseTest.skipVectorize);
if (decTestConfig != null && decTestConfig.nativeQueryIgnore().isPresent()) {
builder.verifyNativeQueries(x -> false);
}
return builder;
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.sql.calcite;
import org.apache.druid.sql.calcite.DisableUnless.DisableUnlessRule;
import org.apache.druid.sql.calcite.NotYetSupported.NotYetSupportedProcessor;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.api.extension.RegisterExtension;
@ -32,9 +31,6 @@ import static org.junit.Assert.assertNotNull;
@ExtendWith(NotYetSupportedProcessor.class)
public class DecoupledPlanningCalciteJoinQueryTest extends CalciteJoinQueryTest
{
@RegisterExtension
public DisableUnlessRule sqlCompatOnly = DisableUnless.SQL_COMPATIBLE;
@RegisterExtension
DecoupledExtension decoupledExtension = new DecoupledExtension(this);

View File

@ -22,9 +22,6 @@ package org.apache.druid.sql.calcite;
import org.apache.calcite.rel.rules.CoreRules;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
@ -42,9 +39,9 @@ public @interface DecoupledTestConfig
*
* The value of this field should describe the root cause of the difference.
*/
NativeQueryIgnore nativeQueryIgnore() default NativeQueryIgnore.NONE;
QuidemTestCaseReason quidemReason() default QuidemTestCaseReason.NONE;
enum NativeQueryIgnore
enum QuidemTestCaseReason
{
NONE,
/**
@ -68,16 +65,6 @@ public @interface DecoupledTestConfig
* Worse plan; may loose vectorization; but no extra queries
*/
SLIGHTLY_WORSE_PLAN,
/**
* {@link TimeseriesQuery} to {@link ScanQuery} change.
*
* Not yet sure if this is improvement; or some issue
*/
TS_TO_SCAN,
/**
* GroupBy doesn't sorted?!
*/
GBY_DOESNT_SORT,
/**
* Equvivalent plan.
*
@ -107,6 +94,7 @@ public @interface DecoupledTestConfig
{
return this != NONE;
}
};
}
boolean separateDefaultModeTest() default false;
}

View File

@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.sql.calcite;
import com.google.common.hash.HashCode;
import com.google.common.hash.Hashing;
import com.google.common.io.Files;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.quidem.DruidQTestInfo;
import org.apache.druid.quidem.DruidQuidemTestBase;
import org.apache.druid.quidem.DruidQuidemTestBase.DruidQuidemRunner;
import org.apache.druid.sql.calcite.QueryTestRunner.QueryRunStep;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
public class QTestCase
{
private StringBuffer sb;
private DruidQTestInfo testInfo;
public QTestCase(DruidQTestInfo testInfo)
{
this.testInfo = testInfo;
sb = new StringBuffer();
sb.append("# ");
sb.append(testInfo.comment);
sb.append("\n");
}
public void println(String str)
{
sb.append(str);
sb.append("\n");
}
public QueryRunStep toRunner()
{
return new QueryRunStep(null)
{
@Override
public void run()
{
try {
if (DruidQuidemRunner.isOverwrite()) {
writeCaseTo(testInfo.getIQFile());
} else {
isValidTestCaseFile(testInfo.getIQFile());
}
DruidQuidemRunner runner = new DruidQuidemTestBase.DruidQuidemRunner();
runner.run(testInfo.getIQFile());
}
catch (Exception e) {
throw new RuntimeException("Error running quidem test", e);
}
}
};
}
protected void isValidTestCaseFile(File iqFile)
{
if (!iqFile.exists()) {
throw new IllegalStateException("testcase doesn't exists; run with (-Dquidem.overwrite) : " + iqFile);
}
try {
String header = makeHeader();
String testCaseFirstLine = Files.asCharSource(iqFile, StandardCharsets.UTF_8).readFirstLine();
if (!header.equals(testCaseFirstLine)) {
throw new IllegalStateException(
"backing quidem testcase doesn't match test - run with (-Dquidem.overwrite) : " + iqFile
);
}
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
private String makeHeader()
{
HashCode hash = Hashing.crc32().hashBytes(sb.toString().getBytes(StandardCharsets.UTF_8));
return StringUtils.format("# %s case-crc:%s", testInfo.testName, hash);
}
public void writeCaseTo(File file) throws IOException
{
FileUtils.mkdirp(file.getParentFile());
try (FileOutputStream fos = new FileOutputStream(file)) {
fos.write(makeHeader().getBytes(StandardCharsets.UTF_8));
fos.write('\n');
fos.write(sb.toString().getBytes(StandardCharsets.UTF_8));
}
catch (IOException e) {
throw new RuntimeException("Error writing testcase to: " + file, e);
}
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.quidem.DruidQTestInfo;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.server.security.AuthConfig;
import org.apache.druid.server.security.AuthenticationResult;
@ -34,6 +35,7 @@ import org.apache.druid.sql.calcite.QueryTestRunner.QueryResults;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.util.CalciteTestBase;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SqlTestFramework;
import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerFixture;
import org.apache.druid.sql.http.SqlParameter;
import java.util.ArrayList;
@ -75,6 +77,13 @@ public class QueryTestBuilder
boolean isRunningMSQ();
Map<String, Object> baseQueryContext();
default DruidQTestInfo getQTestInfo()
{
return null;
}
SqlTestFramework queryFramework();
}
protected final QueryTestConfig config;

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedMap;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlExplainFormat;
@ -34,6 +35,7 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.quidem.DruidQTestInfo;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.sql.DirectStatement;
@ -56,6 +58,7 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
@ -657,6 +660,31 @@ public class QueryTestRunner
public QueryTestRunner(QueryTestBuilder builder)
{
QueryTestConfig config = builder.config;
DruidQTestInfo iqTestInfo = config.getQTestInfo();
if (iqTestInfo != null) {
QTestCase qt = new QTestCase(iqTestInfo);
Map<String, Object> queryContext = ImmutableSortedMap.<String, Object>naturalOrder()
.putAll(builder.getQueryContext())
.putAll(builder.plannerConfig.getNonDefaultAsQueryContext())
.build();
for (Entry<String, Object> entry : queryContext.entrySet()) {
qt.println(StringUtils.format("!set %s %s", entry.getKey(), entry.getValue()));
}
qt.println("!set outputformat mysql");
qt.println("!use " + builder.config.queryFramework().getDruidTestURI());
qt.println(builder.sql + ";");
if (builder.expectedResults != null) {
qt.println("!ok");
}
qt.println("!logicalPlan");
qt.println("!druidPlan");
if (builder.expectedQueries != null) {
qt.println("!nativePlan");
}
runSteps.add(qt.toRunner());
return;
}
if (builder.expectedResultsVerifier == null && builder.expectedResults != null) {
builder.expectedResultsVerifier = config.defaultResultsVerifier(
builder.expectedResults,

View File

@ -19,10 +19,14 @@
package org.apache.druid.sql.calcite;
import com.google.common.base.CaseFormat;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.topn.TopNQueryConfig;
@ -30,6 +34,7 @@ import org.apache.druid.sql.calcite.util.CacheTestHelperModule.ResultCacheMode;
import org.apache.druid.sql.calcite.util.SqlTestFramework;
import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier;
import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier;
import org.apache.http.client.utils.URIBuilder;
import org.junit.jupiter.api.extension.AfterAllCallback;
import org.junit.jupiter.api.extension.BeforeEachCallback;
import org.junit.jupiter.api.extension.ExtensionContext;
@ -45,14 +50,19 @@ import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import java.lang.reflect.Constructor;
import java.lang.reflect.Method;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
/**
@ -146,6 +156,13 @@ public class SqlTestFrameworkConfig
Class<? extends QueryComponentSupplier> value();
}
private static final Set<String> KNOWN_CONFIG_KEYS = ImmutableSet.<String>builder()
.add(NumMergeBuffers.PROCESSOR.getConfigName())
.add(MinTopNThreshold.PROCESSOR.getConfigName())
.add(ResultCache.PROCESSOR.getConfigName())
.add(ComponentSupplier.PROCESSOR.getConfigName())
.build();
public final int numMergeBuffers;
public final int minTopNThreshold;
public final ResultCacheMode resultCache;
@ -166,6 +183,7 @@ public class SqlTestFrameworkConfig
public SqlTestFrameworkConfig(Map<String, String> queryParams)
{
validateConfigKeys(queryParams.keySet());
try {
numMergeBuffers = NumMergeBuffers.PROCESSOR.fromMap(queryParams);
minTopNThreshold = MinTopNThreshold.PROCESSOR.fromMap(queryParams);
@ -177,6 +195,15 @@ public class SqlTestFrameworkConfig
}
}
private void validateConfigKeys(Set<String> keySet)
{
Set<String> diff = Sets.difference(keySet, KNOWN_CONFIG_KEYS);
if (diff.isEmpty()) {
return;
}
throw new IAE("Invalid configuration key(s) specified [%s]; valid options are [%s]", diff, KNOWN_CONFIG_KEYS);
}
@Override
public int hashCode()
{
@ -244,6 +271,7 @@ public class SqlTestFrameworkConfig
SqlTestFrameworkConfigStore configStore = new SqlTestFrameworkConfigStore();
private SqlTestFrameworkConfig config;
private Method method;
private String testName;
@Override
public void afterAll(ExtensionContext context)
@ -259,12 +287,31 @@ public class SqlTestFrameworkConfig
private void setConfig(ExtensionContext context)
{
testName = buildTestCaseName(context);
method = context.getTestMethod().get();
Class<?> testClass = context.getTestClass().get();
List<Annotation> annotations = collectAnnotations(testClass, method);
config = new SqlTestFrameworkConfig(annotations);
}
/**
* Returns a string identifying the testcase.
*
*
*/
public String buildTestCaseName(ExtensionContext context)
{
List<String> names = new ArrayList<String>();
Pattern pattern = Pattern.compile("\\([^)]*\\)");
// this will add all name pieces - except the "last" which would be the
// Class level name
do {
names.add(0, pattern.matcher(context.getDisplayName()).replaceAll(""));
context = context.getParent().get();
} while (context.getTestMethod().isPresent());
return Joiner.on("@").join(names);
}
public SqlTestFrameworkConfig getConfig()
{
return config;
@ -282,7 +329,7 @@ public class SqlTestFrameworkConfig
public String testName()
{
return method.getName();
return testName;
}
}
@ -292,8 +339,8 @@ public class SqlTestFrameworkConfig
ConfigurationInstance(SqlTestFrameworkConfig config, QueryComponentSupplier testHost)
{
SqlTestFramework.Builder builder = new SqlTestFramework.Builder(testHost)
.withConfig(config)
.catalogResolver(testHost.createCatalogResolver())
.minTopNThreshold(config.minTopNThreshold)
.mergeBufferCount(config.numMergeBuffers)
@ -321,6 +368,38 @@ public class SqlTestFrameworkConfig
}
}
public URI getDruidTestURI()
{
try {
Map<String, String> params = getNonDefaultMap();
URIBuilder ub = new URIBuilder("druidtest:///");
for (Entry<String, String> entry : params.entrySet()) {
ub.setParameter(entry.getKey(), entry.getValue());
}
ub.setPath("///");
return ub.build();
}
catch (URISyntaxException e) {
throw new RuntimeException(e);
}
}
private Map<String, String> getNonDefaultMap()
{
Map<String, String> map = new HashMap<>();
SqlTestFrameworkConfig def = new SqlTestFrameworkConfig(Collections.emptyList());
if (def.numMergeBuffers != numMergeBuffers) {
map.put("numMergeBuffers", String.valueOf(numMergeBuffers));
}
if (def.minTopNThreshold != minTopNThreshold) {
map.put("minTopNThreshold", String.valueOf(minTopNThreshold));
}
if (!equals(new SqlTestFrameworkConfig(map))) {
throw new IAE("Can't reproduce config via map!");
}
return map;
}
abstract static class ConfigOptionProcessor<T>
{
final Class<? extends Annotation> annotationClass;
@ -330,6 +409,11 @@ public class SqlTestFrameworkConfig
this.annotationClass = annotationClass;
}
public final String getConfigName()
{
return CaseFormat.UPPER_CAMEL.to(CaseFormat.LOWER_CAMEL, annotationClass.getSimpleName());
}
@SuppressWarnings("unchecked")
public final T fromAnnotations(List<Annotation> annotations) throws Exception
{
@ -358,7 +442,7 @@ public class SqlTestFrameworkConfig
public final T fromMap(Map<String, String> map) throws Exception
{
String key = annotationClass.getSimpleName();
String key = getConfigName();
String value = map.get(key);
if (value == null) {
return defaultValue();

View File

@ -19,7 +19,9 @@
package org.apache.druid.sql.calcite;
import com.google.common.collect.ImmutableMap;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.MinTopNThreshold;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.NumMergeBuffers;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.ResultCache;
@ -30,6 +32,7 @@ import java.lang.annotation.Annotation;
import java.util.List;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class SqlTestFrameworkConfigTest
{
@ -118,4 +121,22 @@ public class SqlTestFrameworkConfigTest
assertEquals(1, config.numMergeBuffers);
assertEquals(ResultCacheMode.DISABLED, config.resultCache);
}
@Test
public void testInvalidConfigKeySpecified()
{
ImmutableMap<String, String> configMap = ImmutableMap.<String, String>builder()
.put("nonExistent", "someValue")
.build();
IAE e = assertThrows(
IAE.class,
() -> new SqlTestFrameworkConfig(configMap)
);
assertEquals(
"Invalid configuration key(s) specified [[nonExistent]]; valid options are [[numMergeBuffers, minTopNThreshold, resultCache, componentSupplier]]",
e.getMessage()
);
}
}

View File

@ -50,6 +50,7 @@ import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.server.security.AuthConfig;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig;
import org.apache.druid.sql.calcite.TempDirProducer;
import org.apache.druid.sql.calcite.aggregation.SqlAggregationModule;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
@ -70,6 +71,7 @@ import org.apache.druid.timeline.DataSegment;
import java.io.Closeable;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
@ -403,6 +405,7 @@ public class SqlTestFramework
private int mergeBufferCount;
private CatalogResolver catalogResolver = CatalogResolver.NULL_RESOLVER;
private List<Module> overrideModules = new ArrayList<>();
private SqlTestFrameworkConfig config;
public Builder(QueryComponentSupplier componentSupplier)
{
@ -437,6 +440,12 @@ public class SqlTestFramework
{
return new SqlTestFramework(this);
}
public Builder withConfig(SqlTestFrameworkConfig config)
{
this.config = config;
return this;
}
}
/**
@ -692,4 +701,9 @@ public class SqlTestFramework
throw new RE(e);
}
}
public URI getDruidTestURI()
{
return builder.config.getDruidTestURI();
}
}

View File

@ -1,4 +1,4 @@
!use druidtest://?NumMergeBuffers=3
!use druidtest://?numMergeBuffers=3
!set outputformat mysql
SELECT

View File

@ -1,4 +1,4 @@
!use druidtest://?ComponentSupplier=NestedComponentSupplier
!use druidtest://?componentSupplier=NestedComponentSupplier
!set outputformat mysql
select count(1) from nested;

View File

@ -0,0 +1,90 @@
# testCommaJoinLeftFunction case-crc:0f0b74b8
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.dim1, foo.dim2, l.k, l.v
FROM foo, lookup.lookyloo l
WHERE SUBSTRING(foo.dim2, 1, 1) = l.k
;
+------+------+---+----+
| dim1 | dim2 | k | v |
+------+------+---+----+
| | a | a | xa |
| 1 | a | a | xa |
| def | abc | a | xa |
+------+------+---+----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4])
LogicalJoin(condition=[=($2, $3)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2], $f2=[SUBSTRING($2, 1, 1)])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
DruidJoin(condition=[=($2, $3)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], $f2=[SUBSTRING($2, 1, 1)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"dim2\", 0, 1)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,115 @@
# testExactTopNOnInnerJoinWithLimit@NullHandling=default case-crc:1b8b1878
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set useApproximateTopN false
!set outputformat mysql
!use druidtest:///?minTopNThreshold=1
select f1."dim4", sum("m1") from numfoo f1 inner join (
select "dim4" from numfoo where dim4 <> 'a' group by 1
) f2 on f1."dim4" = f2."dim4" group by 1 limit 1;
+------+--------+
| dim4 | EXPR$1 |
+------+--------+
| b | 15.0 |
+------+--------+
(1 row)
!ok
LogicalSort(fetch=[1])
LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim4=[$4], m1=[$14])
LogicalTableScan(table=[[druid, numfoo]])
LogicalAggregate(group=[{4}])
LogicalFilter(condition=[<>($4, 'a')])
LogicalTableScan(table=[[druid, numfoo]])
!logicalPlan
DruidSort(fetch=[1], druid=[logical])
DruidAggregate(group=[{0}], EXPR$1=[SUM($1)], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim4=[$4], m1=[$14], druid=[logical])
DruidTableScan(table=[[druid, numfoo]], druid=[logical])
DruidAggregate(group=[{4}], druid=[logical])
DruidFilter(condition=[<>($4, 'a')])
DruidTableScan(table=[[druid, numfoo]], druid=[logical])
!druidPlan
{
"queryType" : "topN",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "numfoo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "table",
"name" : "numfoo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"filter" : {
"type" : "not",
"field" : {
"type" : "selector",
"dimension" : "dim4",
"value" : "a"
}
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "dim4",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim4\" == \"j0._d0\")",
"joinType" : "INNER"
},
"dimension" : {
"type" : "default",
"dimension" : "dim4",
"outputName" : "d0",
"outputType" : "STRING"
},
"metric" : {
"type" : "dimension",
"ordering" : {
"type" : "lexicographic"
}
},
"threshold" : 1,
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "doubleSum",
"name" : "a0",
"fieldName" : "m1"
} ]
}
!nativePlan

View File

@ -0,0 +1,116 @@
# testExactTopNOnInnerJoinWithLimit@NullHandling=sql case-crc:1b8b1878
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set useApproximateTopN false
!set outputformat mysql
!use druidtest:///?minTopNThreshold=1
select f1."dim4", sum("m1") from numfoo f1 inner join (
select "dim4" from numfoo where dim4 <> 'a' group by 1
) f2 on f1."dim4" = f2."dim4" group by 1 limit 1;
+------+--------+
| dim4 | EXPR$1 |
+------+--------+
| b | 15.0 |
+------+--------+
(1 row)
!ok
LogicalSort(fetch=[1])
LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim4=[$4], m1=[$14])
LogicalTableScan(table=[[druid, numfoo]])
LogicalAggregate(group=[{4}])
LogicalFilter(condition=[<>($4, 'a')])
LogicalTableScan(table=[[druid, numfoo]])
!logicalPlan
DruidSort(fetch=[1], druid=[logical])
DruidAggregate(group=[{0}], EXPR$1=[SUM($1)], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim4=[$4], m1=[$14], druid=[logical])
DruidTableScan(table=[[druid, numfoo]], druid=[logical])
DruidAggregate(group=[{4}], druid=[logical])
DruidFilter(condition=[<>($4, 'a')])
DruidTableScan(table=[[druid, numfoo]], druid=[logical])
!druidPlan
{
"queryType" : "topN",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "numfoo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "table",
"name" : "numfoo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"filter" : {
"type" : "not",
"field" : {
"type" : "equals",
"column" : "dim4",
"matchValueType" : "STRING",
"matchValue" : "a"
}
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "dim4",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim4\" == \"j0._d0\")",
"joinType" : "INNER"
},
"dimension" : {
"type" : "default",
"dimension" : "dim4",
"outputName" : "d0",
"outputType" : "STRING"
},
"metric" : {
"type" : "dimension",
"ordering" : {
"type" : "lexicographic"
}
},
"threshold" : 1,
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "doubleSum",
"name" : "a0",
"fieldName" : "m1"
} ]
}
!nativePlan

View File

@ -0,0 +1,163 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default case-crc:9802d63b
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,165 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql case-crc:9802d63b
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,163 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default case-crc:06fe6714
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,165 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql case-crc:06fe6714
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,160 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default case-crc:2e780a44
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,162 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql case-crc:2e780a44
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,163 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default case-crc:ec1996e8
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,165 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql case-crc:ec1996e8
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,163 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default case-crc:ddcd7387
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,165 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql case-crc:ddcd7387
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,163 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default case-crc:cf16e9a6
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,165 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql case-crc:cf16e9a6
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,163 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default case-crc:830604f9
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,165 @@
# testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql case-crc:830604f9
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT dim1 from (SELECT dim1,__time FROM (SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1) GROUP BY 1,2) GROUP BY dim1
;
+------+
| dim1 |
+------+
| 10.1 |
+------+
(1 row)
!ok
LogicalAggregate(group=[{0}])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR])
LogicalAggregate(group=[{1}])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidAggregate(group=[{0}], druid=[logical])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], druid=[logical])
DruidAggregate(group=[{1}], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "groupBy",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "__time",
"outputName" : "d0",
"outputType" : "LONG"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "default",
"dimension" : "v0",
"outputName" : "_d0",
"outputType" : "STRING"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@all_disabled@NullHandling=default case-crc:8cd9e460
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@all_disabled@NullHandling=sql case-crc:8cd9e460
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@all_enabled@NullHandling=default case-crc:57e7eb69
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@all_enabled@NullHandling=sql case-crc:57e7eb69
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,87 @@
# testInnerJoinCastLeft@default@NullHandling=default case-crc:55da2e8b
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,87 @@
# testInnerJoinCastLeft@default@NullHandling=sql case-crc:55da2e8b
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default case-crc:7f9dea41
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql case-crc:7f9dea41
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default case-crc:1723b06b
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql case-crc:1723b06b
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@filter-rewrites@NullHandling=default case-crc:fb3b0939
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@filter-rewrites@NullHandling=sql case-crc:fb3b0939
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@join-to-filter@NullHandling=default case-crc:2b873437
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR NOT NULL], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinCastLeft@join-to-filter@NullHandling=sql case-crc:2b873437
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.m1, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k
;
+----+---+---+
| m1 | k | v |
+----+---+---+
+----+---+---+
(0 rows)
!ok
LogicalProject(m1=[$0], k=[$2], v=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(m1=[$5], m10=[CAST($5):VARCHAR])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(m1=[$5], m10=[CAST($5):VARCHAR], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "CAST(\"m1\", 'STRING')",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,93 @@
# testInnerJoinLeftFunction@all_disabled case-crc:a8494934
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.dim1, foo.dim2, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k
;
+------+------+---+----+
| dim1 | dim2 | k | v |
+------+------+---+----+
| | a | a | xa |
| 1 | a | a | xa |
| def | abc | a | xa |
+------+------+---+----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4])
LogicalJoin(condition=[=($2, $3)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
DruidJoin(condition=[=($2, $3)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"dim2\", 0, 1)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,93 @@
# testInnerJoinLeftFunction@all_enabled case-crc:51164f2b
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.dim1, foo.dim2, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k
;
+------+------+---+----+
| dim1 | dim2 | k | v |
+------+------+---+----+
| | a | a | xa |
| 1 | a | a | xa |
| def | abc | a | xa |
+------+------+---+----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4])
LogicalJoin(condition=[=($2, $3)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
DruidJoin(condition=[=($2, $3)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"dim2\", 0, 1)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,90 @@
# testInnerJoinLeftFunction@default case-crc:cbea2a3d
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.dim1, foo.dim2, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k
;
+------+------+---+----+
| dim1 | dim2 | k | v |
+------+------+---+----+
| | a | a | xa |
| 1 | a | a | xa |
| def | abc | a | xa |
+------+------+---+----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4])
LogicalJoin(condition=[=($2, $3)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
DruidJoin(condition=[=($2, $3)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"dim2\", 0, 1)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,93 @@
# testInnerJoinLeftFunction@filter-on-value-column_disabled case-crc:e04823f2
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.dim1, foo.dim2, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k
;
+------+------+---+----+
| dim1 | dim2 | k | v |
+------+------+---+----+
| | a | a | xa |
| 1 | a | a | xa |
| def | abc | a | xa |
+------+------+---+----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4])
LogicalJoin(condition=[=($2, $3)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
DruidJoin(condition=[=($2, $3)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"dim2\", 0, 1)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,93 @@
# testInnerJoinLeftFunction@filter-rewrites-disabled case-crc:49782da2
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.dim1, foo.dim2, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k
;
+------+------+---+----+
| dim1 | dim2 | k | v |
+------+------+---+----+
| | a | a | xa |
| 1 | a | a | xa |
| def | abc | a | xa |
+------+------+---+----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4])
LogicalJoin(condition=[=($2, $3)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
DruidJoin(condition=[=($2, $3)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"dim2\", 0, 1)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,93 @@
# testInnerJoinLeftFunction@filter-rewrites case-crc:88fbe7a0
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.dim1, foo.dim2, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k
;
+------+------+---+----+
| dim1 | dim2 | k | v |
+------+------+---+----+
| | a | a | xa |
| 1 | a | a | xa |
| def | abc | a | xa |
+------+------+---+----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4])
LogicalJoin(condition=[=($2, $3)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
DruidJoin(condition=[=($2, $3)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"dim2\", 0, 1)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,93 @@
# testInnerJoinLeftFunction@join-to-filter case-crc:d8b2c6e1
# quidem testcase reason: JOIN_FILTER_LOCATIONS
!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 foo.dim1, foo.dim2, l.k, l.v
FROM foo
INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k
;
+------+------+---+----+
| dim1 | dim2 | k | v |
+------+------+---+----+
| | a | a | xa |
| 1 | a | a | xa |
| def | abc | a | xa |
+------+------+---+----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4])
LogicalJoin(condition=[=($2, $3)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)])
LogicalTableScan(table=[[druid, foo]])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
DruidJoin(condition=[=($2, $3)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], $f8=[SUBSTRING($2, 1, 1)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"dim2\", 0, 1)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default case-crc:c8387bad
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql case-crc:c8387bad
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default case-crc:3117d34a
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql case-crc:3117d34a
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,130 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default case-crc:8573b42e
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinLeftTableScanDirect true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,132 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql case-crc:8573b42e
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinLeftTableScanDirect true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default case-crc:aba2d8ac
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql case-crc:aba2d8ac
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default case-crc:6bf67b17
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql case-crc:6bf67b17
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default case-crc:bf731e2a
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql case-crc:bf731e2a
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default case-crc:a0b23b27
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql case-crc:a0b23b27
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default case-crc:bc06e265
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql case-crc:bc06e265
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default case-crc:19717e46
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql case-crc:19717e46
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,130 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default case-crc:cddd4617
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinLeftTableScanDirect true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,132 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql case-crc:cddd4617
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinLeftTableScanDirect true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default case-crc:e72bd235
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql case-crc:e72bd235
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default case-crc:50015dc5
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql case-crc:50015dc5
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default case-crc:ae732a4e
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql case-crc:ae732a4e
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,133 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default case-crc:a3f1513a
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "selector",
"dimension" : "dim1",
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,135 @@
# testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql case-crc:a3f1513a
# quidem testcase reason: JOIN_LEFT_DIRECT_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableJoinLeftTableScanDirect true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
with abc as
(
SELECT dim1, "__time", m1 from foo WHERE "dim1" = '10.1'
)
SELECT t1.dim1, t1."__time" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1
;
+------+---------------------+
| dim1 | __time |
+------+---------------------+
| 10.1 | 2000-01-02 00:00:00 |
+------+---------------------+
(1 row)
!ok
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1])
LogicalJoin(condition=[=($0, $2)], joinType=[inner])
LogicalProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(dim1=[$1])
LogicalFilter(condition=[=($1, '10.1')])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
DruidJoin(condition=[=($0, $2)], joinType=[inner])
DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$0], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(dim1=[$1], druid=[logical])
DruidFilter(condition=[=($1, '10.1')])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"filter" : {
"type" : "equals",
"column" : "dim1",
"matchValueType" : "STRING",
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"v0\" == \"j0.dim1\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "_v0",
"expression" : "'10.1'",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,113 @@
# testInnerJoinQueryOfLookup@all_disabled case-crc:d41a4a0d
# quidem testcase reason: FINALIZING_FIELD_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.v, t1.v
FROM foo
INNER JOIN
(SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1
ON foo.dim2 = t1.k;
+------+------+------+------+
| dim1 | dim2 | v | v |
+------+------+------+------+
| | a | xabc | xabc |
| 1 | a | xabc | xabc |
+------+------+------+------+
(2 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)])
LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical])
DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "extraction",
"dimension" : "k",
"outputName" : "d0",
"outputType" : "STRING",
"extractionFn" : {
"type" : "substring",
"index" : 0,
"length" : 1
}
} ],
"aggregations" : [ {
"type" : "stringAny",
"name" : "a0:a",
"fieldName" : "v",
"maxStringBytes" : 10,
"aggregateMultipleValues" : true
} ],
"postAggregations" : [ {
"type" : "finalizingFieldAccess",
"name" : "a0",
"fieldName" : "a0:a"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.d0\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.a0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,113 @@
# testInnerJoinQueryOfLookup@all_enabled case-crc:93392df4
# quidem testcase reason: FINALIZING_FIELD_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.v, t1.v
FROM foo
INNER JOIN
(SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1
ON foo.dim2 = t1.k;
+------+------+------+------+
| dim1 | dim2 | v | v |
+------+------+------+------+
| | a | xabc | xabc |
| 1 | a | xabc | xabc |
+------+------+------+------+
(2 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)])
LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical])
DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "extraction",
"dimension" : "k",
"outputName" : "d0",
"outputType" : "STRING",
"extractionFn" : {
"type" : "substring",
"index" : 0,
"length" : 1
}
} ],
"aggregations" : [ {
"type" : "stringAny",
"name" : "a0:a",
"fieldName" : "v",
"maxStringBytes" : 10,
"aggregateMultipleValues" : true
} ],
"postAggregations" : [ {
"type" : "finalizingFieldAccess",
"name" : "a0",
"fieldName" : "a0:a"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.d0\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.a0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,110 @@
# testInnerJoinQueryOfLookup@default case-crc:ee151062
# quidem testcase reason: FINALIZING_FIELD_ACCESS
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.v, t1.v
FROM foo
INNER JOIN
(SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1
ON foo.dim2 = t1.k;
+------+------+------+------+
| dim1 | dim2 | v | v |
+------+------+------+------+
| | a | xabc | xabc |
| 1 | a | xabc | xabc |
+------+------+------+------+
(2 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)])
LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical])
DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "extraction",
"dimension" : "k",
"outputName" : "d0",
"outputType" : "STRING",
"extractionFn" : {
"type" : "substring",
"index" : 0,
"length" : 1
}
} ],
"aggregations" : [ {
"type" : "stringAny",
"name" : "a0:a",
"fieldName" : "v",
"maxStringBytes" : 10,
"aggregateMultipleValues" : true
} ],
"postAggregations" : [ {
"type" : "finalizingFieldAccess",
"name" : "a0",
"fieldName" : "a0:a"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.d0\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.a0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,113 @@
# testInnerJoinQueryOfLookup@filter-on-value-column_disabled case-crc:dbd4147e
# quidem testcase reason: FINALIZING_FIELD_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.v, t1.v
FROM foo
INNER JOIN
(SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1
ON foo.dim2 = t1.k;
+------+------+------+------+
| dim1 | dim2 | v | v |
+------+------+------+------+
| | a | xabc | xabc |
| 1 | a | xabc | xabc |
+------+------+------+------+
(2 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)])
LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical])
DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "extraction",
"dimension" : "k",
"outputName" : "d0",
"outputType" : "STRING",
"extractionFn" : {
"type" : "substring",
"index" : 0,
"length" : 1
}
} ],
"aggregations" : [ {
"type" : "stringAny",
"name" : "a0:a",
"fieldName" : "v",
"maxStringBytes" : 10,
"aggregateMultipleValues" : true
} ],
"postAggregations" : [ {
"type" : "finalizingFieldAccess",
"name" : "a0",
"fieldName" : "a0:a"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.d0\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.a0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,113 @@
# testInnerJoinQueryOfLookup@filter-rewrites-disabled case-crc:57dd8dfa
# quidem testcase reason: FINALIZING_FIELD_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.v, t1.v
FROM foo
INNER JOIN
(SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1
ON foo.dim2 = t1.k;
+------+------+------+------+
| dim1 | dim2 | v | v |
+------+------+------+------+
| | a | xabc | xabc |
| 1 | a | xabc | xabc |
+------+------+------+------+
(2 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)])
LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical])
DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "extraction",
"dimension" : "k",
"outputName" : "d0",
"outputType" : "STRING",
"extractionFn" : {
"type" : "substring",
"index" : 0,
"length" : 1
}
} ],
"aggregations" : [ {
"type" : "stringAny",
"name" : "a0:a",
"fieldName" : "v",
"maxStringBytes" : 10,
"aggregateMultipleValues" : true
} ],
"postAggregations" : [ {
"type" : "finalizingFieldAccess",
"name" : "a0",
"fieldName" : "a0:a"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.d0\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.a0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,113 @@
# testInnerJoinQueryOfLookup@filter-rewrites case-crc:10d0367d
# quidem testcase reason: FINALIZING_FIELD_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.v, t1.v
FROM foo
INNER JOIN
(SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1
ON foo.dim2 = t1.k;
+------+------+------+------+
| dim1 | dim2 | v | v |
+------+------+------+------+
| | a | xabc | xabc |
| 1 | a | xabc | xabc |
+------+------+------+------+
(2 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)])
LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical])
DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "extraction",
"dimension" : "k",
"outputName" : "d0",
"outputType" : "STRING",
"extractionFn" : {
"type" : "substring",
"index" : 0,
"length" : 1
}
} ],
"aggregations" : [ {
"type" : "stringAny",
"name" : "a0:a",
"fieldName" : "v",
"maxStringBytes" : 10,
"aggregateMultipleValues" : true
} ],
"postAggregations" : [ {
"type" : "finalizingFieldAccess",
"name" : "a0",
"fieldName" : "a0:a"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.d0\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.a0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,113 @@
# testInnerJoinQueryOfLookup@join-to-filter case-crc:967213e2
# quidem testcase reason: FINALIZING_FIELD_ACCESS
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.v, t1.v
FROM foo
INNER JOIN
(SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1
ON foo.dim2 = t1.k;
+------+------+------+------+
| dim1 | dim2 | v | v |
+------+------+------+------+
| | a | xabc | xabc |
| 1 | a | xabc | xabc |
+------+------+------+------+
(2 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)])
LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical])
DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "groupBy",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"dimensions" : [ {
"type" : "extraction",
"dimension" : "k",
"outputName" : "d0",
"outputType" : "STRING",
"extractionFn" : {
"type" : "substring",
"index" : 0,
"length" : 1
}
} ],
"aggregations" : [ {
"type" : "stringAny",
"name" : "a0:a",
"fieldName" : "v",
"maxStringBytes" : 10,
"aggregateMultipleValues" : true
} ],
"postAggregations" : [ {
"type" : "finalizingFieldAccess",
"name" : "a0",
"fieldName" : "a0:a"
} ],
"limitSpec" : {
"type" : "NoopLimitSpec"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.d0\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.a0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,96 @@
# testInnerJoinQueryOfLookupRemovable@all_disabled case-crc:f9e61e4e
# quidem testcase reason: DEFINETLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.sk
FROM foo
INNER JOIN
(SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1
ON foo.dim2 = t1.k;
+------+------+-----+
| dim1 | dim2 | sk |
+------+------+-----+
| | a | xa |
| 1 | a | xa |
| def | abc | xab |
+------+------+-----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], sk=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"v\", 0, 3)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "k", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,96 @@
# testInnerJoinQueryOfLookupRemovable@all_enabled case-crc:110c167a
# quidem testcase reason: DEFINETLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.sk
FROM foo
INNER JOIN
(SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1
ON foo.dim2 = t1.k;
+------+------+-----+
| dim1 | dim2 | sk |
+------+------+-----+
| | a | xa |
| 1 | a | xa |
| def | abc | xab |
+------+------+-----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], sk=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"v\", 0, 3)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "k", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,93 @@
# testInnerJoinQueryOfLookupRemovable@default case-crc:56875820
# quidem testcase reason: DEFINETLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.sk
FROM foo
INNER JOIN
(SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1
ON foo.dim2 = t1.k;
+------+------+-----+
| dim1 | dim2 | sk |
+------+------+-----+
| | a | xa |
| 1 | a | xa |
| def | abc | xab |
+------+------+-----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], sk=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"v\", 0, 3)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "k", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,96 @@
# testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled case-crc:1624eceb
# quidem testcase reason: DEFINETLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.sk
FROM foo
INNER JOIN
(SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1
ON foo.dim2 = t1.k;
+------+------+-----+
| dim1 | dim2 | sk |
+------+------+-----+
| | a | xa |
| 1 | a | xa |
| def | abc | xab |
+------+------+-----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], sk=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"v\", 0, 3)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "k", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,96 @@
# testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled case-crc:d2cf1525
# quidem testcase reason: DEFINETLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.sk
FROM foo
INNER JOIN
(SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1
ON foo.dim2 = t1.k;
+------+------+-----+
| dim1 | dim2 | sk |
+------+------+-----+
| | a | xa |
| 1 | a | xa |
| def | abc | xab |
+------+------+-----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], sk=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"v\", 0, 3)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "k", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,96 @@
# testInnerJoinQueryOfLookupRemovable@filter-rewrites case-crc:f6c15132
# quidem testcase reason: DEFINETLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.sk
FROM foo
INNER JOIN
(SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1
ON foo.dim2 = t1.k;
+------+------+-----+
| dim1 | dim2 | sk |
+------+------+-----+
| | a | xa |
| 1 | a | xa |
| def | abc | xab |
+------+------+-----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], sk=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"v\", 0, 3)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "k", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

View File

@ -0,0 +1,96 @@
# testInnerJoinQueryOfLookupRemovable@join-to-filter case-crc:dfb4dee9
# quidem testcase reason: DEFINETLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
SELECT dim1, dim2, t1.sk
FROM foo
INNER JOIN
(SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1
ON foo.dim2 = t1.k;
+------+------+-----+
| dim1 | dim2 | sk |
+------+------+-----+
| | a | xa |
| 1 | a | xa |
| def | abc | xab |
+------+------+-----+
(3 rows)
!ok
LogicalProject(dim1=[$0], dim2=[$1], sk=[$3])
LogicalJoin(condition=[=($1, $2)], joinType=[inner])
LogicalProject(dim1=[$1], dim2=[$2])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0], sk=[SUBSTRING($1, 1, 3)])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical])
DruidJoin(condition=[=($1, $2)], joinType=[inner])
DruidProject(dim1=[$1], dim2=[$2], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], sk=[SUBSTRING($1, 1, 3)], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "query",
"query" : {
"queryType" : "scan",
"dataSource" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"virtualColumns" : [ {
"type" : "expression",
"name" : "v0",
"expression" : "substring(\"v\", 0, 3)",
"outputType" : "STRING"
} ],
"resultFormat" : "compactedList",
"columns" : [ "k", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
},
"rightPrefix" : "j0.",
"condition" : "(\"dim2\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
}
}
!nativePlan

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