Skip to content

Commit

Permalink
br: fix panic during merging files (#33797)
Browse files Browse the repository at this point in the history
close #33801
  • Loading branch information
YuJuncen authored Apr 15, 2022
1 parent 56e1789 commit 5889d4c
Show file tree
Hide file tree
Showing 3 changed files with 80 additions and 30 deletions.
11 changes: 11 additions & 0 deletions br/pkg/logutil/logging.go
Original file line number Diff line number Diff line change
Expand Up @@ -209,8 +209,19 @@ func Keys(keys [][]byte) zap.Field {
return zap.Object("keys", zapKeysMarshaler(keys))
}

// AShortError make the zap field with key to display error without verbose representation (e.g. the stack trace).
func AShortError(key string, err error) zap.Field {
if err == nil {
return zap.Skip()
}
return zap.String(key, err.Error())
}

// ShortError make the zap field to display error without verbose representation (e.g. the stack trace).
func ShortError(err error) zap.Field {
if err == nil {
return zap.Skip()
}
return zap.String("error", err.Error())
}

Expand Down
74 changes: 44 additions & 30 deletions br/pkg/restore/merge.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,9 @@ import (
"github.com/docker/go-units"
"github.com/pingcap/errors"
backuppb "github.com/pingcap/kvproto/pkg/brpb"
"github.com/pingcap/log"
berrors "github.com/pingcap/tidb/br/pkg/errors"
"github.com/pingcap/tidb/br/pkg/logutil"
"github.com/pingcap/tidb/br/pkg/rtree"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -39,6 +41,47 @@ type MergeRangesStat struct {
MergedRegionBytesAvg int
}

// NeedsMerge checks whether two ranges needs to be merged.
func NeedsMerge(left, right *rtree.Range, splitSizeBytes, splitKeyCount uint64) bool {
leftBytes, leftKeys := left.BytesAndKeys()
rightBytes, rightKeys := right.BytesAndKeys()
if rightBytes == 0 {
return true
}
if leftBytes+rightBytes > splitSizeBytes {
return false
}
if leftKeys+rightKeys > splitKeyCount {
return false
}
tableID1, indexID1, isRecord1, err1 := tablecodec.DecodeKeyHead(kv.Key(left.StartKey))
tableID2, indexID2, isRecord2, err2 := tablecodec.DecodeKeyHead(kv.Key(right.StartKey))

// Failed to decode the file key head... can this happen?
if err1 != nil || err2 != nil {
log.Warn("Failed to parse the key head for merging files, skipping",
logutil.Key("left-start-key", left.StartKey),
logutil.Key("right-start-key", right.StartKey),
logutil.AShortError("left-err", err1),
logutil.AShortError("right-err", err2),
)
return false
}
// Merge if they are both record keys
if isRecord1 && isRecord2 {
// Do not merge ranges in different tables.
return tableID1 == tableID2
}
// If they are all index keys...
if !isRecord1 && !isRecord2 {
// Do not merge ranges in different indexes even if they are in the same
// table, as rewrite rule only supports rewriting one pattern.
// Merge left and right if they are in the same index.
return tableID1 == tableID2 && indexID1 == indexID2
}
return false
}

// MergeFileRanges returns ranges of the files are merged based on
// splitSizeBytes and splitKeyCount.
//
Expand Down Expand Up @@ -94,38 +137,9 @@ func MergeFileRanges(
}
}

needMerge := func(left, right *rtree.Range) bool {
leftBytes, leftKeys := left.BytesAndKeys()
rightBytes, rightKeys := right.BytesAndKeys()
if rightBytes == 0 {
return true
}
if leftBytes+rightBytes > splitSizeBytes {
return false
}
if leftKeys+rightKeys > splitKeyCount {
return false
}
// Do not merge ranges in different tables.
if tablecodec.DecodeTableID(kv.Key(left.StartKey)) != tablecodec.DecodeTableID(kv.Key(right.StartKey)) {
return false
}
// Do not merge ranges in different indexes even if they are in the same
// table, as rewrite rule only supports rewriting one pattern.
// tableID, indexID, indexValues, err
_, indexID1, _, err1 := tablecodec.DecodeIndexKey(kv.Key(left.StartKey))
_, indexID2, _, err2 := tablecodec.DecodeIndexKey(kv.Key(right.StartKey))
// If both of them are index keys, ...
if err1 == nil && err2 == nil {
// Merge left and right if they are in the same index.
return indexID1 == indexID2
}
// Otherwise, merge if they are both record keys
return err1 != nil && err2 != nil
}
sortedRanges := rangeTree.GetSortedRanges()
for i := 1; i < len(sortedRanges); {
if !needMerge(&sortedRanges[i-1], &sortedRanges[i]) {
if !NeedsMerge(&sortedRanges[i-1], &sortedRanges[i], splitSizeBytes, splitKeyCount) {
i++
continue
}
Expand Down
25 changes: 25 additions & 0 deletions br/pkg/restore/merge_fuzz_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0.
//go:build go1.18
// +build go1.18

package restore_test

import (
"testing"

backup "github.com/pingcap/kvproto/pkg/brpb"
"github.com/pingcap/tidb/br/pkg/restore"
"github.com/pingcap/tidb/br/pkg/rtree"
"github.com/pingcap/tidb/tablecodec"
)

func FuzzMerge(f *testing.F) {
baseKeyA := tablecodec.EncodeIndexSeekKey(42, 1, nil)
baseKeyB := tablecodec.EncodeIndexSeekKey(42, 1, nil)
f.Add([]byte(baseKeyA), []byte(baseKeyB))
f.Fuzz(func(t *testing.T, a, b []byte) {
left := rtree.Range{StartKey: a, Files: []*backup.File{{TotalKvs: 1, TotalBytes: 1}}}
right := rtree.Range{StartKey: b, Files: []*backup.File{{TotalKvs: 1, TotalBytes: 1}}}
restore.NeedsMerge(&left, &right, 42, 42)
})
}

0 comments on commit 5889d4c

Please sign in to comment.