Skip to content

Commit

Permalink
tikv, tablecodec: log decoded key for TxnLockNotFound/WriteConflict e…
Browse files Browse the repository at this point in the history
…rror (pingcap#12869)
  • Loading branch information
lysu authored and sre-bot committed Oct 29, 2019
1 parent bb2e349 commit 6878b97
Show file tree
Hide file tree
Showing 4 changed files with 89 additions and 1 deletion.
38 changes: 37 additions & 1 deletion store/tikv/snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,9 @@ package tikv
import (
"bytes"
"context"
"encoding/json"
"fmt"
"strings"
"sync"
"time"
"unsafe"
Expand Down Expand Up @@ -394,7 +396,8 @@ func extractKeyErr(keyErr *pb.KeyError) error {
return newWriteConflictError(keyErr.Conflict)
}
if keyErr.Retryable != "" {
return kv.ErrTxnRetryable.FastGenByArgs("tikv restarts txn: " + keyErr.GetRetryable())
notFoundDetail := prettyLockNotFoundKey(keyErr.GetRetryable())
return kv.ErrTxnRetryable.FastGenByArgs("tikv restarts txn: " + keyErr.GetRetryable() + " " + notFoundDetail)
}
if keyErr.Abort != "" {
err := errors.Errorf("tikv aborts txn: %s", keyErr.GetAbort())
Expand All @@ -404,6 +407,30 @@ func extractKeyErr(keyErr *pb.KeyError) error {
return errors.Errorf("unexpected KeyError: %s", keyErr.String())
}

func prettyLockNotFoundKey(rawRetry string) string {
if !strings.Contains(rawRetry, "TxnLockNotFound") {
return ""
}
start := strings.Index(rawRetry, "[")
if start == -1 {
return ""
}
rawRetry = rawRetry[start:]
end := strings.Index(rawRetry, "]")
if end == -1 {
return ""
}
rawRetry = rawRetry[:end+1]
var key []byte
err := json.Unmarshal([]byte(rawRetry), &key)
if err != nil {
return ""
}
var buf bytes.Buffer
prettyWriteKey(&buf, key)
return buf.String()
}

func newWriteConflictError(conflict *pb.WriteConflict) error {
var buf bytes.Buffer
prettyWriteKey(&buf, conflict.Key)
Expand Down Expand Up @@ -438,6 +465,15 @@ func prettyWriteKey(buf *bytes.Buffer, key []byte) {
return
}

mKey, mField, err := tablecodec.DecodeMetaKey(key)
if err == nil {
_, err3 := fmt.Fprintf(buf, "{metaKey=true, key=%s, field=%s}", string(mKey), string(mField))
if err3 != nil {
logutil.Logger(context.Background()).Error("error", zap.Error(err3))
}
return
}

_, err4 := fmt.Fprintf(buf, "%#v", key)
if err4 != nil {
logutil.BgLogger().Error("error", zap.Error(err4))
Expand Down
21 changes: 21 additions & 0 deletions store/tikv/snapshot_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,4 +183,25 @@ func (s *testSnapshotSuite) TestWriteConflictPrettyFormat(c *C) {
"primary={tableID=411, indexID=1, indexValues={RW01, 768221109, , }} " +
kv.TxnRetryableMark
c.Assert(newWriteConflictError(conflict).Error(), Equals, expectedStr)

conflict = &pb.WriteConflict{
StartTs: 399402937522847774,
ConflictTs: 399402937719455772,
ConflictCommitTs: 399402937719455773,
Key: []byte{0x6d, 0x44, 0x42, 0x3a, 0x35, 0x36, 0x0, 0x0, 0x0, 0xfc, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x68, 0x54, 0x49, 0x44, 0x3a, 0x31, 0x30, 0x38, 0x0, 0xfe},
Primary: []byte{0x6d, 0x44, 0x42, 0x3a, 0x35, 0x36, 0x0, 0x0, 0x0, 0xfc, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x68, 0x54, 0x49, 0x44, 0x3a, 0x31, 0x30, 0x38, 0x0, 0xfe},
}
expectedStr = "[kv:9007]Write conflict, " +
"txnStartTS=399402937522847774, conflictStartTS=399402937719455772, conflictCommitTS=399402937719455773, " +
"key={metaKey=true, key=DB:56, field=TID:108} " +
"primary={metaKey=true, key=DB:56, field=TID:108} " +
kv.TxnRetryableMark
c.Assert(newWriteConflictError(conflict).Error(), Equals, expectedStr)
}

func (s *testSnapshotSuite) TestLockNotFoundPrint(c *C) {
msg := "Txn(Mvcc(TxnLockNotFound { start_ts: 408090278408224772, commit_ts: 408090279311835140, " +
"key: [116, 128, 0, 0, 0, 0, 0, 50, 137, 95, 105, 128, 0, 0, 0, 0,0 ,0, 1, 1, 67, 49, 57, 48, 57, 50, 57, 48, 255, 48, 48, 48, 48, 48, 52, 56, 54, 255, 50, 53, 53, 50, 51, 0, 0, 0, 252] }))"
key := prettyLockNotFoundKey(msg)
c.Assert(key, Equals, "{tableID=12937, indexID=1, indexValues={C19092900000048625523, }}")
}
23 changes: 23 additions & 0 deletions tablecodec/tablecodec.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/structure"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
)
Expand Down Expand Up @@ -149,6 +150,28 @@ func DecodeIndexKey(key kv.Key) (tableID int64, indexID int64, indexValues []str
return
}

// DecodeMetaKey decodes the key and get the meta key and meta field.
func DecodeMetaKey(ek kv.Key) (key []byte, field []byte, err error) {
var tp uint64
prefix := []byte("m")
if !bytes.HasPrefix(ek, prefix) {
return nil, nil, errors.New("invalid encoded hash data key prefix")
}
ek = ek[len(prefix):]
ek, key, err = codec.DecodeBytes(ek, nil)
if err != nil {
return nil, nil, errors.Trace(err)
}
ek, tp, err = codec.DecodeUint(ek)
if err != nil {
return nil, nil, errors.Trace(err)
} else if structure.TypeFlag(tp) != structure.HashData {
return nil, nil, errors.Errorf("invalid encoded hash data key flag %c", byte(tp))
}
_, field, err = codec.DecodeBytes(ek, nil)
return key, field, errors.Trace(err)
}

// DecodeIndexKeyPrefix decodes the key and gets the tableID, indexID, indexValues.
func DecodeIndexKeyPrefix(key kv.Key) (tableID int64, indexID int64, indexValues []byte, err error) {
k := key
Expand Down
8 changes: 8 additions & 0 deletions tablecodec/tablecodec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -477,6 +477,14 @@ func (s *testTableCodecSuite) TestRange(c *C) {
c.Assert([]byte(s2), Less, []byte(e2))
}

func (s *testTableCodecSuite) TestDecodeAutoIDMeta(c *C) {
keyBytes := []byte{0x6d, 0x44, 0x42, 0x3a, 0x35, 0x36, 0x0, 0x0, 0x0, 0xfc, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x68, 0x54, 0x49, 0x44, 0x3a, 0x31, 0x30, 0x38, 0x0, 0xfe}
key, field, err := DecodeMetaKey(kv.Key(keyBytes))
c.Assert(err, IsNil)
c.Assert(string(key), Equals, "DB:56")
c.Assert(string(field), Equals, "TID:108")
}

func BenchmarkHasTablePrefix(b *testing.B) {
k := kv.Key("foobar")
for i := 0; i < b.N; i++ {
Expand Down

0 comments on commit 6878b97

Please sign in to comment.