Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

store,kv: snapshot doesn't cache the non-exists kv entries lead to poor 'insert ignore' performance #12872

Merged
merged 5 commits into from
Oct 23, 2019
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
25 changes: 22 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,13 @@ 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 {
// Updating cache using the reliable truth from TiKV, we set cache[key] = nil to mean non-exist.
if value, ok := m[string(key)]; ok {
s.cached[string(key)] = value
} else {
s.cached[string(key)] = nil
}
}

return m, nil
Expand Down Expand Up @@ -253,6 +266,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 Down