Skip to content

Commit

Permalink
kv: fix KVVars not set on Transaction (pingcap#18351)
Browse files Browse the repository at this point in the history
  • Loading branch information
coocood authored Jul 3, 2020
1 parent f902265 commit 180c021
Show file tree
Hide file tree
Showing 5 changed files with 29 additions and 45 deletions.
4 changes: 4 additions & 0 deletions kv/interface_mock_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,10 @@ func (t *mockTxn) SetVars(vars *Variables) {

}

func (t *mockTxn) GetVars() *Variables {
return nil
}

// newMockTxn new a mockTxn.
func newMockTxn() Transaction {
return &mockTxn{
Expand Down
2 changes: 2 additions & 0 deletions kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -202,6 +202,8 @@ type Transaction interface {
GetSnapshot() Snapshot
// SetVars sets variables to the transaction.
SetVars(vars *Variables)
// GetVars gets variables from the transaction.
GetVars() *Variables
// BatchGet gets kv from the memory buffer of statement and transaction, and the kv storage.
// Do not use len(value) == 0 or value == nil to represent non-exist.
// If a key doesn't exist, there shouldn't be any corresponding entry in the result map.
Expand Down
1 change: 1 addition & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1450,6 +1450,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) {
s.sessionVars.SetStatusFlag(mysql.ServerStatusInTrans, true)
}
s.sessionVars.TxnCtx.CouldRetry = s.isTxnRetryable()
s.txn.SetVars(s.sessionVars.KVVars)
if s.sessionVars.GetReplicaRead().IsFollowerRead() {
s.txn.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower)
}
Expand Down
63 changes: 18 additions & 45 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2627,52 +2627,25 @@ func (s *testSessionSuite2) TestDBUserNameLength(c *C) {
}

func (s *testSessionSerialSuite) TestKVVars(c *C) {
c.Skip("there is no backoff here in the large txn, so this test is stale")
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("create table kvvars (a int, b int)")
tk.MustExec("insert kvvars values (1, 1)")
tk2 := testkit.NewTestKitWithInit(c, s.store)
tk2.MustExec("set @@tidb_backoff_lock_fast = 1")
tk2.MustExec("set @@tidb_backoff_weight = 100")
backoffVal := new(int64)
backOffWeightVal := new(int32)
tk2.Se.GetSessionVars().KVVars.Hook = func(name string, vars *kv.Variables) {
atomic.StoreInt64(backoffVal, int64(vars.BackoffLockFast))
atomic.StoreInt32(backOffWeightVal, int32(vars.BackOffWeight))
}
wg := new(sync.WaitGroup)
wg.Add(2)
go func() {
for {
tk2.MustQuery("select * from kvvars")
if atomic.LoadInt64(backoffVal) != 0 {
break
}
}
wg.Done()
}()

c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockSleepBetween2PC", "return"), IsNil)
go func() {
for {
tk.MustExec("update kvvars set b = b + 1 where a = 1")
if atomic.LoadInt64(backoffVal) != 0 {
break
}
}
wg.Done()
}()
wg.Wait()
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockSleepBetween2PC"), IsNil)

for {
tk2.MustQuery("select * from kvvars")
if atomic.LoadInt32(backOffWeightVal) != 0 {
break
}
}
c.Assert(atomic.LoadInt64(backoffVal), Equals, int64(1))
c.Assert(atomic.LoadInt32(backOffWeightVal), Equals, int32(100))
tk.MustExec("set @@tidb_backoff_lock_fast = 1")
tk.MustExec("set @@tidb_backoff_weight = 100")
tk.MustExec("create table if not exists kvvars (a int)")
tk.MustExec("begin")
txn, err := tk.Se.Txn(false)
c.Assert(err, IsNil)
vars := txn.GetVars()
c.Assert(vars.BackoffLockFast, Equals, 1)
c.Assert(vars.BackOffWeight, Equals, 100)
tk.MustExec("rollback")
tk.MustExec("set @@tidb_backoff_weight = 50")
tk.MustExec("set @@autocommit = 0")
tk.MustExec("select * from kvvars")
c.Assert(tk.Se.GetSessionVars().InTxn(), IsTrue)
txn, err = tk.Se.Txn(false)
c.Assert(err, IsNil)
vars = txn.GetVars()
c.Assert(vars.BackOffWeight, Equals, 50)
}

func (s *testSessionSuite2) TestCommitRetryCount(c *C) {
Expand Down
4 changes: 4 additions & 0 deletions store/tikv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,10 @@ func (txn *tikvTxn) SetVars(vars *kv.Variables) {
txn.snapshot.vars = vars
}

func (txn *tikvTxn) GetVars() *kv.Variables {
return txn.vars
}

// tikvTxnStagingBuffer is the staging buffer returned to tikvTxn user.
// Because tikvTxn needs to maintain dirty state when Flush staging data into txn.
type tikvTxnStagingBuffer struct {
Expand Down

0 comments on commit 180c021

Please sign in to comment.