Skip to content

Commit 529ce88

Browse files
authored
*: add TxnManager to manage txn in session (#30574)
* *: add TxnManager to manage txn in session * modify * add tests * move failpoint content to a single file
1 parent 3bd732f commit 529ce88

File tree

17 files changed

+1090
-40
lines changed

17 files changed

+1090
-40
lines changed

ddl/db_change_test.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -411,7 +411,9 @@ func (t *testExecInfo) compileSQL(idx int) (err error) {
411411
compiler := executor.Compiler{Ctx: c.session}
412412
se := c.session
413413
ctx := context.TODO()
414-
se.PrepareTxnCtx(ctx)
414+
if err = se.PrepareTxnCtx(ctx); err != nil {
415+
return err
416+
}
415417
sctx := se.(sessionctx.Context)
416418
if err = executor.ResetContextOfStmt(sctx, c.rawStmt); err != nil {
417419
return errors.Trace(err)

executor/adapter.go

Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,7 @@ import (
4646
"github.com/pingcap/tidb/plugin"
4747
"github.com/pingcap/tidb/sessionctx"
4848
"github.com/pingcap/tidb/sessionctx/variable"
49+
"github.com/pingcap/tidb/sessiontxn"
4950
"github.com/pingcap/tidb/types"
5051
"github.com/pingcap/tidb/util/chunk"
5152
"github.com/pingcap/tidb/util/execdetails"
@@ -263,6 +264,12 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec
263264
a.PsStmt.Executor = newExecutor
264265
}
265266
pointExecutor := a.PsStmt.Executor.(*PointGetExecutor)
267+
268+
failpoint.Inject("assertTxnManagerInShortPointGetPlan", func() {
269+
sessiontxn.RecordAssert(a.Ctx, "assertTxnManagerInShortPointGetPlan", true)
270+
sessiontxn.AssertTxnManagerInfoSchema(a.Ctx, is)
271+
})
272+
266273
if err = pointExecutor.Open(ctx); err != nil {
267274
terror.Call(pointExecutor.Close)
268275
return nil, err
@@ -298,6 +305,16 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) {
298305
if err := plannercore.Preprocess(a.Ctx, a.StmtNode, plannercore.InTxnRetry, plannercore.WithPreprocessorReturn(ret)); err != nil {
299306
return 0, err
300307
}
308+
309+
failpoint.Inject("assertTxnManagerInRebuildPlan", func() {
310+
if is, ok := a.Ctx.Value(sessiontxn.AssertTxnInfoSchemaAfterRetryKey).(infoschema.InfoSchema); ok {
311+
a.Ctx.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is)
312+
a.Ctx.SetValue(sessiontxn.AssertTxnInfoSchemaAfterRetryKey, nil)
313+
}
314+
sessiontxn.RecordAssert(a.Ctx, "assertTxnManagerInRebuildPlan", true)
315+
sessiontxn.AssertTxnManagerInfoSchema(a.Ctx, ret.InfoSchema)
316+
})
317+
301318
a.InfoSchema = ret.InfoSchema
302319
a.SnapshotTS = ret.LastSnapshotTS
303320
a.IsStaleness = ret.IsStaleness
@@ -755,6 +772,10 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, err error) (E
755772
a.Ctx.GetSessionVars().StmtCtx.ResetForRetry()
756773
a.Ctx.GetSessionVars().RetryInfo.ResetOffset()
757774

775+
failpoint.Inject("assertTxnManagerAfterPessimisticLockErrorRetry", func() {
776+
sessiontxn.RecordAssert(a.Ctx, "assertTxnManagerAfterPessimisticLockErrorRetry", true)
777+
})
778+
758779
if err = e.Open(ctx); err != nil {
759780
return nil, err
760781
}
@@ -809,6 +830,11 @@ func (a *ExecStmt) buildExecutor() (Executor, error) {
809830
return nil, errors.Trace(b.err)
810831
}
811832

833+
failpoint.Inject("assertTxnManagerAfterBuildExecutor", func() {
834+
sessiontxn.RecordAssert(a.Ctx, "assertTxnManagerAfterBuildExecutor", true)
835+
sessiontxn.AssertTxnManagerInfoSchema(b.ctx, b.is)
836+
})
837+
812838
// ExecuteExec is not a real Executor, we only use it to build another Executor from a prepared statement.
813839
if executorExec, ok := e.(*ExecuteExec); ok {
814840
err := executorExec.Build(b)

executor/compiler.go

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import (
2727
"github.com/pingcap/tidb/planner"
2828
plannercore "github.com/pingcap/tidb/planner/core"
2929
"github.com/pingcap/tidb/sessionctx"
30+
"github.com/pingcap/tidb/sessiontxn"
3031
)
3132

3233
var (
@@ -57,11 +58,21 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm
5758

5859
ret := &plannercore.PreprocessorReturn{}
5960
pe := &plannercore.PreprocessExecuteISUpdate{ExecuteInfoSchemaUpdate: planner.GetExecuteForUpdateReadIS, Node: stmtNode}
60-
err := plannercore.Preprocess(c.Ctx, stmtNode, plannercore.WithPreprocessorReturn(ret), plannercore.WithExecuteInfoSchemaUpdate(pe))
61+
err := plannercore.Preprocess(c.Ctx,
62+
stmtNode,
63+
plannercore.WithPreprocessorReturn(ret),
64+
plannercore.WithExecuteInfoSchemaUpdate(pe),
65+
plannercore.InitTxnContextProvider,
66+
)
6167
if err != nil {
6268
return nil, err
6369
}
6470

71+
failpoint.Inject("assertTxnManagerInCompile", func() {
72+
sessiontxn.RecordAssert(c.Ctx, "assertTxnManagerInCompile", true)
73+
sessiontxn.AssertTxnManagerInfoSchema(c.Ctx, ret.InfoSchema)
74+
})
75+
6576
finalPlan, names, err := planner.Optimize(ctx, c.Ctx, stmtNode, ret.InfoSchema)
6677
if err != nil {
6778
return nil, err

executor/executor.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,6 @@ import (
4545
"github.com/pingcap/tidb/parser/model"
4646
"github.com/pingcap/tidb/parser/mysql"
4747
"github.com/pingcap/tidb/parser/terror"
48-
"github.com/pingcap/tidb/planner"
4948
plannercore "github.com/pingcap/tidb/planner/core"
5049
"github.com/pingcap/tidb/privilege"
5150
"github.com/pingcap/tidb/sessionctx"
@@ -1718,7 +1717,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
17181717
sc.MemTracker.SetActionOnExceed(action)
17191718
}
17201719
if execStmt, ok := s.(*ast.ExecuteStmt); ok {
1721-
prepareStmt, err := planner.GetPreparedStmt(execStmt, vars)
1720+
prepareStmt, err := plannercore.GetPreparedStmt(execStmt, vars)
17221721
if err != nil {
17231722
return err
17241723
}

executor/prepared.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import (
2121
"time"
2222

2323
"github.com/pingcap/errors"
24+
"github.com/pingcap/failpoint"
2425
"github.com/pingcap/log"
2526
"github.com/pingcap/tidb/expression"
2627
"github.com/pingcap/tidb/infoschema"
@@ -31,6 +32,7 @@ import (
3132
plannercore "github.com/pingcap/tidb/planner/core"
3233
"github.com/pingcap/tidb/sessionctx"
3334
"github.com/pingcap/tidb/sessionctx/variable"
35+
"github.com/pingcap/tidb/sessiontxn"
3436
"github.com/pingcap/tidb/types"
3537
driver "github.com/pingcap/tidb/types/parser_driver"
3638
"github.com/pingcap/tidb/util"
@@ -340,6 +342,11 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context,
340342
return nil, false, false, err
341343
}
342344

345+
failpoint.Inject("assertTxnManagerInCompile", func() {
346+
sessiontxn.RecordAssert(sctx, "assertTxnManagerInCompile", true)
347+
sessiontxn.AssertTxnManagerInfoSchema(sctx, is)
348+
})
349+
343350
stmt := &ExecStmt{
344351
GoCtx: ctx,
345352
InfoSchema: is,

executor/seqtest/prepared_test.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -163,7 +163,8 @@ func TestPrepared(t *testing.T) {
163163
require.Equal(t, query, stmt.OriginText())
164164

165165
// Check that rebuild plan works.
166-
tk.Session().PrepareTxnCtx(ctx)
166+
err = tk.Session().PrepareTxnCtx(ctx)
167+
require.NoError(t, err)
167168
_, err = stmt.RebuildPlan(ctx)
168169
require.NoError(t, err)
169170
rs, err = stmt.Exec(ctx)

planner/core/cache.go

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ import (
1919
"sync/atomic"
2020
"time"
2121

22+
"github.com/pingcap/errors"
2223
"github.com/pingcap/tidb/kv"
2324
"github.com/pingcap/tidb/parser"
2425
"github.com/pingcap/tidb/parser/ast"
@@ -213,3 +214,22 @@ type CachedPrepareStmt struct {
213214
ForUpdateRead bool
214215
SnapshotTSEvaluator func(sessionctx.Context) (uint64, error)
215216
}
217+
218+
// GetPreparedStmt extract the prepared statement from the execute statement.
219+
func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*CachedPrepareStmt, error) {
220+
var ok bool
221+
execID := stmt.ExecID
222+
if stmt.Name != "" {
223+
if execID, ok = vars.PreparedStmtNameToID[stmt.Name]; !ok {
224+
return nil, ErrStmtNotFound
225+
}
226+
}
227+
if preparedPointer, ok := vars.PreparedStmts[execID]; ok {
228+
preparedObj, ok := preparedPointer.(*CachedPrepareStmt)
229+
if !ok {
230+
return nil, errors.Errorf("invalid CachedPrepareStmt type")
231+
}
232+
return preparedObj, nil
233+
}
234+
return nil, ErrStmtNotFound
235+
}

planner/core/preprocess.go

Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@ import (
3737
"github.com/pingcap/tidb/privilege"
3838
"github.com/pingcap/tidb/sessionctx"
3939
"github.com/pingcap/tidb/sessionctx/variable"
40+
"github.com/pingcap/tidb/sessiontxn"
4041
"github.com/pingcap/tidb/table"
4142
"github.com/pingcap/tidb/table/temptable"
4243
"github.com/pingcap/tidb/types"
@@ -59,6 +60,11 @@ func InTxnRetry(p *preprocessor) {
5960
p.flag |= inTxnRetry
6061
}
6162

63+
// InitTxnContextProvider is a PreprocessOpt that indicates preprocess should init transaction's context
64+
func InitTxnContextProvider(p *preprocessor) {
65+
p.flag |= initTxnContextProvider
66+
}
67+
6268
// WithPreprocessorReturn returns a PreprocessOpt to initialize the PreprocessorReturn.
6369
func WithPreprocessorReturn(ret *PreprocessorReturn) PreprocessOpt {
6470
return func(p *preprocessor) {
@@ -117,6 +123,9 @@ func Preprocess(ctx sessionctx.Context, node ast.Node, preprocessOpt ...Preproce
117123
node.Accept(&v)
118124
// InfoSchema must be non-nil after preprocessing
119125
v.ensureInfoSchema()
126+
127+
v.initTxnContextProviderIfNecessary(node)
128+
120129
return errors.Trace(v.err)
121130
}
122131

@@ -136,6 +145,8 @@ const (
136145
// inSequenceFunction is set when visiting a sequence function.
137146
// This flag indicates the tableName in these function should be checked as sequence object.
138147
inSequenceFunction
148+
// initTxnContextProvider is set when we should init txn context in preprocess
149+
initTxnContextProvider
139150
)
140151

141152
// Make linter happy.
@@ -193,6 +204,9 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
193204
// handle the insert table name imminently
194205
// insert into t with t ..., the insert can not see t here. We should hand it before the CTE statement
195206
p.handleTableName(node.Table.TableRefs.Left.(*ast.TableSource).Source.(*ast.TableName))
207+
case *ast.ExecuteStmt:
208+
p.stmtTp = TypeExecute
209+
p.resolveExecuteStmt(node)
196210
case *ast.CreateTableStmt:
197211
p.stmtTp = TypeCreate
198212
p.flag |= inCreateOrDropTable
@@ -361,6 +375,8 @@ const (
361375
TypeRepair
362376
// TypeShow for ShowStmt
363377
TypeShow
378+
// TypeExecute for ExecuteStmt
379+
TypeExecute
364380
)
365381

366382
func bindableStmtType(node ast.StmtNode) byte {
@@ -1489,6 +1505,32 @@ func (p *preprocessor) resolveShowStmt(node *ast.ShowStmt) {
14891505
}
14901506
}
14911507

1508+
func (p *preprocessor) resolveExecuteStmt(node *ast.ExecuteStmt) {
1509+
prepared, err := GetPreparedStmt(node, p.ctx.GetSessionVars())
1510+
if err != nil {
1511+
p.err = err
1512+
return
1513+
}
1514+
1515+
if prepared.SnapshotTSEvaluator != nil {
1516+
snapshotTS, err := prepared.SnapshotTSEvaluator(p.ctx)
1517+
if err != nil {
1518+
p.err = err
1519+
return
1520+
}
1521+
1522+
is, err := domain.GetDomain(p.ctx).GetSnapshotInfoSchema(snapshotTS)
1523+
if err != nil {
1524+
p.err = err
1525+
return
1526+
}
1527+
1528+
p.LastSnapshotTS = snapshotTS
1529+
p.initedLastSnapshotTS = true
1530+
p.InfoSchema = temptable.AttachLocalTemporaryTableInfoSchema(p.ctx, is)
1531+
}
1532+
}
1533+
14921534
func (p *preprocessor) resolveCreateTableStmt(node *ast.CreateTableStmt) {
14931535
for _, val := range node.Constraints {
14941536
if val.Refer != nil && val.Refer.Table.Schema.String() == "" {
@@ -1689,3 +1731,13 @@ func (p *preprocessor) ensureInfoSchema() infoschema.InfoSchema {
16891731
p.InfoSchema = p.ctx.GetInfoSchema().(infoschema.InfoSchema)
16901732
return p.InfoSchema
16911733
}
1734+
1735+
func (p *preprocessor) initTxnContextProviderIfNecessary(node ast.Node) {
1736+
if p.err != nil || p.flag&initTxnContextProvider == 0 {
1737+
return
1738+
}
1739+
1740+
p.err = sessiontxn.GetTxnManager(p.ctx).SetContextProvider(&sessiontxn.SimpleTxnContextProvider{
1741+
InfoSchema: p.ensureInfoSchema(),
1742+
})
1743+
}

planner/optimize.go

Lines changed: 1 addition & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -48,29 +48,10 @@ import (
4848
"go.uber.org/zap"
4949
)
5050

51-
// GetPreparedStmt extract the prepared statement from the execute statement.
52-
func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*plannercore.CachedPrepareStmt, error) {
53-
var ok bool
54-
execID := stmt.ExecID
55-
if stmt.Name != "" {
56-
if execID, ok = vars.PreparedStmtNameToID[stmt.Name]; !ok {
57-
return nil, plannercore.ErrStmtNotFound
58-
}
59-
}
60-
if preparedPointer, ok := vars.PreparedStmts[execID]; ok {
61-
preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt)
62-
if !ok {
63-
return nil, errors.Errorf("invalid CachedPrepareStmt type")
64-
}
65-
return preparedObj, nil
66-
}
67-
return nil, plannercore.ErrStmtNotFound
68-
}
69-
7051
// IsReadOnly check whether the ast.Node is a read only statement.
7152
func IsReadOnly(node ast.Node, vars *variable.SessionVars) bool {
7253
if execStmt, isExecStmt := node.(*ast.ExecuteStmt); isExecStmt {
73-
prepareStmt, err := GetPreparedStmt(execStmt, vars)
54+
prepareStmt, err := plannercore.GetPreparedStmt(execStmt, vars)
7455
if err != nil {
7556
logutil.BgLogger().Warn("GetPreparedStmt failed", zap.Error(err))
7657
return false

0 commit comments

Comments
 (0)