Skip to content

Commit

Permalink
store/tikv: extract methods for LockCtx (#24736)
Browse files Browse the repository at this point in the history
  • Loading branch information
disksing committed May 24, 2021
1 parent 5acff63 commit 559a006
Show file tree
Hide file tree
Showing 4 changed files with 36 additions and 18 deletions.
9 changes: 3 additions & 6 deletions executor/batch_point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
driver "github.com/pingcap/tidb/store/driver/txn"
"github.com/pingcap/tidb/store/tikv"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -417,8 +416,7 @@ func LockKeys(ctx context.Context, seCtx sessionctx.Context, lockWaitTime int64,
txnCtx := seCtx.GetSessionVars().TxnCtx
lctx := newLockCtx(seCtx.GetSessionVars(), lockWaitTime)
if txnCtx.IsPessimistic {
lctx.ReturnValues = true
lctx.Values = make(map[string]tikvstore.ReturnedValue, len(keys))
lctx.InitReturnValues(len(keys))
}
err := doLockKeys(ctx, seCtx, lctx, keys...)
if err != nil {
Expand All @@ -428,9 +426,8 @@ func LockKeys(ctx context.Context, seCtx sessionctx.Context, lockWaitTime int64,
// When doLockKeys returns without error, no other goroutines access the map,
// it's safe to read it without mutex.
for _, key := range keys {
rv := lctx.Values[string(key)]
if !rv.AlreadyLocked {
txnCtx.SetPessimisticLockCache(key, rv.Value)
if v, ok := lctx.GetValueNotLocked(key); ok {
txnCtx.SetPessimisticLockCache(key, v)
}
}
}
Expand Down
14 changes: 4 additions & 10 deletions executor/point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@ import (
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/store/tikv"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -321,19 +320,14 @@ func (e *PointGetExecutor) lockKeyIfNeeded(ctx context.Context, key []byte) erro
if e.lock {
seVars := e.ctx.GetSessionVars()
lockCtx := newLockCtx(seVars, e.lockWaitTime)
lockCtx.ReturnValues = true
lockCtx.Values = map[string]tikvstore.ReturnedValue{}
lockCtx.InitReturnValues(1)
err := doLockKeys(ctx, e.ctx, lockCtx, key)
if err != nil {
return err
}
lockCtx.ValuesLock.Lock()
defer lockCtx.ValuesLock.Unlock()
for key, val := range lockCtx.Values {
if !val.AlreadyLocked {
seVars.TxnCtx.SetPessimisticLockCache(kv.Key(key), val.Value)
}
}
lockCtx.IterateValuesNotLocked(func(k, v []byte) {
seVars.TxnCtx.SetPessimisticLockCache(kv.Key(k), v)
})
if len(e.handleVal) > 0 {
seVars.TxnCtx.SetPessimisticLockCache(e.idxKey, e.handleVal)
}
Expand Down
28 changes: 28 additions & 0 deletions store/tikv/kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,3 +31,31 @@ type LockCtx struct {
ResourceGroupTag []byte
OnDeadlock func(*tikverr.ErrDeadlock)
}

// InitReturnValues creates the map to store returned value.
func (ctx *LockCtx) InitReturnValues(valueLen int) {
ctx.ReturnValues = true
ctx.Values = make(map[string]ReturnedValue, valueLen)
}

// GetValueNotLocked returns a value if the key is not already locked.
// (nil, false) means already locked.
func (ctx *LockCtx) GetValueNotLocked(key []byte) ([]byte, bool) {
rv := ctx.Values[string(key)]
if !rv.AlreadyLocked {
return rv.Value, true
}
return nil, false
}

// IterateValuesNotLocked applies f to all key-values that are not already
// locked.
func (ctx *LockCtx) IterateValuesNotLocked(f func([]byte, []byte)) {
ctx.ValuesLock.Lock()
defer ctx.ValuesLock.Unlock()
for key, val := range ctx.Values {
if !val.AlreadyLocked {
f([]byte(key), val.Value)
}
}
}
3 changes: 1 addition & 2 deletions store/tikv/tests/2pc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -712,8 +712,7 @@ func (s *testCommitterSuite) TestPessimisticLockReturnValues(c *C) {
txn = s.begin(c)
txn.SetPessimistic(true)
lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()}
lockCtx.ReturnValues = true
lockCtx.Values = map[string]kv.ReturnedValue{}
lockCtx.InitReturnValues(2)
c.Assert(txn.LockKeys(context.Background(), lockCtx, key, key2), IsNil)
c.Assert(lockCtx.Values, HasLen, 2)
c.Assert(lockCtx.Values[string(key)].Value, BytesEquals, key)
Expand Down

0 comments on commit 559a006

Please sign in to comment.