[enhancement](Nereids) compare LogicalProperties with output set instead of output list (#12743)

We used output list to compare two LogicalProperties before. Since join reorder will change the children order of a join plan and caused output list changed. the two join plan will not equals anymore in memo although they should be. So we must add a project on the new join to keep the LogicalProperties the same.
This PR changes the equals and hashCode funtions of LogicalProperties. use a set of output to compare two LogicalProperties. Then we do not need add the top peoject anymore. This help us keep memo simple and efficient.
This commit is contained in:
morrySnow
2022-09-20 10:55:29 +08:00
committed by GitHub
parent d435f0de41
commit 954c44db39
14 changed files with 135 additions and 177 deletions

View File

@ -549,7 +549,6 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
.map(TupleDescriptor::getSlots)
.flatMap(Collection::stream)
.collect(Collectors.toList());
TupleDescriptor outputDescriptor = context.generateTupleDesc();
Map<ExprId, SlotReference> outputSlotReferenceMap = Maps.newHashMap();
hashJoin.getOutput().stream()
@ -579,6 +578,15 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
.map(SlotReference.class::cast)
.forEach(s -> hashOutputSlotReferenceMap.put(s.getExprId(), s));
Map<ExprId, SlotReference> leftChildOutputMap = Maps.newHashMap();
hashJoin.child(0).getOutput().stream()
.map(SlotReference.class::cast)
.forEach(s -> leftChildOutputMap.put(s.getExprId(), s));
Map<ExprId, SlotReference> rightChildOutputMap = Maps.newHashMap();
hashJoin.child(1).getOutput().stream()
.map(SlotReference.class::cast)
.forEach(s -> rightChildOutputMap.put(s.getExprId(), s));
//make intermediate tuple
List<SlotDescriptor> leftIntermediateSlotDescriptor = Lists.newArrayList();
List<SlotDescriptor> rightIntermediateSlotDescriptor = Lists.newArrayList();
@ -586,47 +594,43 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
if (!hashJoin.getOtherJoinCondition().isPresent()
&& (joinType == JoinType.LEFT_ANTI_JOIN || joinType == JoinType.LEFT_SEMI_JOIN)) {
leftIntermediateSlotDescriptor = hashJoin.child(0).getOutput().stream()
.map(SlotReference.class::cast)
.map(s -> context.createSlotDesc(intermediateDescriptor, s))
.collect(Collectors.toList());
for (SlotDescriptor leftSlotDescriptor : leftSlotDescriptors) {
SlotReference sf = leftChildOutputMap.get(context.findExprId(leftSlotDescriptor.getId()));
SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf);
leftIntermediateSlotDescriptor.add(sd);
}
} else if (!hashJoin.getOtherJoinCondition().isPresent()
&& (joinType == JoinType.RIGHT_ANTI_JOIN || joinType == JoinType.RIGHT_SEMI_JOIN)) {
rightIntermediateSlotDescriptor = hashJoin.child(1).getOutput().stream()
.map(SlotReference.class::cast)
.map(s -> context.createSlotDesc(intermediateDescriptor, s))
.collect(Collectors.toList());
for (SlotDescriptor rightSlotDescriptor : rightSlotDescriptors) {
SlotReference sf = rightChildOutputMap.get(context.findExprId(rightSlotDescriptor.getId()));
SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf);
rightIntermediateSlotDescriptor.add(sd);
}
} else {
for (int i = 0; i < hashJoin.child(0).getOutput().size(); i++) {
SlotReference sf = (SlotReference) hashJoin.child(0).getOutput().get(i);
for (SlotDescriptor leftSlotDescriptor : leftSlotDescriptors) {
SlotReference sf = leftChildOutputMap.get(context.findExprId(leftSlotDescriptor.getId()));
SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf);
if (hashOutputSlotReferenceMap.get(sf.getExprId()) != null) {
hashJoinNode.addSlotIdToHashOutputSlotIds(leftSlotDescriptors.get(i).getId());
hashJoinNode.addSlotIdToHashOutputSlotIds(leftSlotDescriptor.getId());
}
leftIntermediateSlotDescriptor.add(sd);
}
for (int i = 0; i < hashJoin.child(1).getOutput().size(); i++) {
SlotReference sf = (SlotReference) hashJoin.child(1).getOutput().get(i);
for (SlotDescriptor rightSlotDescriptor : rightSlotDescriptors) {
SlotReference sf = rightChildOutputMap.get(context.findExprId(rightSlotDescriptor.getId()));
SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf);
if (hashOutputSlotReferenceMap.get(sf.getExprId()) != null) {
hashJoinNode.addSlotIdToHashOutputSlotIds(rightSlotDescriptors.get(i).getId());
hashJoinNode.addSlotIdToHashOutputSlotIds(rightSlotDescriptor.getId());
}
rightIntermediateSlotDescriptor.add(sd);
}
}
//set slots as nullable for outer join
if (joinType == JoinType.FULL_OUTER_JOIN) {
rightIntermediateSlotDescriptor.stream()
.forEach(sd -> sd.setIsNullable(true));
leftIntermediateSlotDescriptor.stream()
.forEach(sd -> sd.setIsNullable(true));
} else if (joinType == JoinType.LEFT_OUTER_JOIN) {
rightIntermediateSlotDescriptor.stream()
.forEach(sd -> sd.setIsNullable(true));
} else if (joinType == JoinType.RIGHT_OUTER_JOIN) {
leftIntermediateSlotDescriptor.stream()
.forEach(sd -> sd.setIsNullable(true));
if (joinType == JoinType.LEFT_OUTER_JOIN || joinType == JoinType.FULL_OUTER_JOIN) {
rightIntermediateSlotDescriptor.forEach(sd -> sd.setIsNullable(true));
}
if (joinType == JoinType.RIGHT_OUTER_JOIN || joinType == JoinType.FULL_OUTER_JOIN) {
leftIntermediateSlotDescriptor.forEach(sd -> sd.setIsNullable(true));
}
List<Expr> otherJoinConjuncts = hashJoin.getOtherJoinCondition()
@ -658,7 +662,8 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
.map(e -> ExpressionTranslator.translate(e, context))
.collect(Collectors.toList());
outputSlotReferences.stream().forEach(s -> context.createSlotDesc(outputDescriptor, s));
TupleDescriptor outputDescriptor = context.generateTupleDesc();
outputSlotReferences.forEach(s -> context.createSlotDesc(outputDescriptor, s));
hashJoinNode.setvOutputTupleDesc(outputDescriptor);
hashJoinNode.setvSrcToOutputSMap(srcToOutput);

View File

@ -489,9 +489,7 @@ public class Memo {
List<Plan> groupPlanChildren = childrenGroups.stream()
.map(GroupPlan::new)
.collect(ImmutableList.toImmutableList());
LogicalProperties logicalProperties = plan.getLogicalProperties();
return plan.withChildren(groupPlanChildren)
.withLogicalProperties(Optional.of(logicalProperties));
return plan.withChildren(groupPlanChildren);
}
/*

View File

@ -17,19 +17,24 @@
package org.apache.doris.nereids.properties;
import org.apache.doris.nereids.trees.expressions.ExprId;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
/**
* Logical properties used for analysis and optimize in Nereids.
*/
public class LogicalProperties {
protected Supplier<List<Slot>> outputSupplier;
protected final Supplier<List<Slot>> outputSupplier;
protected final Supplier<HashSet<ExprId>> outputSetSupplier;
/**
* constructor of LogicalProperties.
@ -41,6 +46,10 @@ public class LogicalProperties {
this.outputSupplier = Suppliers.memoize(
Objects.requireNonNull(outputSupplier, "outputSupplier can not be null")
);
this.outputSetSupplier = Suppliers.memoize(
() -> outputSupplier.get().stream().map(NamedExpression::getExprId)
.collect(Collectors.toCollection(HashSet::new))
);
}
public List<Slot> getOutput() {
@ -60,11 +69,11 @@ public class LogicalProperties {
return false;
}
LogicalProperties that = (LogicalProperties) o;
return Objects.equals(outputSupplier.get(), that.outputSupplier.get());
return Objects.equals(outputSetSupplier.get(), that.outputSetSupplier.get());
}
@Override
public int hashCode() {
return Objects.hash(outputSupplier.get());
return Objects.hash(outputSetSupplier.get());
}
}

View File

@ -20,7 +20,6 @@ package org.apache.doris.nereids.rules;
import org.apache.doris.nereids.rules.exploration.join.InnerJoinLAsscom;
import org.apache.doris.nereids.rules.exploration.join.InnerJoinLAsscomProject;
import org.apache.doris.nereids.rules.exploration.join.JoinCommute;
import org.apache.doris.nereids.rules.exploration.join.JoinCommuteProject;
import org.apache.doris.nereids.rules.exploration.join.OuterJoinLAsscom;
import org.apache.doris.nereids.rules.exploration.join.OuterJoinLAsscomProject;
import org.apache.doris.nereids.rules.exploration.join.SemiJoinLogicalJoinTranspose;
@ -57,7 +56,6 @@ import java.util.List;
public class RuleSet {
public static final List<Rule> EXPLORATION_RULES = planRuleFactories()
.add(JoinCommute.LEFT_DEEP)
.add(JoinCommuteProject.LEFT_DEEP)
.add(InnerJoinLAsscom.INSTANCE)
.add(InnerJoinLAsscomProject.INSTANCE)
.add(OuterJoinLAsscom.INSTANCE)

View File

@ -23,9 +23,6 @@ import org.apache.doris.nereids.rules.exploration.OneExplorationRuleFactory;
import org.apache.doris.nereids.rules.exploration.join.JoinCommuteHelper.SwapType;
import org.apache.doris.nereids.trees.plans.GroupPlan;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.util.PlanUtils;
import java.util.ArrayList;
/**
* Join Commute
@ -58,7 +55,7 @@ public class JoinCommute extends OneExplorationRuleFactory {
newJoin.getJoinReorderContext().setHasCommuteZigZag(true);
}
return PlanUtils.project(new ArrayList<>(join.getOutput()), newJoin).get();
return newJoin;
}).toRule(RuleType.LOGICAL_JOIN_COMMUTATE);
}
}

View File

@ -1,68 +0,0 @@
// 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.doris.nereids.rules.exploration.join;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.rules.exploration.OneExplorationRuleFactory;
import org.apache.doris.nereids.rules.exploration.join.JoinCommuteHelper.SwapType;
import org.apache.doris.nereids.trees.plans.GroupPlan;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.util.PlanUtils;
import java.util.ArrayList;
/**
* Project-Join Commute.
* This rule can prevent double JoinCommute cause dead-loop in Memo.
*/
public class JoinCommuteProject extends OneExplorationRuleFactory {
public static final JoinCommuteProject LEFT_DEEP = new JoinCommuteProject(SwapType.LEFT_DEEP);
public static final JoinCommuteProject ZIG_ZAG = new JoinCommuteProject(SwapType.ZIG_ZAG);
public static final JoinCommuteProject BUSHY = new JoinCommuteProject(SwapType.BUSHY);
private final SwapType swapType;
public JoinCommuteProject(SwapType swapType) {
this.swapType = swapType;
}
@Override
public Rule build() {
return logicalProject(logicalJoin())
.when(project -> JoinCommuteHelper.check(swapType, project.child()))
.then(project -> {
LogicalJoin<GroupPlan, GroupPlan> join = project.child();
// prevent this join match by JoinCommute.
join.getGroupExpression().get().setApplied(RuleType.LOGICAL_JOIN_COMMUTATE);
LogicalJoin<GroupPlan, GroupPlan> newJoin = new LogicalJoin<>(
join.getJoinType().swap(),
join.getHashJoinConjuncts(),
join.getOtherJoinCondition(),
join.right(), join.left(),
join.getJoinReorderContext());
newJoin.getJoinReorderContext().setHasCommute(true);
if (swapType == SwapType.ZIG_ZAG && JoinCommuteHelper.isNotBottomJoin(join)) {
newJoin.getJoinReorderContext().setHasCommuteZigZag(true);
}
return PlanUtils.project(new ArrayList<>(project.getProjects()), newJoin).get();
}).toRule(RuleType.LOGICAL_JOIN_COMMUTATE);
}
}

View File

@ -96,6 +96,10 @@ class JoinLAsscomHelper extends ThreeJoinHelper {
topJoin.getJoinReorderContext());
newTopJoin.getJoinReorderContext().setHasLAsscom(true);
if (topJoin.getLogicalProperties().equals(newTopJoin.getLogicalProperties())) {
return newTopJoin;
}
return PlanUtils.projectOrSelf(new ArrayList<>(topJoin.getOutput()), newTopJoin);
}

View File

@ -87,9 +87,9 @@ public class InnerJoinLAsscomProjectTest {
Assertions.assertEquals(2, root.getLogicalExpressions().size());
Assertions.assertTrue(root.logicalExpressionsAt(0).getPlan() instanceof LogicalJoin);
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalProject);
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalJoin);
GroupExpression newTopJoinGroupExpr = root.logicalExpressionsAt(1).child(0).getLogicalExpression();
GroupExpression newTopJoinGroupExpr = root.logicalExpressionsAt(1);
GroupExpression leftProjectGroupExpr = newTopJoinGroupExpr.child(0).getLogicalExpression();
GroupExpression rightProjectGroupExpr = newTopJoinGroupExpr.child(1).getLogicalExpression();
Plan leftProject = newTopJoinGroupExpr.child(0).getLogicalExpression().getPlan();

View File

@ -25,7 +25,6 @@ import org.apache.doris.nereids.trees.plans.JoinType;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import org.apache.doris.nereids.util.MemoTestUtils;
import org.apache.doris.nereids.util.PlanChecker;
import org.apache.doris.nereids.util.PlanConstructor;
@ -76,9 +75,9 @@ public class InnerJoinLAsscomTest {
Assertions.assertEquals(2, root.getLogicalExpressions().size());
Assertions.assertTrue(root.logicalExpressionsAt(0).getPlan() instanceof LogicalJoin);
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalProject);
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalJoin);
GroupExpression newTopJoinGroupExpr = root.logicalExpressionsAt(1).child(0).getLogicalExpression();
GroupExpression newTopJoinGroupExpr = root.logicalExpressionsAt(1);
GroupExpression newBottomJoinGroupExpr = newTopJoinGroupExpr.child(0).getLogicalExpression();
Plan bottomLeft = newBottomJoinGroupExpr.child(0).getLogicalExpression().getPlan();
Plan bottomRight = newBottomJoinGroupExpr.child(1).getLogicalExpression().getPlan();

View File

@ -25,7 +25,6 @@ import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import org.apache.doris.nereids.util.LogicalPlanBuilder;
import org.apache.doris.nereids.util.MemoTestUtils;
import org.apache.doris.nereids.util.PlanChecker;
@ -51,9 +50,9 @@ public class JoinCommuteTest {
Assertions.assertEquals(2, root.getLogicalExpressions().size());
Assertions.assertTrue(root.logicalExpressionsAt(0).getPlan() instanceof LogicalJoin);
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalProject);
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalJoin);
GroupExpression newJoinGroupExpr = root.logicalExpressionsAt(1).child(0).getLogicalExpression();
GroupExpression newJoinGroupExpr = root.logicalExpressionsAt(1);
Plan left = newJoinGroupExpr.child(0).getLogicalExpression().getPlan();
Plan right = newJoinGroupExpr.child(1).getLogicalExpression().getPlan();
Assertions.assertTrue(left instanceof LogicalOlapScan);

View File

@ -1,17 +1,17 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !scalar_less_than_corr --
1 3
1 2
1 3
2 4
3 4
3 3
3 4
-- !scalar_not_equal_corr --
1 3
1 2
1 3
2 4
3 4
3 3
3 4
20 2
22 3
24 4
@ -19,10 +19,10 @@
-- !scalar_equal_to_corr --
-- !not_in_corr --
1 3
1 2
2 5
1 3
2 4
2 5
3 4
20 2
22 3
@ -35,11 +35,11 @@
2 5
-- !exist_corr --
1 3
1 2
1 3
2 4
3 4
3 3
3 4
20 2
22 3
24 4
@ -49,108 +49,108 @@
-- !exist_and_not_exist --
-- !scalar_unCorrelated --
1 3
1 2
2 5
1 3
2 4
3 4
2 5
3 3
3 4
-- !scalar_equal_to_uncorr --
-- !not_scalar_unCorrelated --
1 3
1 2
2 5
1 3
2 4
3 4
2 5
3 3
3 4
20 2
22 3
24 4
-- !scalar_not_equal_uncorr --
1 3
1 2
2 5
1 3
2 4
3 4
2 5
3 3
3 4
20 2
22 3
24 4
-- !in_unCorrelated --
2 5
2 4
3 4
2 5
3 3
3 4
-- !in_subquery_uncorr --
2 5
2 4
3 4
2 5
3 3
3 4
-- !not_in_unCorrelated --
1 3
1 2
1 3
20 2
22 3
24 4
-- !not_in_uncorr --
1 3
1 2
1 3
20 2
22 3
24 4
-- !exist_unCorrelated --
1 3
1 2
2 5
1 3
2 4
3 4
2 5
3 3
3 4
20 2
22 3
24 4
-- !exist_uncorr --
1 3
1 2
2 5
1 3
2 4
3 4
2 5
3 3
3 4
20 2
22 3
24 4
-- !alias_scalar --
1 3
1 2
1 3
-- !alias_in --
3 3
-- !alias_not_in --
1 3
1 2
2 5
1 3
2 4
2 5
3 4
20 2
22 3
24 4
-- !alias_exist --
1 3
1 2
1 3
2 4
3 4
3 3
3 4
20 2
22 3
24 4

View File

@ -21,8 +21,8 @@
-- !select_5 --
1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK 1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK
1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK 1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL
1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL 1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK
1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK 1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL
1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL 1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL
1310179 6 1312 1455 29 19921110 3-MEDIUM 0 15 1705830 20506457 10 1535247 68233 8 19930114 FOB 1310179 6 1312 1455 29 19921110 3-MEDIUM 0 15 1705830 20506457 10 1535247 68233 8 19930114 FOB

View File

@ -20,10 +20,19 @@ suite ("sub_query_correlated") {
sql """
SET enable_vectorized_engine=true
"""
sql """
SET enable_nereids_planner=true
"""
sql """
SET enable_bucket_shuffle_join=false
"""
sql """
SET disable_colocate_plan=true
"""
sql """
DROP TABLE IF EXISTS `subquery1`
"""
@ -93,133 +102,133 @@ suite ("sub_query_correlated") {
//------------------Correlated-----------------
qt_scalar_less_than_corr """
select * from subquery1 where subquery1.k1 < (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1
select * from subquery1 where subquery1.k1 < (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
"""
qt_scalar_not_equal_corr """
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
"""
qt_scalar_equal_to_corr """
select * from subquery1 where subquery1.k1 = (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1
select * from subquery1 where subquery1.k1 = (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
"""
qt_not_in_corr """
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3 where subquery3.v2 = subquery1.k2) order by k1
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
"""
qt_in_subquery_corr """
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3 where subquery3.v2 = subquery1.k2) order by k1
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
"""
qt_not_exist_corr """
select * from subquery1 where not exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2) order by k1
select * from subquery1 where not exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2) order by k1, k2
"""
qt_exist_corr """
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2) order by k1
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2) order by k1, k2
"""
qt_in_with_in_and_scalar """
select * from subquery1 where subquery1.k1 in (
select subquery3.k3 from subquery3 where
subquery3.k3 in (select subquery4.k1 from subquery4 where subquery4.k1 = 3)
and subquery3.v2 > (select sum(subquery2.k2) from subquery2 where subquery2.k2 = subquery3.v1)) order by k1
and subquery3.v2 > (select sum(subquery2.k2) from subquery2 where subquery2.k2 = subquery3.v1)) order by k1, k2
"""
qt_exist_and_not_exist """
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2)
and not exists (select subquery4.k2 from subquery4 where subquery1.k2 = subquery4.k2) order by k1
and not exists (select subquery4.k2 from subquery4 where subquery1.k2 = subquery4.k2) order by k1, k2
"""
//------------------unCorrelated-----------------
qt_scalar_unCorrelated """
select * from subquery1 where subquery1.k1 < (select sum(subquery3.k3) from subquery3 where subquery3.v2 = 2) order by k1
select * from subquery1 where subquery1.k1 < (select sum(subquery3.k3) from subquery3 where subquery3.v2 = 2) order by k1, k2
"""
qt_scalar_equal_to_uncorr """
select * from subquery1 where subquery1.k1 = (select sum(subquery3.k3) from subquery3) order by k1
select * from subquery1 where subquery1.k1 = (select sum(subquery3.k3) from subquery3) order by k1, k2
"""
qt_not_scalar_unCorrelated """
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3 where subquery3.v2 = 2) order by k1
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3 where subquery3.v2 = 2) order by k1, k2
"""
qt_scalar_not_equal_uncorr """
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3) order by k1
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3) order by k1, k2
"""
qt_in_unCorrelated """
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1, k2
"""
qt_in_subquery_uncorr """
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3) order by k1
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3) order by k1, k2
"""
qt_not_in_unCorrelated """
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1, k2
"""
qt_not_in_uncorr """
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3) order by k1
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3) order by k1, k2
"""
qt_exist_unCorrelated """
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1, k2
"""
qt_exist_uncorr """
select * from subquery1 where exists (select subquery3.k3 from subquery3) order by k1
select * from subquery1 where exists (select subquery3.k3 from subquery3) order by k1, k2
"""
//----------with subquery alias----------
qt_alias_scalar """
select * from subquery1
where subquery1.k1 < (select max(aa) from
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
"""
qt_alias_in """
select * from subquery1
where subquery1.k1 in (select aa from
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
"""
qt_alias_not_in """
select * from subquery1
where subquery1.k1 not in (select aa from
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
"""
qt_alias_exist """
select * from subquery1
where exists (select aa from
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
"""
qt_alias_not_exist """
select * from subquery1
where not exists (select aa from
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
"""
//----------complex subqueries----------
qt_scalar_subquery """
select * from subquery1
where k1 = (select sum(k1) from subquery3 where subquery1.k1 = subquery3.v1 and subquery3.v2 = 2)
order by k1
order by k1, k2
"""
qt_in_subquery """
select * from subquery3
where (k1 = 1 or k1 = 2 or k1 = 3) and v1 in (select k1 from subquery1 where subquery1.k2 = subquery3.v2 and subquery1.k1 = 3)
order by k1
order by k1, k2
"""
qt_exist_subquery """
select * from subquery3
where k1 = 2 and exists (select * from subquery1 where subquery1.k1 = subquery3.v2 and subquery1.k2 = 4)
order by k1;
order by k1, k2
"""
}

View File

@ -24,6 +24,14 @@ suite("view") {
SET enable_nereids_planner=true
"""
sql """
SET enable_nereids_planner=true
"""
sql """
SET enable_bucket_shuffle_join=false
"""
sql """
create view if not exists v1 as
select *