Skip to content

Commit

Permalink
store,kv: snapshot doesn't cache the non-exists kv entries lead to po…
Browse files Browse the repository at this point in the history
…or 'insert ignore' performance (#12872)
  • Loading branch information
tiancaiamao authored and sre-bot committed Oct 23, 2019
1 parent 34b3c9a commit 58fc7d4
Show file tree
Hide file tree
Showing 4 changed files with 52 additions and 3 deletions.
2 changes: 2 additions & 0 deletions kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,8 @@ type Transaction interface {
// SetVars sets variables to the transaction.
SetVars(vars *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.
BatchGet(ctx context.Context, keys []Key) (map[string][]byte, error)
IsPessimistic() bool
}
Expand Down
7 changes: 7 additions & 0 deletions store/mockstore/mocktikv/rpc.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"time"

"github.com/golang/protobuf/proto"
"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/coprocessor"
Expand Down Expand Up @@ -708,6 +709,12 @@ func (c *RPCClient) checkArgs(ctx context.Context, addr string) (*rpcHandler, er

// SendRequest sends a request to mock cluster.
func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("RPCClient.SendRequest", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}

failpoint.Inject("rpcServerBusy", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(tikvrpc.GenRegionErrorResp(req, &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}}))
Expand Down
24 changes: 21 additions & 3 deletions store/tikv/snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"time"
"unsafe"

"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
pb "github.com/pingcap/kvproto/pkg/kvrpcpb"
Expand Down Expand Up @@ -62,6 +63,10 @@ type tikvSnapshot struct {
// Cache the result of BatchGet.
// The invariance is that calling BatchGet multiple times using the same start ts,
// the result should not change.
// NOTE: This representation here is different from the BatchGet API.
// cached use len(value)=0 to represent a key-value entry doesn't exist (a reliable truth from TiKV).
// In the BatchGet API, it use no key-value entry to represent non-exist.
// It's OK as long as there are no zero-byte values in the protocol.
cached map[string][]byte
}

Expand Down Expand Up @@ -95,7 +100,9 @@ func (s *tikvSnapshot) BatchGet(ctx context.Context, keys []kv.Key) (map[string]
tmp := keys[:0]
for _, key := range keys {
if val, ok := s.cached[string(key)]; ok {
m[string(key)] = val
if len(val) > 0 {
m[string(key)] = val
}
} else {
tmp = append(tmp, key)
}
Expand All @@ -121,6 +128,7 @@ func (s *tikvSnapshot) BatchGet(ctx context.Context, keys []kv.Key) (map[string]
if len(v) == 0 {
return
}

mu.Lock()
m[string(k)] = v
mu.Unlock()
Expand All @@ -138,8 +146,8 @@ func (s *tikvSnapshot) BatchGet(ctx context.Context, keys []kv.Key) (map[string]
if s.cached == nil {
s.cached = make(map[string][]byte, len(m))
}
for key, value := range m {
s.cached[key] = value
for _, key := range keys {
s.cached[string(key)] = m[string(key)]
}

return m, nil
Expand Down Expand Up @@ -253,6 +261,12 @@ func (s *tikvSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, coll

// Get gets the value for key k from snapshot.
func (s *tikvSnapshot) Get(ctx context.Context, k kv.Key) ([]byte, error) {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("tikvSnapshot.get", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}

ctx = context.WithValue(ctx, txnStartKey, s.version.Ver)
val, err := s.get(NewBackoffer(ctx, getMaxBackoff), k)
if err != nil {
Expand All @@ -272,6 +286,10 @@ func (s *tikvSnapshot) get(bo *Backoffer, k kv.Key) ([]byte, error) {
}
}

failpoint.Inject("snapshot-get-cache-fail", func(_ failpoint.Value) {
panic("cache miss")
})

sender := NewRegionRequestSender(s.store.regionCache, s.store.client)

req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet,
Expand Down
22 changes: 22 additions & 0 deletions store/tikv/snapshot_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"time"

. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
pb "github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -117,6 +118,27 @@ func (s *testSnapshotSuite) TestBatchGet(c *C) {
}
}

func (s *testSnapshotSuite) TestSnapshotCache(c *C) {
txn := s.beginTxn(c)
c.Assert(txn.Set(kv.Key("x"), []byte("x")), IsNil)
c.Assert(txn.Commit(context.Background()), IsNil)

txn = s.beginTxn(c)
snapshot := newTiKVSnapshot(s.store, kv.Version{Ver: txn.StartTS()}, 0)
_, err := snapshot.BatchGet(context.Background(), []kv.Key{kv.Key("x"), kv.Key("y")})
c.Assert(err, IsNil)

c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/snapshot-get-cache-fail", `return(true)`), IsNil)
ctx := context.WithValue(context.Background(), "TestSnapshotCache", true)
_, err = snapshot.Get(ctx, kv.Key("x"))
c.Assert(err, IsNil)

_, err = snapshot.Get(ctx, kv.Key("y"))
c.Assert(kv.IsErrNotFound(err), IsTrue)

c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/snapshot-get-cache-fail"), IsNil)
}

func (s *testSnapshotSuite) TestBatchGetNotExist(c *C) {
for _, rowNum := range s.rowNums {
logutil.BgLogger().Debug("test BatchGetNotExist",
Expand Down

0 comments on commit 58fc7d4

Please sign in to comment.