Skip to content

Commit

Permalink
planner: fixing wrong result after applying predicate push down for C…
Browse files Browse the repository at this point in the history
…TEs (#47891) (#48195)

close #47881
  • Loading branch information
ti-chi-bot authored Nov 6, 2023
1 parent 2b7742e commit 0b8815f
Show file tree
Hide file tree
Showing 9 changed files with 194 additions and 11 deletions.
1 change: 1 addition & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ go_library(
"point_get_plan.go",
"preprocess.go",
"property_cols_prune.go",
"recheck_cte.go",
"resolve_indices.go",
"rule_aggregation_elimination.go",
"rule_aggregation_push_down.go",
Expand Down
18 changes: 13 additions & 5 deletions pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4858,13 +4858,21 @@ func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName
}

if cte.cteClass == nil {
cte.cteClass = &CTEClass{IsDistinct: cte.isDistinct, seedPartLogicalPlan: cte.seedLP,
recursivePartLogicalPlan: cte.recurLP, IDForStorage: cte.storageID,
optFlag: cte.optFlag, HasLimit: hasLimit, LimitBeg: limitBeg,
LimitEnd: limitEnd, pushDownPredicates: make([]expression.Expression, 0), ColumnMap: make(map[string]*expression.Column)}
cte.cteClass = &CTEClass{
IsDistinct: cte.isDistinct,
seedPartLogicalPlan: cte.seedLP,
recursivePartLogicalPlan: cte.recurLP,
IDForStorage: cte.storageID,
optFlag: cte.optFlag,
HasLimit: hasLimit,
LimitBeg: limitBeg,
LimitEnd: limitEnd,
pushDownPredicates: make([]expression.Expression, 0),
ColumnMap: make(map[string]*expression.Column),
}
}
var p LogicalPlan
lp := LogicalCTE{cteAsName: tn.Name, cteName: tn.Name, cte: cte.cteClass, seedStat: cte.seedStat, isOuterMostCTE: !b.buildingCTE}.Init(b.ctx, b.getSelectOffset())
lp := LogicalCTE{cteAsName: tn.Name, cteName: tn.Name, cte: cte.cteClass, seedStat: cte.seedStat}.Init(b.ctx, b.getSelectOffset())
prevSchema := cte.seedLP.Schema().Clone()
lp.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars()))

Expand Down
10 changes: 5 additions & 5 deletions pkg/planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -2338,6 +2338,7 @@ type CTEClass struct {
// pushDownPredicates may be push-downed by different references.
pushDownPredicates []expression.Expression
ColumnMap map[string]*expression.Column
isOuterMostCTE bool
}

const emptyCTEClassSize = int64(unsafe.Sizeof(CTEClass{}))
Expand Down Expand Up @@ -2369,11 +2370,10 @@ func (cc *CTEClass) MemoryUsage() (sum int64) {
type LogicalCTE struct {
logicalSchemaProducer

cte *CTEClass
cteAsName model.CIStr
cteName model.CIStr
seedStat *property.StatsInfo
isOuterMostCTE bool
cte *CTEClass
cteAsName model.CIStr
cteName model.CIStr
seedStat *property.StatsInfo

onlyUsedAsStorage bool
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,9 @@ func BuildLogicalPlanForTest(ctx context.Context, sctx sessionctx.Context, node
if err != nil {
return nil, nil, err
}
if logic, ok := p.(LogicalPlan); ok {
RecheckCTE(logic)
}
return p, p.OutputNames(), err
}

Expand Down
53 changes: 53 additions & 0 deletions pkg/planner/core/recheck_cte.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
// Copyright 2023 PingCAP, Inc.
//
// Licensed 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 core

import "github.com/pingcap/tidb/pkg/planner/funcdep"

// RecheckCTE fills the IsOuterMostCTE field for CTEs.
// It's a temp solution to before we fully use the Sequence to optimize the CTEs.
// This func checks whether the CTE is referenced only by the main query or not.
func RecheckCTE(p LogicalPlan) {
visited := funcdep.NewFastIntSet()
findCTEs(p, &visited, true)
}

func findCTEs(
p LogicalPlan,
visited *funcdep.FastIntSet,
isRootTree bool,
) {
if cteReader, ok := p.(*LogicalCTE); ok {
cte := cteReader.cte
if !isRootTree {
// Set it to false since it's referenced by other CTEs.
cte.isOuterMostCTE = false
}
if visited.Has(cte.IDForStorage) {
return
}
visited.Insert(cte.IDForStorage)
// Set it when we meet it first time.
cte.isOuterMostCTE = isRootTree
findCTEs(cte.seedPartLogicalPlan, visited, false)
if cte.recursivePartLogicalPlan != nil {
findCTEs(cte.recursivePartLogicalPlan, visited, false)
}
return
}
for _, child := range p.Children() {
findCTEs(child, visited, isRootTree)
}
}
2 changes: 1 addition & 1 deletion pkg/planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -998,7 +998,7 @@ func (p *LogicalCTE) PredicatePushDown(predicates []expression.Expression, _ *lo
// Doesn't support recursive CTE yet.
return predicates, p.self
}
if !p.isOuterMostCTE {
if !p.cte.isOuterMostCTE {
return predicates, p.self
}
pushedPredicates := make([]expression.Expression, len(predicates))
Expand Down
2 changes: 2 additions & 0 deletions pkg/planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -520,6 +520,8 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
return p, names, 0, nil
}

core.RecheckCTE(logic)

// Handle the logical plan statement, use cascades planner if enabled.
if sessVars.GetEnableCascadesPlanner() {
finalPlan, cost, err := cascades.DefaultOptimizer.FindBestPlan(sctx, logic)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,3 +121,62 @@ select /*+ use_index_merge( tbl_39) */ col_239 from tbl_39 where not( tbl_39.c
col_239
1994
1997
drop table if exists t, t1, t2;
create table t (id int,name varchar(10));
insert into t values(1,'tt');
create table t1(id int,name varchar(10),name1 varchar(10),name2 varchar(10));
insert into t1 values(1,'tt','ttt','tttt'),(2,'dd','ddd','dddd');
create table t2(id int,name varchar(10),name1 varchar(10),name2 varchar(10),`date1` date);
insert into t2 values(1,'tt','ttt','tttt','2099-12-31'),(2,'dd','ddd','dddd','2099-12-31');
WITH bzzs AS (
SELECT
count(1) AS bzn
FROM
t c
),
tmp1 AS (
SELECT
t1.*
FROM
t1
LEFT JOIN bzzs ON 1 = 1
WHERE
name IN ('tt')
AND bzn <> 1
),
tmp2 AS (
SELECT
tmp1.*,
date('2099-12-31') AS endate
FROM
tmp1
),
tmp3 AS (
SELECT
*
FROM
tmp2
WHERE
endate > CURRENT_DATE
UNION ALL
SELECT
'1' AS id,
'ss' AS name,
'sss' AS name1,
'ssss' AS name2,
date('2099-12-31') AS endate
FROM
bzzs t1
WHERE
bzn = 1
)
SELECT
c2.id,
c3.id
FROM
t2 db
LEFT JOIN tmp3 c2 ON c2.id = '1'
LEFT JOIN tmp3 c3 ON c3.id = '1';
id id
1 1
1 1
57 changes: 57 additions & 0 deletions tests/integrationtest/t/planner/core/issuetest/planner_issue.test
Original file line number Diff line number Diff line change
Expand Up @@ -79,3 +79,60 @@ insert into tbl_39 values (1994),(1995),(1996),(1997);
explain select /*+ use_index_merge( tbl_39) */ col_239 from tbl_39 where not( tbl_39.col_239 not in ( '1994' ) ) and tbl_39.col_239 not in ( '2004' , '2010' , '2010' ) or not( tbl_39.col_239 <= '1996' ) and not( tbl_39.col_239 between '2026' and '2011' ) order by tbl_39.col_239 limit 382;
select /*+ use_index_merge( tbl_39) */ col_239 from tbl_39 where not( tbl_39.col_239 not in ( '1994' ) ) and tbl_39.col_239 not in ( '2004' , '2010' , '2010' ) or not( tbl_39.col_239 <= '1996' ) and not( tbl_39.col_239 between '2026' and '2011' ) order by tbl_39.col_239 limit 382;

# https://github.com/pingcap/tidb/issues/47881
drop table if exists t, t1, t2;
create table t (id int,name varchar(10));
insert into t values(1,'tt');
create table t1(id int,name varchar(10),name1 varchar(10),name2 varchar(10));
insert into t1 values(1,'tt','ttt','tttt'),(2,'dd','ddd','dddd');
create table t2(id int,name varchar(10),name1 varchar(10),name2 varchar(10),`date1` date);
insert into t2 values(1,'tt','ttt','tttt','2099-12-31'),(2,'dd','ddd','dddd','2099-12-31');
WITH bzzs AS (
SELECT
count(1) AS bzn
FROM
t c
),
tmp1 AS (
SELECT
t1.*
FROM
t1
LEFT JOIN bzzs ON 1 = 1
WHERE
name IN ('tt')
AND bzn <> 1
),
tmp2 AS (
SELECT
tmp1.*,
date('2099-12-31') AS endate
FROM
tmp1
),
tmp3 AS (
SELECT
*
FROM
tmp2
WHERE
endate > CURRENT_DATE
UNION ALL
SELECT
'1' AS id,
'ss' AS name,
'sss' AS name1,
'ssss' AS name2,
date('2099-12-31') AS endate
FROM
bzzs t1
WHERE
bzn = 1
)
SELECT
c2.id,
c3.id
FROM
t2 db
LEFT JOIN tmp3 c2 ON c2.id = '1'
LEFT JOIN tmp3 c3 ON c3.id = '1';

0 comments on commit 0b8815f

Please sign in to comment.