mirror of https://github.com/apache/druid.git
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:
parent
10ea88e5bf
commit
12f79acc7e
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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()),
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
!use druidtest://?NumMergeBuffers=3
|
||||
!use druidtest://?numMergeBuffers=3
|
||||
!set outputformat mysql
|
||||
|
||||
SELECT
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
!use druidtest://?ComponentSupplier=NestedComponentSupplier
|
||||
!use druidtest://?componentSupplier=NestedComponentSupplier
|
||||
!set outputformat mysql
|
||||
|
||||
select count(1) from nested;
|
||||
|
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
Loading…
Reference in New Issue