Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

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: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -427,7 +427,7 @@ lightning_web: ## Build Lightning web UI

.PHONY: build_br
build_br: ## Build BR (backup and restore) tool
CGO_ENABLED=1 $(GOBUILD_NO_TAGS) -tags codes $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o $(BR_BIN) ./br/cmd/br
CGO_ENABLED=1 $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o $(BR_BIN) ./br/cmd/br

.PHONY: build_lightning_for_web
build_lightning_for_web:
Expand Down
3 changes: 3 additions & 0 deletions br/pkg/rtree/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ go_library(
"@com_github_pingcap_kvproto//pkg/brpb",
"@com_github_pingcap_kvproto//pkg/kvrpcpb",
"@com_github_pingcap_log//:log",
"@com_github_tikv_client_go_v2//tikv",
"@org_uber_go_zap//:zap",
"@org_uber_go_zap//zapcore",
],
Expand All @@ -44,8 +45,10 @@ go_test(
"//pkg/tablecodec",
"//pkg/testkit/testsetup",
"@com_github_pingcap_kvproto//pkg/brpb",
"@com_github_pingcap_kvproto//pkg/keyspacepb",
"@com_github_pingcap_kvproto//pkg/kvrpcpb",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//tikv",
"@org_uber_go_goleak//:goleak",
"@org_uber_go_zap//:zap",
"@org_uber_go_zap//zapcore",
Expand Down
16 changes: 14 additions & 2 deletions br/pkg/rtree/rtree.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/util/redact"
"github.com/tikv/client-go/v2/tikv"
"go.uber.org/zap"
)

Expand Down Expand Up @@ -160,8 +161,19 @@ func NeedsMerge(left, right *RangeStats, splitSizeBytes, splitKeyCount uint64) b
if leftKeys+rightKeys > splitKeyCount {
return false
}
tableID1, indexID1, isRecord1, err1 := tablecodec.DecodeKeyHead(left.StartKey)
tableID2, indexID2, isRecord2, err2 := tablecodec.DecodeKeyHead(right.StartKey)

parseInnerKey := func(key []byte) (int64, int64, bool, error) {
// Trim the keyspace prefix.
_, innerKey, err := tikv.DecodeKey(key, kvrpcpb.APIVersion_V2)
if err != nil {
// Not a V2 (keyspaced) key.
return tablecodec.DecodeKeyHead(key)
}
return tablecodec.DecodeKeyHead(innerKey)
}

tableID1, indexID1, isRecord1, err1 := parseInnerKey(left.StartKey)
tableID2, indexID2, isRecord2, err2 := parseInnerKey(right.StartKey)

// Failed to decode the file key head... can this happen?
if err1 != nil || err2 != nil {
Expand Down
24 changes: 23 additions & 1 deletion br/pkg/rtree/rtree_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,13 +8,15 @@ import (
"testing"

backuppb "github.com/pingcap/kvproto/pkg/brpb"
"github.com/pingcap/kvproto/pkg/keyspacepb"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/tidb/br/pkg/metautil"
"github.com/pingcap/tidb/br/pkg/rtree"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/tikv"
)

func newRange(start, end []byte) *rtree.Range {
Expand Down Expand Up @@ -323,7 +325,27 @@ func encodeTableRecord(prefix kv.Key, rowID uint64) []byte {
return tablecodec.EncodeRecordKey(prefix, kv.IntHandle(rowID))
}

func TestRangeTreeMerge(t *testing.T) {
func makeEncodeKeyspacedTableRecord(keyspace uint32) func(prefix kv.Key, rowID uint64) []byte {
codec, err := tikv.NewCodecV2(tikv.ModeTxn, &keyspacepb.KeyspaceMeta{
Id: keyspace,
Name: "test",
})
if err != nil {
panic(err)
}
return func(prefix kv.Key, rowID uint64) []byte {
return codec.EncodeKey(tablecodec.EncodeRecordKey(prefix, kv.IntHandle(rowID)))
}
}

func TestRangeTreeMerge(t0 *testing.T) {
t0.Run("default-keyspace", func(t *testing.T) { testRangeTreeMerge(t, encodeTableRecord) })
t0.Run("keyspaced", func(t *testing.T) {
testRangeTreeMerge(t, makeEncodeKeyspacedTableRecord(1))
})
}

func testRangeTreeMerge(t *testing.T, encodeTableRecord func(kv.Key, uint64) []byte) {
rangeTree := rtree.NewRangeStatsTree()
tablePrefix := tablecodec.GenTableRecordPrefix(1)
for i := uint64(0); i < 10000; i += 1 {
Expand Down
1 change: 1 addition & 0 deletions br/pkg/version/build/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
importpath = "github.com/pingcap/tidb/br/pkg/version/build",
visibility = ["//visibility:public"],
deps = [
"//pkg/config/kerneltype",
"//pkg/parser/mysql",
"//pkg/util/israce",
"//pkg/util/versioninfo",
Expand Down
12 changes: 10 additions & 2 deletions br/pkg/version/build/info.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (
"strings"

"github.com/pingcap/log"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/util/israce"
"github.com/pingcap/tidb/pkg/util/versioninfo"
Expand Down Expand Up @@ -56,7 +57,9 @@ func LogInfo(name AppName) {
zap.String("git-branch", GitBranch),
zap.String("go-version", goVersion),
zap.String("utc-build-time", BuildTS),
zap.Bool("race-enabled", israce.RaceEnabled))
zap.Bool("race-enabled", israce.RaceEnabled),
zap.Bool("for-next-gen?", kerneltype.IsNextGen()),
)
}

// Info returns version information.
Expand All @@ -67,6 +70,11 @@ func Info() string {
fmt.Fprintf(&buf, "Git Branch: %s\n", GitBranch)
fmt.Fprintf(&buf, "Go Version: %s\n", goVersion)
fmt.Fprintf(&buf, "UTC Build Time: %s\n", BuildTS)
fmt.Fprintf(&buf, "Race Enabled: %t", israce.RaceEnabled)
fmt.Fprintf(&buf, "Race Enabled: %t\n", israce.RaceEnabled)
kt := "Classic"
if kerneltype.IsNextGen() {
kt = "Next-Gen"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to check kernel type has a conflict with config --keyspace-name before backup and restore?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TiKV itself supports ApiV2. I think it is reasonable to restore to a keyspace with TiKV.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We use this flag to determine restore logic (e.g. download one peer/three peers)
is that ok using next-gen type br to restore ApiV2 classical tikv?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we'd better use kerneltype.IsNextGen to determine whether download to all peers.

}
fmt.Fprintf(&buf, "Kernel Type: %s", kt)
return buf.String()
}
2 changes: 2 additions & 0 deletions br/pkg/version/build/info_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@ func TestInfo(t *testing.T) {
require.Regexp(t, "^Git Branch", lines[2])
require.Regexp(t, "^Go Version", lines[3])
require.Regexp(t, "^UTC Build Time", lines[4])
require.Regexp(t, "^Race Enabled", lines[5])
require.Regexp(t, "^Kernel Type", lines[6])
}

func TestLogInfo(*testing.T) {
Expand Down
Loading