Skip to content

Commit 7f3bff8

Browse files
jackyspcoocood
authored andcommitted
session: do not keep history when the transaction retry is disabled (pingcap#11192)
1 parent d36d663 commit 7f3bff8

File tree

5 files changed

+137
-24
lines changed

5 files changed

+137
-24
lines changed

ddl/db_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -3098,9 +3098,9 @@ func (s *testDBSuite2) TestLockTables(c *C) {
30983098
// Test lock table by other session in transaction and commit with retry.
30993099
tk.MustExec("unlock tables")
31003100
tk2.MustExec("unlock tables")
3101+
tk.MustExec("set @@session.tidb_disable_txn_auto_retry=0")
31013102
tk.MustExec("begin")
31023103
tk.MustExec("insert into t1 set a=1")
3103-
tk.MustExec("set @@session.tidb_disable_txn_auto_retry=0")
31043104
tk2.MustExec("lock tables t1 write")
31053105
_, err = tk.Exec("commit")
31063106
c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue, Commentf("err: %v\n", err))

session/session.go

+48-13
Original file line numberDiff line numberDiff line change
@@ -134,10 +134,8 @@ var (
134134
)
135135

136136
type stmtRecord struct {
137-
stmtID uint32
138137
st sqlexec.Statement
139138
stmtCtx *stmtctx.StatementContext
140-
params []interface{}
141139
}
142140

143141
// StmtHistory holds all histories of statements in a txn.
@@ -146,12 +144,10 @@ type StmtHistory struct {
146144
}
147145

148146
// Add appends a stmt to history list.
149-
func (h *StmtHistory) Add(stmtID uint32, st sqlexec.Statement, stmtCtx *stmtctx.StatementContext, params ...interface{}) {
147+
func (h *StmtHistory) Add(st sqlexec.Statement, stmtCtx *stmtctx.StatementContext) {
150148
s := &stmtRecord{
151-
stmtID: stmtID,
152149
st: st,
153150
stmtCtx: stmtCtx,
154-
params: append(([]interface{})(nil), params...),
155151
}
156152
h.history = append(h.history, s)
157153
}
@@ -451,14 +447,8 @@ func (s *session) doCommitWithRetry(ctx context.Context) error {
451447
err := s.doCommit(ctx)
452448
if err != nil {
453449
commitRetryLimit := s.sessionVars.RetryLimit
454-
if s.sessionVars.DisableTxnAutoRetry && !s.sessionVars.InRestrictedSQL {
455-
// Do not retry non-autocommit transactions.
456-
// For autocommit single statement transactions, the history count is always 1.
457-
// For explicit transactions, the statement count is more than 1.
458-
history := GetHistory(s)
459-
if history.Count() > 1 {
460-
commitRetryLimit = 0
461-
}
450+
if !s.sessionVars.TxnCtx.CouldRetry {
451+
commitRetryLimit = 0
462452
}
463453
// Don't retry in BatchInsert mode. As a counter-example, insert into t1 select * from t2,
464454
// BatchInsert already commit the first batch 1000 rows, then it commit 1000-2000 and retry the statement,
@@ -517,6 +507,13 @@ func (s *session) CommitTxn(ctx context.Context) error {
517507
if commitDetail != nil {
518508
s.sessionVars.StmtCtx.MergeExecDetails(nil, commitDetail)
519509
}
510+
511+
failpoint.Inject("keepHistory", func(val failpoint.Value) {
512+
if val.(bool) {
513+
failpoint.Return(err)
514+
}
515+
})
516+
520517
s.sessionVars.TxnCtx.Cleanup()
521518
s.recordTransactionCounter(err)
522519
return err
@@ -1247,10 +1244,48 @@ func (s *session) Txn(active bool) (kv.Transaction, error) {
12471244
if !s.sessionVars.IsAutocommit() {
12481245
s.sessionVars.SetStatusFlag(mysql.ServerStatusInTrans, true)
12491246
}
1247+
s.sessionVars.TxnCtx.CouldRetry = s.isTxnRetryable()
12501248
}
12511249
return &s.txn, nil
12521250
}
12531251

1252+
// isTxnRetryable (if returns true) means the transaction could retry.
1253+
// If the transaction is in pessimistic mode, do not retry.
1254+
// If the session is already in transaction, enable retry or internal SQL could retry.
1255+
// If not, the transaction could always retry, because it should be auto committed transaction.
1256+
// Anyway the retry limit is 0, the transaction could not retry.
1257+
func (s *session) isTxnRetryable() bool {
1258+
sessVars := s.sessionVars
1259+
1260+
// The pessimistic transaction no need to retry.
1261+
if sessVars.TxnCtx.IsPessimistic {
1262+
return false
1263+
}
1264+
1265+
// If retry limit is 0, the transaction could not retry.
1266+
if sessVars.RetryLimit == 0 {
1267+
return false
1268+
}
1269+
1270+
// If the session is not InTxn, it is an auto-committed transaction.
1271+
// The auto-committed transaction could always retry.
1272+
if !sessVars.InTxn() {
1273+
return true
1274+
}
1275+
1276+
// The internal transaction could always retry.
1277+
if sessVars.InRestrictedSQL {
1278+
return true
1279+
}
1280+
1281+
// If the retry is enabled, the transaction could retry.
1282+
if !sessVars.DisableTxnAutoRetry {
1283+
return true
1284+
}
1285+
1286+
return false
1287+
}
1288+
12541289
func (s *session) NewTxn(ctx context.Context) error {
12551290
if s.txn.Valid() {
12561291
txnID := s.txn.StartTS()

session/session_test.go

+76-2
Original file line numberDiff line numberDiff line change
@@ -545,7 +545,7 @@ func (s *testSessionSuite) TestRetryCleanTxn(c *C) {
545545
c.Assert(err, IsNil)
546546
stmt, _ := session.Compile(context.TODO(), tk.Se, stmtNode)
547547
executor.ResetContextOfStmt(tk.Se, stmtNode)
548-
history.Add(0, stmt, tk.Se.GetSessionVars().StmtCtx)
548+
history.Add(stmt, tk.Se.GetSessionVars().StmtCtx)
549549
_, err = tk.Exec("commit")
550550
c.Assert(err, NotNil)
551551
txn, err := tk.Se.Txn(false)
@@ -559,6 +559,7 @@ func (s *testSessionSuite) TestReadOnlyNotInHistory(c *C) {
559559
tk.MustExec("create table history (a int)")
560560
tk.MustExec("insert history values (1), (2), (3)")
561561
tk.MustExec("set @@autocommit = 0")
562+
tk.MustExec("set tidb_disable_txn_auto_retry = 0")
562563
tk.MustQuery("select * from history")
563564
history := session.GetHistory(tk.Se)
564565
c.Assert(history.Count(), Equals, 0)
@@ -572,6 +573,76 @@ func (s *testSessionSuite) TestReadOnlyNotInHistory(c *C) {
572573
c.Assert(history.Count(), Equals, 0)
573574
}
574575

576+
func (s *testSessionSuite) TestNoHistoryWhenDisableRetry(c *C) {
577+
tk := testkit.NewTestKitWithInit(c, s.store)
578+
tk.MustExec("create table history (a int)")
579+
tk.MustExec("set @@autocommit = 0")
580+
581+
// retry_limit = 0 will not add history.
582+
tk.MustExec("set @@tidb_retry_limit = 0")
583+
tk.MustExec("insert history values (1)")
584+
c.Assert(session.GetHistory(tk.Se).Count(), Equals, 0)
585+
586+
// Disable auto_retry will add history for auto committed only
587+
tk.MustExec("set @@autocommit = 1")
588+
tk.MustExec("set @@tidb_retry_limit = 10")
589+
tk.MustExec("set @@tidb_disable_txn_auto_retry = 1")
590+
c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/keepHistory", `1*return(true)->return(false)`), IsNil)
591+
tk.MustExec("insert history values (1)")
592+
c.Assert(session.GetHistory(tk.Se).Count(), Equals, 1)
593+
c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/keepHistory"), IsNil)
594+
tk.MustExec("begin")
595+
tk.MustExec("insert history values (1)")
596+
c.Assert(session.GetHistory(tk.Se).Count(), Equals, 0)
597+
tk.MustExec("commit")
598+
599+
// Enable auto_retry will add history for both.
600+
tk.MustExec("set @@tidb_disable_txn_auto_retry = 0")
601+
c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/keepHistory", `1*return(true)->return(false)`), IsNil)
602+
tk.MustExec("insert history values (1)")
603+
c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/keepHistory"), IsNil)
604+
c.Assert(session.GetHistory(tk.Se).Count(), Equals, 1)
605+
tk.MustExec("begin")
606+
tk.MustExec("insert history values (1)")
607+
c.Assert(session.GetHistory(tk.Se).Count(), Equals, 2)
608+
tk.MustExec("commit")
609+
}
610+
611+
func (s *testSessionSuite) TestNoRetryForCurrentTxn(c *C) {
612+
tk := testkit.NewTestKitWithInit(c, s.store)
613+
tk1 := testkit.NewTestKitWithInit(c, s.store)
614+
tk.MustExec("create table history (a int)")
615+
tk.MustExec("insert history values (1)")
616+
617+
// Firstly, disable retry.
618+
tk.MustExec("set tidb_disable_txn_auto_retry = 1")
619+
tk.MustExec("begin")
620+
tk.MustExec("update history set a = 2")
621+
// Enable retry now.
622+
tk.MustExec("set tidb_disable_txn_auto_retry = 0")
623+
624+
tk1.MustExec("update history set a = 3")
625+
c.Assert(tk.ExecToErr("commit"), NotNil)
626+
}
627+
628+
func (s *testSessionSuite) TestRetryForCurrentTxn(c *C) {
629+
tk := testkit.NewTestKitWithInit(c, s.store)
630+
tk1 := testkit.NewTestKitWithInit(c, s.store)
631+
tk.MustExec("create table history (a int)")
632+
tk.MustExec("insert history values (1)")
633+
634+
// Firstly, enable retry.
635+
tk.MustExec("set tidb_disable_txn_auto_retry = 0")
636+
tk.MustExec("begin")
637+
tk.MustExec("update history set a = 2")
638+
// Disable retry now.
639+
tk.MustExec("set tidb_disable_txn_auto_retry = 1")
640+
641+
tk1.MustExec("update history set a = 3")
642+
tk.MustExec("commit")
643+
tk.MustQuery("select * from history").Check(testkit.Rows("2"))
644+
}
645+
575646
// TestTruncateAlloc tests that the auto_increment ID does not reuse the old table's allocator.
576647
func (s *testSessionSuite) TestTruncateAlloc(c *C) {
577648
tk := testkit.NewTestKitWithInit(c, s.store)
@@ -990,6 +1061,7 @@ func (s *testSessionSuite) TestBinaryReadOnly(c *C) {
9901061
id2, _, _, err := tk.Se.PrepareStmt("insert into t values (?)")
9911062
c.Assert(err, IsNil)
9921063
tk.MustExec("set autocommit = 0")
1064+
tk.MustExec("set tidb_disable_txn_auto_retry = 0")
9931065
_, err = tk.Se.ExecutePreparedStmt(context.Background(), id, []types.Datum{types.NewDatum(1)})
9941066
c.Assert(err, IsNil)
9951067
c.Assert(session.GetHistory(tk.Se).Count(), Equals, 0)
@@ -2177,6 +2249,7 @@ func (s *testSessionSuite) TestStatementCountLimit(c *C) {
21772249
defer func() {
21782250
config.GetGlobalConfig().Performance.StmtCountLimit = saved
21792251
}()
2252+
tk.MustExec("set tidb_disable_txn_auto_retry = 0")
21802253
tk.MustExec("begin")
21812254
tk.MustExec("insert into stmt_count_limit values (1)")
21822255
tk.MustExec("insert into stmt_count_limit values (2)")
@@ -2195,6 +2268,7 @@ func (s *testSessionSuite) TestStatementCountLimit(c *C) {
21952268
func (s *testSessionSuite) TestBatchCommit(c *C) {
21962269
tk := testkit.NewTestKitWithInit(c, s.store)
21972270
tk.MustExec("set tidb_batch_commit = 1")
2271+
tk.MustExec("set tidb_disable_txn_auto_retry = 0")
21982272
tk.MustExec("create table t (id int)")
21992273
saved := config.GetGlobalConfig().Performance
22002274
config.GetGlobalConfig().Performance.StmtCountLimit = 3
@@ -2440,7 +2514,7 @@ func (s *testSchemaSuite) TestDisableTxnAutoRetry(c *C) {
24402514
// session 1 starts a transaction early.
24412515
// execute a select statement to clear retry history.
24422516
tk1.MustExec("select 1")
2443-
tk1.Se.NewTxn(context.Background())
2517+
tk1.Se.PrepareTxnCtx(context.Background())
24442518
// session 2 update the value.
24452519
tk2.MustExec("update no_retry set id = 4")
24462520
// AutoCommit update will retry, so it would not fail.

session/tidb.go

+10-7
Original file line numberDiff line numberDiff line change
@@ -176,13 +176,14 @@ func finishStmt(ctx context.Context, sctx sessionctx.Context, se *session, sessV
176176
return se.CommitTxn(ctx)
177177
}
178178

179-
return checkStmtLimit(ctx, sctx, se, sessVars)
179+
return checkStmtLimit(ctx, sctx, se)
180180
}
181181

182-
func checkStmtLimit(ctx context.Context, sctx sessionctx.Context, se *session, sessVars *variable.SessionVars) error {
182+
func checkStmtLimit(ctx context.Context, sctx sessionctx.Context, se *session) error {
183183
// If the user insert, insert, insert ... but never commit, TiDB would OOM.
184184
// So we limit the statement count in a transaction here.
185185
var err error
186+
sessVars := se.GetSessionVars()
186187
history := GetHistory(sctx)
187188
if history.Count() > int(config.GetGlobalConfig().Performance.StmtCountLimit) {
188189
if !sessVars.BatchCommit {
@@ -195,7 +196,7 @@ func checkStmtLimit(ctx context.Context, sctx sessionctx.Context, se *session, s
195196
// The last history could not be "commit"/"rollback" statement.
196197
// It means it is impossible to start a new transaction at the end of the transaction.
197198
// Because after the server executed "commit"/"rollback" statement, the session is out of the transaction.
198-
se.sessionVars.SetStatusFlag(mysql.ServerStatusInTrans, true)
199+
sessVars.SetStatusFlag(mysql.ServerStatusInTrans, true)
199200
}
200201
return err
201202
}
@@ -222,12 +223,14 @@ func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement)
222223
}
223224
rs, err = s.Exec(ctx)
224225
sessVars := se.GetSessionVars()
225-
// All the history should be added here.
226226
sessVars.TxnCtx.StatementCount++
227227
if !s.IsReadOnly(sessVars) {
228-
if err == nil && !sessVars.TxnCtx.IsPessimistic {
229-
GetHistory(sctx).Add(0, s, se.sessionVars.StmtCtx)
228+
// All the history should be added here.
229+
if err == nil && sessVars.TxnCtx.CouldRetry {
230+
GetHistory(sctx).Add(s, sessVars.StmtCtx)
230231
}
232+
233+
// Handle the stmt commit/rollback.
231234
if txn, err1 := sctx.Txn(false); err1 == nil {
232235
if txn.Valid() {
233236
if err != nil {
@@ -240,8 +243,8 @@ func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement)
240243
logutil.BgLogger().Error("get txn error", zap.Error(err1))
241244
}
242245
}
243-
244246
err = finishStmt(ctx, sctx, se, sessVars, err)
247+
245248
if se.txn.pending() {
246249
// After run statement finish, txn state is still pending means the
247250
// statement never need a Txn(), such as:

sessionctx/variable/session.go

+2-1
Original file line numberDiff line numberDiff line change
@@ -104,6 +104,7 @@ type TransactionContext struct {
104104
DirtyDB interface{}
105105
Binlog interface{}
106106
InfoSchema interface{}
107+
CouldRetry bool
107108
History interface{}
108109
SchemaVersion int64
109110
StartTS uint64
@@ -135,7 +136,7 @@ func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, co
135136

136137
// Cleanup clears up transaction info that no longer use.
137138
func (tc *TransactionContext) Cleanup() {
138-
//tc.InfoSchema = nil; we cannot do it now, because some operation like handleFieldList depend on this.
139+
// tc.InfoSchema = nil; we cannot do it now, because some operation like handleFieldList depend on this.
139140
tc.DirtyDB = nil
140141
tc.Binlog = nil
141142
tc.History = nil

0 commit comments

Comments
 (0)