Skip to content

Commit

Permalink
planner: fix Merge hint for view (#36678) (#36760)
Browse files Browse the repository at this point in the history
close #36666
  • Loading branch information
ti-srebot authored Aug 1, 2022
1 parent 8211186 commit c5718d7
Show file tree
Hide file tree
Showing 6 changed files with 115 additions and 23 deletions.
2 changes: 1 addition & 1 deletion planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -339,7 +339,7 @@ func (er *expressionRewriter) buildSubquery(ctx context.Context, subq *ast.Subqu
er.b.hasValidSemiJoinHint = oldHasHint
}()

np, err = er.b.buildResultSetNode(ctx, subq.Query)
np, err = er.b.buildResultSetNode(ctx, subq.Query, false)
if err != nil {
return nil, false, err
}
Expand Down
63 changes: 44 additions & 19 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -342,10 +342,12 @@ func (b *PlanBuilder) buildTableRefs(ctx context.Context, from *ast.TableRefsCla
cte.recursiveRef = false
}
}()
return b.buildResultSetNode(ctx, from.TableRefs)
return b.buildResultSetNode(ctx, from.TableRefs, false)
}

func (b *PlanBuilder) buildResultSetNode(ctx context.Context, node ast.ResultSetNode) (p LogicalPlan, err error) {
func (b *PlanBuilder) buildResultSetNode(ctx context.Context, node ast.ResultSetNode, IsCTE bool) (p LogicalPlan, err error) {
//If it is building the CTE queries, we will mark them.
b.isCTE = IsCTE
switch x := node.(type) {
case *ast.Join:
return b.buildJoin(ctx, x)
Expand Down Expand Up @@ -686,19 +688,19 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (Logica
// "DELETE", "UPDATE", "REPLACE". For this scenario "joinNode.Right" is nil
// and we only build the left "ResultSetNode".
if joinNode.Right == nil {
return b.buildResultSetNode(ctx, joinNode.Left)
return b.buildResultSetNode(ctx, joinNode.Left, false)
}

b.optFlag = b.optFlag | flagPredicatePushDown
// Add join reorder flag regardless of inner join or outer join.
b.optFlag = b.optFlag | flagJoinReOrder

leftPlan, err := b.buildResultSetNode(ctx, joinNode.Left)
leftPlan, err := b.buildResultSetNode(ctx, joinNode.Left, false)
if err != nil {
return nil, err
}

rightPlan, err := b.buildResultSetNode(ctx, joinNode.Right)
rightPlan, err := b.buildResultSetNode(ctx, joinNode.Right, false)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -3797,10 +3799,20 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L
b.isForUpdateRead = true
}

// Determines whether to use the Merge hint in a CTE query.
if b.buildingCTE {
if hints := b.TableHints(); hints != nil {
b.outerCTEs[len(b.outerCTEs)-1].isInline = hints.MergeHints.preferMerge
// Verify Merge hints in the current query, we will update parameters for those that meet the rules, and warn those that do not.
//If the current query uses Merge Hint and the query is a CTE, we update the HINT information for the current query.
//If the current query is not a CTE query (it may be a subquery within a CTE query or an external non-CTE query), we will give a warning.
//In particular, recursive CTE have separate warnings, so they are no longer called.
if hints := b.TableHints(); hints != nil && hints.MergeHints.preferMerge {
if b.buildingCTE {
if b.isCTE {
b.outerCTEs[len(b.outerCTEs)-1].isInline = hints.MergeHints.preferMerge
} else if !b.buildingRecursivePartForCTE {
//If there has subquery which is not CTE and using `MERGE()` hint, we will show this warning;
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Hint merge() is inapplicable. Please check whether the hint is using in the right place, you should use this hint in CTE inner query."))
}
} else if !b.buildingCTE && !b.isCTE {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Hint merge() is inapplicable. Please check whether the hint is using in the right place, you should use this hint in CTE inner query."))
}
}

Expand Down Expand Up @@ -4020,11 +4032,6 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L
}
}

// If Merge hint is using in outer query, we will not apply this hint.
if hints := b.TableHints(); hints.MergeHints.preferMerge && !b.buildingCTE && len(b.tableHintInfo) == 1 {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Hint merge() is inapplicable. Please check whether the hint is using in outer query, you should use this hint in CTE inner query."))
}

sel.Fields.Fields = originalFields
if oldLen != p.Schema().Len() {
proj := LogicalProjection{Exprs: expression.Column2Exprs(p.Schema().Columns[:oldLen])}.Init(b.ctx, b.getSelectOffset())
Expand Down Expand Up @@ -4228,7 +4235,7 @@ func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName
}

func (b *PlanBuilder) buildDataSourceFromCTEMerge(ctx context.Context, cte *ast.CommonTableExpression) (LogicalPlan, error) {
p, err := b.buildResultSetNode(ctx, cte.Query.Query)
p, err := b.buildResultSetNode(ctx, cte.Query.Query, true)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -4834,6 +4841,23 @@ func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName model.
}
originalVisitInfo := b.visitInfo
b.visitInfo = make([]visitInfo, 0)

//For the case that views appear in CTE queries,
//we need to save the CTEs after the views are established.
var saveCte []*cteInfo
if len(b.outerCTEs) > 0 {
saveCte = make([]*cteInfo, len(b.outerCTEs))
copy(saveCte, b.outerCTEs)
} else {
saveCte = nil
}
o := b.buildingCTE
b.buildingCTE = false
defer func() {
b.outerCTEs = saveCte
b.buildingCTE = o
}()

selectLogicalPlan, err := b.Build(ctx, selectNode)
if err != nil {
if terror.ErrorNotEqual(err, ErrViewRecursive) &&
Expand Down Expand Up @@ -5142,7 +5166,7 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) (
}
}

p, err := b.buildResultSetNode(ctx, update.TableRefs.TableRefs)
p, err := b.buildResultSetNode(ctx, update.TableRefs.TableRefs, false)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -5524,7 +5548,7 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, ds *ast.DeleteStmt) (Plan
}
}

p, err := b.buildResultSetNode(ctx, ds.TableRefs.TableRefs)
p, err := b.buildResultSetNode(ctx, ds.TableRefs.TableRefs, false)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -6695,7 +6719,7 @@ func (b *PlanBuilder) buildCte(ctx context.Context, cte *ast.CommonTableExpressi
}
b.buildingRecursivePartForCTE = saveCheck
} else {
p, err = b.buildResultSetNode(ctx, cte.Query.Query)
p, err = b.buildResultSetNode(ctx, cte.Query.Query, true)
if err != nil {
return nil, err
}
Expand All @@ -6713,6 +6737,7 @@ func (b *PlanBuilder) buildCte(ctx context.Context, cte *ast.CommonTableExpressi

// buildRecursiveCTE handles the with clause `with recursive xxx as xx`.
func (b *PlanBuilder) buildRecursiveCTE(ctx context.Context, cte ast.ResultSetNode) error {
b.isCTE = true
cInfo := b.outerCTEs[len(b.outerCTEs)-1]
switch x := (cte).(type) {
case *ast.SetOprStmt:
Expand Down Expand Up @@ -6855,7 +6880,7 @@ func (b *PlanBuilder) buildRecursiveCTE(ctx context.Context, cte ast.ResultSetNo
}
return nil
default:
p, err := b.buildResultSetNode(ctx, x)
p, err := b.buildResultSetNode(ctx, x, true)
if err != nil {
// Refine the error message.
if errors.ErrorEqual(err, ErrCTERecursiveRequiresNonRecursiveFirst) {
Expand Down
2 changes: 2 additions & 0 deletions planner/core/physical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1121,6 +1121,7 @@ func TestCTEMergeHint(t *testing.T) {
tk.MustExec("drop table if exists t2")
tk.MustExec("drop table if exists t3")
tk.MustExec("drop table if exists t4")
tk.MustExec("drop view if exists v")
tk.MustExec("create table tc(a int)")
tk.MustExec("create table te(c int)")
tk.MustExec("create table t1(a int)")
Expand All @@ -1139,6 +1140,7 @@ func TestCTEMergeHint(t *testing.T) {
tk.MustExec("analyze table t2;")
tk.MustExec("analyze table t3;")
tk.MustExec("analyze table t4;")
tk.MustExec("create definer='root'@'localhost' view v as select * from tc")
var (
input []string
output []struct {
Expand Down
2 changes: 2 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -509,6 +509,8 @@ type PlanBuilder struct {
allocIDForCTEStorage int
buildingRecursivePartForCTE bool
buildingCTE bool
//Check whether the current building query is a CTE
isCTE bool

// checkSemiJoinHint checks whether the SEMI_JOIN_REWRITE hint is possible to be applied on the current SELECT stmt.
// We need this variable for the hint since the hint is set in subquery, but we check its availability in its outer scope.
Expand Down
6 changes: 5 additions & 1 deletion planner/core/testdata/plan_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -577,6 +577,7 @@
"cases": [
"with cte as (select /*+ MERGE() */ * from tc where tc.a < 60) select * from cte where cte.a <18",
"with cte as (select * from tc where tc.a < 60) select * from cte where cte.a <18",
"with cte as (select /*+ MERGE() */ * from v) select * from cte",
"WITH cte1 AS (SELECT /*+ MERGE() */ a FROM tc), cte2 AS (SELECT /*+ MERGE()*/ c FROM te) SELECT * FROM cte1 JOIN cte2 WHERE cte1.a = cte2.c;",
"WITH cte1 AS (SELECT a FROM tc), cte2 AS (SELECT /*+ MERGE() */ c FROM te) SELECT * FROM cte1 JOIN cte2 WHERE cte1.a = cte2.c;",
"with recursive cte1(c1) as (select 1 union select /*+ MERGE() */ c1 + 1 c1 from cte1 where c1 < 100) select * from cte1;",
Expand All @@ -589,7 +590,10 @@
"with cte1 as (select * from t1), cte2 as (with cte3 as (with cte5 as (select * from t2),cte6 as (select * from t3) select * from cte5,cte6) ,cte4 as (select /*+ MERGE() */ * from t4) select * from cte3,cte4) select * from cte1,cte2;",
"with cte1 as (select * from t1), cte2 as (with cte3 as (with cte5 as (select * from t2),cte6 as (select /*+ MERGE() */ * from t3) select * from cte5,cte6) ,cte4 as (select * from t4) select * from cte3,cte4) select * from cte1,cte2;",
"with cte2 as (with cte4 as (select * from tc) select * from te, cte4) select * from cte2;",
"with cte2 as (with cte4 as (select /*+ merge() */ * from tc) select * from te, cte4) select * from cte2;"
"with cte2 as (with cte4 as (select /*+ merge() */ * from tc) select * from te, cte4) select * from cte2;",
"with cte1 as (with cte2 as (with cte3 as (select /*+ MERGE() */ * from t2) select /*+ MERGE() */ * from cte3) select * from cte2,(select /*+ MERGE() */ * from t1) ttt) select * from cte1;",
"with cte1 as (with cte2 as (with cte3 as (select /*+ MERGE() */ * from t2) select /*+ MERGE() */ * from cte3) select * from cte2,(select * from t1) ttt) select * from cte1,(select /*+ MERGE() */ * from t3) ttw;",
"with cte1 as (with cte2 as (with cte3 as (select /*+ MERGE() */ * from t2) select /*+ MERGE() */ * from cte3) select * from cte2,(select * from t1) ttt) select * from cte1,(select * from t3) ttw;"
]
},
{
Expand Down
63 changes: 61 additions & 2 deletions planner/core/testdata/plan_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -1618,6 +1618,14 @@
],
"Warning": null
},
{
"SQL": "with cte as (select /*+ MERGE() */ * from v) select * from cte",
"Plan": [
"TableReader 7.00 root data:TableFullScan",
"└─TableFullScan 7.00 cop[tikv] table:tc keep order:false"
],
"Warning": null
},
{
"SQL": "WITH cte1 AS (SELECT /*+ MERGE()*/ a FROM tc), cte2 AS (SELECT /*+ MERGE()*/ c FROM te) SELECT * FROM cte1 JOIN cte2 WHERE cte1.a = cte2.c;",
"Plan": [
Expand Down Expand Up @@ -1679,7 +1687,7 @@
" └─TableFullScan 7.00 cop[tikv] table:t1 keep order:false"
],
"Warning": [
"[planner:1815]Hint merge() is inapplicable. Please check whether the hint is using in outer query, you should use this hint in CTE inner query."
"[planner:1815]Hint merge() is inapplicable. Please check whether the hint is using in the right place, you should use this hint in CTE inner query."
]
},
{
Expand Down Expand Up @@ -1866,7 +1874,58 @@
" └─TableFullScan 7.00 cop[tikv] table:te keep order:false"
],
"Warning": null
}
},
{
"SQL": "with cte1 as (with cte2 as (with cte3 as (select /*+ MERGE() */ * from t2) select /*+ MERGE() */ * from cte3) select * from cte2,(select /*+ MERGE() */ * from t1) ttt) select * from cte1;",
"Plan": [
"CTEFullScan 49.00 root CTE:cte1 data:CTE_0",
"CTE_0 49.00 root Non-Recursive CTE",
"└─HashJoin(Seed Part) 49.00 root CARTESIAN inner join",
" ├─TableReader(Build) 7.00 root data:TableFullScan",
" │ └─TableFullScan 7.00 cop[tikv] table:t1 keep order:false",
" └─TableReader(Probe) 7.00 root data:TableFullScan",
" └─TableFullScan 7.00 cop[tikv] table:t2 keep order:false"
],
"Warning": [
"[planner:1815]Hint merge() is inapplicable. Please check whether the hint is using in the right place, you should use this hint in CTE inner query."
]
},
{
"SQL": "with cte1 as (with cte2 as (with cte3 as (select /*+ MERGE() */ * from t2) select /*+ MERGE() */ * from cte3) select * from cte2,(select * from t1) ttt) select * from cte1,(select /*+ MERGE() */ * from t3) ttw;",
"Plan": [
"Projection 343.00 root test.t2.b, test.t1.a, test.t3.c",
"└─HashJoin 343.00 root CARTESIAN inner join",
" ├─TableReader(Build) 7.00 root data:TableFullScan",
" │ └─TableFullScan 7.00 cop[tikv] table:t3 keep order:false",
" └─CTEFullScan(Probe) 49.00 root CTE:cte1 data:CTE_0",
"CTE_0 49.00 root Non-Recursive CTE",
"└─HashJoin(Seed Part) 49.00 root CARTESIAN inner join",
" ├─TableReader(Build) 7.00 root data:TableFullScan",
" │ └─TableFullScan 7.00 cop[tikv] table:t1 keep order:false",
" └─TableReader(Probe) 7.00 root data:TableFullScan",
" └─TableFullScan 7.00 cop[tikv] table:t2 keep order:false"
],
"Warning": [
"[planner:1815]Hint merge() is inapplicable. Please check whether the hint is using in the right place, you should use this hint in CTE inner query."
]
},
{
"SQL": "with cte1 as (with cte2 as (with cte3 as (select /*+ MERGE() */ * from t2) select /*+ MERGE() */ * from cte3) select * from cte2,(select * from t1) ttt) select * from cte1,(select * from t3) ttw;",
"Plan": [
"Projection 343.00 root test.t2.b, test.t1.a, test.t3.c",
"└─HashJoin 343.00 root CARTESIAN inner join",
" ├─TableReader(Build) 7.00 root data:TableFullScan",
" │ └─TableFullScan 7.00 cop[tikv] table:t3 keep order:false",
" └─CTEFullScan(Probe) 49.00 root CTE:cte1 data:CTE_0",
"CTE_0 49.00 root Non-Recursive CTE",
"└─HashJoin(Seed Part) 49.00 root CARTESIAN inner join",
" ├─TableReader(Build) 7.00 root data:TableFullScan",
" │ └─TableFullScan 7.00 cop[tikv] table:t1 keep order:false",
" └─TableReader(Probe) 7.00 root data:TableFullScan",
" └─TableFullScan 7.00 cop[tikv] table:t2 keep order:false"
],
"Warning": null
}
]
},
{
Expand Down

0 comments on commit c5718d7

Please sign in to comment.