Skip to content

Commit b9a31b2

Browse files
authored
planner: add CloneForPlanCache method to Plan interface (#54285)
ref #54057
1 parent 9ee6421 commit b9a31b2

File tree

7 files changed

+171
-0
lines changed

7 files changed

+171
-0
lines changed

pkg/planner/core/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -230,6 +230,7 @@ go_test(
230230
"plan_cache_instance_test.go",
231231
"plan_cache_lru_test.go",
232232
"plan_cache_param_test.go",
233+
"plan_cache_rebuild_test.go",
233234
"plan_cache_test.go",
234235
"plan_cacheable_checker_test.go",
235236
"plan_cost_detail_test.go",

pkg/planner/core/base/plan_base.go

+6
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,12 @@ type Plan interface {
7979
QueryBlockOffset() int
8080

8181
BuildPlanTrace() *tracing.PlanTrace
82+
83+
// CloneForPlanCache clones this physical plan specially for instance level plan cache.
84+
// A cached plan might be shared across multiple sessions, so for safety we have to clone it to make it thread-safe.
85+
// Compared with the prior PhysicalPlan.Clone(), CloneForPlanCache() doesn't deep clones all fields instead it only
86+
// deep clones fields that might be modified during reusing it and shallow clones all other fields for performance.
87+
CloneForPlanCache() (cloned Plan, ok bool)
8288
}
8389

8490
// PhysicalPlan is a tree of the physical operators.

pkg/planner/core/operator/baseimpl/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@ go_library(
88
deps = [
99
"//pkg/expression",
1010
"//pkg/planner/context",
11+
"//pkg/planner/core/base",
1112
"//pkg/planner/property",
1213
"//pkg/types",
1314
"//pkg/util/stringutil",

pkg/planner/core/operator/baseimpl/plan.go

+6
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import (
2121

2222
"github.com/pingcap/tidb/pkg/expression"
2323
"github.com/pingcap/tidb/pkg/planner/context"
24+
"github.com/pingcap/tidb/pkg/planner/core/base"
2425
"github.com/pingcap/tidb/pkg/planner/property"
2526
"github.com/pingcap/tidb/pkg/types"
2627
"github.com/pingcap/tidb/pkg/util/stringutil"
@@ -136,3 +137,8 @@ func (p *Plan) BuildPlanTrace() *tracing.PlanTrace {
136137
planTrace := &tracing.PlanTrace{ID: p.ID(), TP: p.TP()}
137138
return planTrace
138139
}
140+
141+
// CloneForPlanCache clones this Plan for instance plan cache.
142+
func (*Plan) CloneForPlanCache() (cloned base.Plan, ok bool) {
143+
return nil, false
144+
}

pkg/planner/core/physical_plans.go

+56
Original file line numberDiff line numberDiff line change
@@ -280,6 +280,19 @@ func (p *PhysicalTableReader) Clone() (base.PhysicalPlan, error) {
280280
return cloned, nil
281281
}
282282

283+
// CloneForPlanCache implements Plan.CloneForPlanCache method.
284+
func (p *PhysicalTableReader) CloneForPlanCache() (base.Plan, bool) {
285+
cloned := new(PhysicalTableReader)
286+
*cloned = *p
287+
t, ok := p.tablePlan.CloneForPlanCache()
288+
if !ok {
289+
return nil, false
290+
}
291+
cloned.tablePlan = t.(base.PhysicalPlan)
292+
cloned.TablePlans = flattenPushDownPlan(cloned.tablePlan)
293+
return cloned, true
294+
}
295+
283296
// SetChildren overrides op.PhysicalPlan SetChildren interface.
284297
func (p *PhysicalTableReader) SetChildren(children ...base.PhysicalPlan) {
285298
p.tablePlan = children[0]
@@ -342,6 +355,19 @@ func (p *PhysicalIndexReader) Clone() (base.PhysicalPlan, error) {
342355
return cloned, err
343356
}
344357

358+
// CloneForPlanCache implements Plan.CloneForPlanCache method.
359+
func (p *PhysicalIndexReader) CloneForPlanCache() (base.Plan, bool) {
360+
cloned := new(PhysicalIndexReader)
361+
*cloned = *p
362+
t, ok := p.indexPlan.CloneForPlanCache()
363+
if !ok {
364+
return nil, false
365+
}
366+
cloned.indexPlan = t.(base.PhysicalPlan)
367+
cloned.IndexPlans = flattenPushDownPlan(cloned.indexPlan)
368+
return cloned, true
369+
}
370+
345371
// SetSchema overrides op.PhysicalPlan SetSchema interface.
346372
func (p *PhysicalIndexReader) SetSchema(_ *expression.Schema) {
347373
if p.indexPlan != nil {
@@ -768,6 +794,14 @@ func (p *PhysicalIndexScan) Clone() (base.PhysicalPlan, error) {
768794
return cloned, nil
769795
}
770796

797+
// CloneForPlanCache implements op.CloneForPlanCache interface.
798+
func (p *PhysicalIndexScan) CloneForPlanCache() (base.Plan, bool) {
799+
cloned := new(PhysicalIndexScan)
800+
*cloned = *p
801+
cloned.Ranges = util.CloneRanges(p.Ranges)
802+
return cloned, true
803+
}
804+
771805
// ExtractCorrelatedCols implements op.PhysicalPlan interface.
772806
func (p *PhysicalIndexScan) ExtractCorrelatedCols() []*expression.CorrelatedColumn {
773807
corCols := make([]*expression.CorrelatedColumn, 0, len(p.AccessCondition))
@@ -949,6 +983,14 @@ func (ts *PhysicalTableScan) Clone() (base.PhysicalPlan, error) {
949983
return clonedScan, nil
950984
}
951985

986+
// CloneForPlanCache implements op.CloneForPlanCache interface.
987+
func (ts *PhysicalTableScan) CloneForPlanCache() (base.Plan, bool) {
988+
cloned := new(PhysicalTableScan)
989+
*cloned = *ts
990+
cloned.Ranges = util.CloneRanges(ts.Ranges)
991+
return cloned, true
992+
}
993+
952994
// ExtractCorrelatedCols implements op.PhysicalPlan interface.
953995
func (ts *PhysicalTableScan) ExtractCorrelatedCols() []*expression.CorrelatedColumn {
954996
corCols := make([]*expression.CorrelatedColumn, 0, len(ts.AccessCondition)+len(ts.LateMaterializationFilterCondition))
@@ -2223,6 +2265,20 @@ func (p *PhysicalSelection) Clone() (base.PhysicalPlan, error) {
22232265
return cloned, nil
22242266
}
22252267

2268+
// CloneForPlanCache implements base.Plan.CloneForPlanCache method.
2269+
func (p *PhysicalSelection) CloneForPlanCache() (base.Plan, bool) {
2270+
cloned := new(PhysicalSelection)
2271+
*cloned = *p
2272+
for i, c := range p.children {
2273+
clonedChild, ok := c.CloneForPlanCache()
2274+
if !ok {
2275+
return nil, false
2276+
}
2277+
cloned.children[i] = clonedChild.(base.PhysicalPlan)
2278+
}
2279+
return cloned, true
2280+
}
2281+
22262282
// ExtractCorrelatedCols implements op.PhysicalPlan interface.
22272283
func (p *PhysicalSelection) ExtractCorrelatedCols() []*expression.CorrelatedColumn {
22282284
corCols := make([]*expression.CorrelatedColumn, 0, len(p.Conditions))

pkg/planner/core/plan_cache.go

+13
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@ import (
3636
"github.com/pingcap/tidb/pkg/util/chunk"
3737
contextutil "github.com/pingcap/tidb/pkg/util/context"
3838
"github.com/pingcap/tidb/pkg/util/dbterror/plannererrors"
39+
"github.com/pingcap/tidb/pkg/util/intest"
3940
"github.com/pingcap/tidb/pkg/util/kvcache"
4041
)
4142

@@ -48,6 +49,12 @@ type PlanCacheKeyTestIssue46760 struct{}
4849
// PlanCacheKeyTestIssue47133 is only for test.
4950
type PlanCacheKeyTestIssue47133 struct{}
5051

52+
// PlanCacheKeyTestBeforeAdjust is only for test.
53+
type PlanCacheKeyTestBeforeAdjust struct{}
54+
55+
// PlanCacheKeyTestAfterAdjust is only for test.
56+
type PlanCacheKeyTestAfterAdjust struct{}
57+
5158
// SetParameterValuesIntoSCtx sets these parameters into session context.
5259
func SetParameterValuesIntoSCtx(sctx base.PlanContext, isNonPrep bool, markers []ast.ParamMarkerExpr, params []expression.Expression) error {
5360
vars := sctx.GetSessionVars()
@@ -239,7 +246,13 @@ func GetPlanFromPlanCache(ctx context.Context, sctx sessionctx.Context,
239246
cacheVal, hit = sctx.GetSessionPlanCache().Get(cacheKey, matchOpts)
240247
}
241248
if hit {
249+
if intest.InTest && ctx.Value(PlanCacheKeyTestBeforeAdjust{}) != nil {
250+
ctx.Value(PlanCacheKeyTestBeforeAdjust{}).(func(cachedVal *PlanCacheValue))(cacheVal.(*PlanCacheValue))
251+
}
242252
if plan, names, ok, err := adjustCachedPlan(sctx, cacheVal.(*PlanCacheValue), isNonPrepared, isPointPlan, cacheKey, bindSQL, is, stmt); err != nil || ok {
253+
if intest.InTest && ctx.Value(PlanCacheKeyTestAfterAdjust{}) != nil {
254+
ctx.Value(PlanCacheKeyTestAfterAdjust{}).(func(cachedVal *PlanCacheValue))(cacheVal.(*PlanCacheValue))
255+
}
243256
return plan, names, err
244257
}
245258
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,88 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package core_test
16+
17+
import (
18+
"context"
19+
"encoding/json"
20+
"testing"
21+
22+
"github.com/pingcap/tidb/pkg/planner/core"
23+
"github.com/pingcap/tidb/pkg/planner/core/base"
24+
"github.com/pingcap/tidb/pkg/testkit"
25+
"github.com/stretchr/testify/require"
26+
)
27+
28+
func TestPlanCacheClone(t *testing.T) {
29+
store := testkit.CreateMockStore(t)
30+
tk := testkit.NewTestKit(t, store)
31+
tk.MustExec(`use test`)
32+
tk.MustExec(`create table t (a int, b int, c int, primary key(a), key(b))`)
33+
34+
// TableScan
35+
testCachedPlanClone(t, tk, `prepare st from 'select * from t where a<?'`,
36+
`set @a1=1, @a2=2`, `execute st using @a1`, `execute st using @a2`)
37+
testCachedPlanClone(t, tk, `prepare st from 'select * from t where a>=?'`,
38+
`set @a1=1, @a2=2`, `execute st using @a1`, `execute st using @a2`)
39+
40+
//// IndexScan
41+
//testCachedPlanClone(t, tk, `prepare st from 'select * from t use index(b) where b<=?'`,
42+
// `set @a1=1, @a2=2`, `execute st using @a1`, `execute st using @a2`)
43+
//testCachedPlanClone(t, tk, `prepare st from 'select * from t use index(b) where b>?'`,
44+
// `set @a1=1, @a2=2`, `execute st using @a1`, `execute st using @a2`)
45+
46+
// TableScan + Selection
47+
testCachedPlanClone(t, tk, `prepare st from 'select * from t use index(primary) where a<? and b<?'`,
48+
`set @a1=1, @b1=1, @a2=2, @b2=2`, `execute st using @a1,@b1`, `execute st using @a2,@b2`)
49+
testCachedPlanClone(t, tk, `prepare st from 'select * from t use index(primary) where a<? and b+?=10'`,
50+
`set @a1=1, @b1=1, @a2=2, @b2=2`, `execute st using @a1,@b1`, `execute st using @a2,@b2`)
51+
52+
//// IndexScan + Selection
53+
//testCachedPlanClone(t, tk, `prepare st from 'select * from t use index(b) where a<? and b<?'`,
54+
// `set @a1=1, @b1=1, @a2=2, @b2=2`, `execute st using @a1,@b1`, `execute st using @a2,@b2`)
55+
//testCachedPlanClone(t, tk, `prepare st from 'select * from t use index(b) where a<? and b+?=10'`,
56+
// `set @a1=1, @b1=1, @a2=2, @b2=2`, `execute st using @a1,@b1`, `execute st using @a2,@b2`)
57+
}
58+
59+
func testCachedPlanClone(t *testing.T, tk *testkit.TestKit, prep, set, exec1, exec2 string) {
60+
tk.MustExec(prep)
61+
tk.MustExec(set)
62+
tk.MustQuery(exec1) // generate the first cached plan
63+
64+
var original base.Plan
65+
var originalFingerprint string
66+
before := func(cachedVal *core.PlanCacheValue) {
67+
// get the current cached plan and its fingerprint
68+
original, originalFingerprint = cachedVal.Plan, planFingerprint(t, cachedVal.Plan)
69+
// replace the cached plan with a cloned one
70+
cloned, ok := original.CloneForPlanCache()
71+
require.True(t, ok)
72+
cachedVal.Plan = cloned
73+
}
74+
after := func(cachedVal *core.PlanCacheValue) {
75+
cloned := cachedVal.Plan
76+
require.True(t, originalFingerprint != planFingerprint(t, cloned)) // this cloned one have been adjusted by the optimizer
77+
require.True(t, originalFingerprint == planFingerprint(t, original)) // the prior one should keep unchanged
78+
}
79+
ctx := context.WithValue(context.Background(), core.PlanCacheKeyTestBeforeAdjust{}, before)
80+
ctx = context.WithValue(ctx, core.PlanCacheKeyTestAfterAdjust{}, after)
81+
tk.MustQueryWithContext(ctx, exec2)
82+
}
83+
84+
func planFingerprint(t *testing.T, p base.Plan) string {
85+
v, err := json.Marshal(p)
86+
require.NoError(t, err)
87+
return string(v)
88+
}

0 commit comments

Comments
 (0)