Skip to content

Commit

Permalink
br: adjust file directory br/pkg/restore (pingcap#52944)
Browse files Browse the repository at this point in the history
  • Loading branch information
Leavrth authored and 3AceShowHand committed May 7, 2024
1 parent 4508b49 commit 5e42826
Show file tree
Hide file tree
Showing 58 changed files with 3,953 additions and 3,376 deletions.
3 changes: 2 additions & 1 deletion br/cmd/br/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,9 @@ go_library(
"//br/pkg/logutil",
"//br/pkg/metautil",
"//br/pkg/mock/mockid",
"//br/pkg/restore",
"//br/pkg/restore/utils",
"//br/pkg/rtree",
"//br/pkg/stream",
"//br/pkg/streamhelper/config",
"//br/pkg/summary",
"//br/pkg/task",
Expand Down
13 changes: 7 additions & 6 deletions br/cmd/br/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,9 @@ import (
"github.com/pingcap/tidb/br/pkg/logutil"
"github.com/pingcap/tidb/br/pkg/metautil"
"github.com/pingcap/tidb/br/pkg/mock/mockid"
"github.com/pingcap/tidb/br/pkg/restore"
restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils"
"github.com/pingcap/tidb/br/pkg/rtree"
"github.com/pingcap/tidb/br/pkg/stream"
"github.com/pingcap/tidb/br/pkg/task"
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/br/pkg/version/build"
Expand Down Expand Up @@ -207,7 +208,7 @@ func newBackupMetaValidateCommand() *cobra.Command {
for offset := uint64(0); offset < tableIDOffset; offset++ {
_, _ = tableIDAllocator.Alloc() // Ignore error
}
rewriteRules := &restore.RewriteRules{
rewriteRules := &restoreutils.RewriteRules{
Data: make([]*import_sstpb.RewriteRule, 0),
}
tableIDMap := make(map[int64]int64)
Expand Down Expand Up @@ -245,13 +246,13 @@ func newBackupMetaValidateCommand() *cobra.Command {
}
}

rules := restore.GetRewriteRules(newTable, table.Info, 0, true)
rules := restoreutils.GetRewriteRules(newTable, table.Info, 0, true)
rewriteRules.Data = append(rewriteRules.Data, rules.Data...)
tableIDMap[table.Info.ID] = int64(tableID)
}
// Validate rewrite rules
for _, file := range files {
err = restore.ValidateFileRewriteRule(file, rewriteRules)
err = restoreutils.ValidateFileRewriteRule(file, rewriteRules)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -447,8 +448,8 @@ func searchStreamBackupCommand() *cobra.Command {
if err != nil {
return errors.Trace(err)
}
comparator := restore.NewStartWithComparator()
bs := restore.NewStreamBackupSearch(s, comparator, keyBytes)
comparator := stream.NewStartWithComparator()
bs := stream.NewStreamBackupSearch(s, comparator, keyBytes)
bs.SetStartTS(startTs)
bs.SetEndTs(endTs)

Expand Down
62 changes: 8 additions & 54 deletions br/pkg/restore/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,19 +5,9 @@ go_library(
srcs = [
"batcher.go",
"client.go",
"data.go",
"db.go",
"import.go",
"import_retry.go",
"log_client.go",
"logutil.go",
"merge.go",
"pipeline_items.go",
"range.go",
"rawkv_client.go",
"search.go",
"split.go",
"stream_metas.go",
"systable_restore.go",
"util.go",
],
Expand All @@ -26,25 +16,27 @@ go_library(
deps = [
"//br/pkg/checkpoint",
"//br/pkg/checksum",
"//br/pkg/common",
"//br/pkg/conn",
"//br/pkg/conn/util",
"//br/pkg/errors",
"//br/pkg/glue",
"//br/pkg/logutil",
"//br/pkg/metautil",
"//br/pkg/pdutil",
"//br/pkg/restore/file_importer",
"//br/pkg/restore/ingestrec",
"//br/pkg/restore/log_restore",
"//br/pkg/restore/prealloc_table_id",
"//br/pkg/restore/rawkv",
"//br/pkg/restore/split",
"//br/pkg/restore/tiflashrec",
"//br/pkg/restore/utils",
"//br/pkg/rtree",
"//br/pkg/storage",
"//br/pkg/stream",
"//br/pkg/summary",
"//br/pkg/utils",
"//br/pkg/utils/iter",
"//br/pkg/utils/storewatch",
"//br/pkg/version",
"//pkg/bindinfo",
"//pkg/ddl",
Expand All @@ -63,41 +55,26 @@ go_library(
"//pkg/util/codec",
"//pkg/util/collate",
"//pkg/util/engine",
"//pkg/util/hack",
"//pkg/util/mathutil",
"//pkg/util/redact",
"//pkg/util/table-filter",
"@com_github_emirpasic_gods//maps/treemap",
"@com_github_fatih_color//:color",
"@com_github_go_sql_driver_mysql//:mysql",
"@com_github_google_uuid//:uuid",
"@com_github_opentracing_opentracing_go//:opentracing-go",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_kvproto//pkg/brpb",
"@com_github_pingcap_kvproto//pkg/errorpb",
"@com_github_pingcap_kvproto//pkg/import_sstpb",
"@com_github_pingcap_kvproto//pkg/kvrpcpb",
"@com_github_pingcap_kvproto//pkg/metapb",
"@com_github_pingcap_kvproto//pkg/recoverdatapb",
"@com_github_pingcap_log//:log",
"@com_github_tikv_client_go_v2//config",
"@com_github_tikv_client_go_v2//kv",
"@com_github_tikv_client_go_v2//oracle",
"@com_github_tikv_client_go_v2//rawkv",
"@com_github_tikv_client_go_v2//tikv",
"@com_github_tikv_client_go_v2//txnkv/rangetask",
"@com_github_tikv_client_go_v2//util",
"@com_github_tikv_pd_client//:client",
"@com_github_tikv_pd_client//http",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//backoff",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//keepalive",
"@org_golang_google_grpc//status",
"@org_golang_x_exp//maps",
"@org_golang_x_sync//errgroup",
"@org_uber_go_multierr//:multierr",
"@org_uber_go_zap//:zap",
Expand All @@ -111,79 +88,56 @@ go_test(
srcs = [
"batcher_test.go",
"client_test.go",
"data_test.go",
"db_test.go",
"import_retry_test.go",
"log_client_test.go",
"main_test.go",
"merge_test.go",
"range_test.go",
"rawkv_client_test.go",
"search_test.go",
"split_test.go",
"stream_metas_test.go",
"util_test.go",
],
embed = [":restore"],
flaky = True,
race = "off",
shard_count = 50,
shard_count = 47,
deps = [
"//br/pkg/backup",
"//br/pkg/conn",
"//br/pkg/errors",
"//br/pkg/glue",
"//br/pkg/gluetidb",
"//br/pkg/logutil",
"//br/pkg/metautil",
"//br/pkg/mock",
"//br/pkg/pdutil",
"//br/pkg/restore/file_importer",
"//br/pkg/restore/log_restore",
"//br/pkg/restore/split",
"//br/pkg/restore/tiflashrec",
"//br/pkg/restore/utils",
"//br/pkg/rtree",
"//br/pkg/storage",
"//br/pkg/stream",
"//br/pkg/utils",
"//br/pkg/utils/iter",
"//pkg/infoschema",
"//pkg/kv",
"//pkg/meta/autoid",
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/parser/types",
"//pkg/session",
"//pkg/sessionctx/stmtctx",
"//pkg/store/pdtypes",
"//pkg/tablecodec",
"//pkg/testkit",
"//pkg/testkit/testsetup",
"//pkg/types",
"//pkg/util/codec",
"//pkg/util/intest",
"//pkg/util/table-filter",
"@com_github_fsouza_fake_gcs_server//fakestorage",
"@com_github_golang_protobuf//proto",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_kvproto//pkg/brpb",
"@com_github_pingcap_kvproto//pkg/encryptionpb",
"@com_github_pingcap_kvproto//pkg/errorpb",
"@com_github_pingcap_kvproto//pkg/import_sstpb",
"@com_github_pingcap_kvproto//pkg/metapb",
"@com_github_pingcap_kvproto//pkg/pdpb",
"@com_github_pingcap_kvproto//pkg/recoverdatapb",
"@com_github_pingcap_log//:log",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//oracle",
"@com_github_tikv_client_go_v2//rawkv",
"@com_github_tikv_client_go_v2//testutils",
"@com_github_tikv_pd_client//:client",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//keepalive",
"@org_golang_google_grpc//status",
"@org_uber_go_goleak//:goleak",
"@org_uber_go_zap//:zap",
"@org_uber_go_zap//zapcore",
],
)
7 changes: 4 additions & 3 deletions br/pkg/restore/batcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (
backuppb "github.com/pingcap/kvproto/pkg/brpb"
"github.com/pingcap/log"
"github.com/pingcap/tidb/br/pkg/glue"
"github.com/pingcap/tidb/br/pkg/restore/utils"
"github.com/pingcap/tidb/br/pkg/rtree"
"github.com/pingcap/tidb/br/pkg/summary"
"go.uber.org/zap"
Expand Down Expand Up @@ -227,7 +228,7 @@ type DrainResult struct {
BlankTablesAfterSend []CreatedTable
// RewriteRules are the rewrite rules for the tables.
// the key is the table id after rewritten.
RewriteRulesMap map[int64]*RewriteRules
RewriteRulesMap map[int64]*utils.RewriteRules
Ranges []rtree.Range
// Record which part of ranges belongs to the table
TableEndOffsetInRanges []int
Expand All @@ -245,7 +246,7 @@ func (result DrainResult) Files() []TableIDWithFiles {
for _, rg := range ranges {
files = append(files, rg.Files...)
}
var rules *RewriteRules
var rules *utils.RewriteRules
if r, ok := result.RewriteRulesMap[tableID]; ok {
rules = r
}
Expand All @@ -266,7 +267,7 @@ func newDrainResult() DrainResult {
return DrainResult{
TablesToSend: make([]CreatedTable, 0),
BlankTablesAfterSend: make([]CreatedTable, 0),
RewriteRulesMap: EmptyRewriteRulesMap(),
RewriteRulesMap: utils.EmptyRewriteRulesMap(),
Ranges: make([]rtree.Range, 0),
TableEndOffsetInRanges: make([]int, 0),
}
Expand Down
13 changes: 7 additions & 6 deletions br/pkg/restore/batcher_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"github.com/pingcap/log"
"github.com/pingcap/tidb/br/pkg/metautil"
"github.com/pingcap/tidb/br/pkg/restore"
"github.com/pingcap/tidb/br/pkg/restore/utils"
"github.com/pingcap/tidb/br/pkg/rtree"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/stretchr/testify/require"
Expand All @@ -23,7 +24,7 @@ import (
type drySender struct {
mu *sync.Mutex

rewriteRules *restore.RewriteRules
rewriteRules *utils.RewriteRules
ranges []rtree.Range
nBatch int

Expand Down Expand Up @@ -60,7 +61,7 @@ func (sender *drySender) Ranges() []rtree.Range {

func newDrySender() *drySender {
return &drySender{
rewriteRules: restore.EmptyRewriteRule(),
rewriteRules: utils.EmptyRewriteRule(),
ranges: []rtree.Range{},
mu: new(sync.Mutex),
}
Expand Down Expand Up @@ -161,7 +162,7 @@ func fakeTableWithRange(id int64, rngs []rtree.Range) restore.TableWithRange {
}
tblWithRng := restore.TableWithRange{
CreatedTable: restore.CreatedTable{
RewriteRule: restore.EmptyRewriteRule(),
RewriteRule: utils.EmptyRewriteRule(),
Table: tbl.Info,
OldTable: tbl,
},
Expand All @@ -170,8 +171,8 @@ func fakeTableWithRange(id int64, rngs []rtree.Range) restore.TableWithRange {
return tblWithRng
}

func fakeRewriteRules(oldPrefix string, newPrefix string) *restore.RewriteRules {
return &restore.RewriteRules{
func fakeRewriteRules(oldPrefix string, newPrefix string) *utils.RewriteRules {
return &utils.RewriteRules{
Data: []*import_sstpb.RewriteRule{
{
OldKeyPrefix: []byte(oldPrefix),
Expand Down Expand Up @@ -299,7 +300,7 @@ func TestRewriteRules(t *testing.T) {
fakeRange("can", "cao"), fakeRange("cap", "caq"),
},
}
rewriteRules := []*restore.RewriteRules{
rewriteRules := []*utils.RewriteRules{
fakeRewriteRules("a", "ada"),
fakeRewriteRules("b", "bob"),
fakeRewriteRules("c", "cpp"),
Expand Down
Loading

0 comments on commit 5e42826

Please sign in to comment.