Skip to content

Commit 1c4a9c6

Browse files
authored
br: separate the restore client (#53197)
ref #52877
1 parent 491740b commit 1c4a9c6

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

94 files changed

+9157
-8804
lines changed

br/pkg/backup/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,7 @@ go_test(
6969
shard_count = 8,
7070
deps = [
7171
"//br/pkg/conn",
72-
"//br/pkg/gluetidb",
72+
"//br/pkg/gluetidb/mock",
7373
"//br/pkg/metautil",
7474
"//br/pkg/mock",
7575
"//br/pkg/pdutil",

br/pkg/backup/client_test.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,7 @@ import (
1313
"github.com/pingcap/kvproto/pkg/encryptionpb"
1414
"github.com/pingcap/tidb/br/pkg/backup"
1515
"github.com/pingcap/tidb/br/pkg/conn"
16-
"github.com/pingcap/tidb/br/pkg/gluetidb"
16+
gluemock "github.com/pingcap/tidb/br/pkg/gluetidb/mock"
1717
"github.com/pingcap/tidb/br/pkg/metautil"
1818
"github.com/pingcap/tidb/br/pkg/mock"
1919
"github.com/pingcap/tidb/br/pkg/pdutil"
@@ -35,7 +35,7 @@ type testBackup struct {
3535

3636
mockPDClient pd.Client
3737
mockCluster *testutils.MockCluster
38-
mockGlue *gluetidb.MockGlue
38+
mockGlue *gluemock.MockGlue
3939
backupClient *backup.Client
4040

4141
cluster *mock.Cluster
@@ -46,7 +46,7 @@ func createBackupSuite(t *testing.T) *testBackup {
4646
tikvClient, mockCluster, pdClient, err := testutils.NewMockTiKV("", nil)
4747
require.NoError(t, err)
4848
s := new(testBackup)
49-
s.mockGlue = &gluetidb.MockGlue{}
49+
s.mockGlue = &gluemock.MockGlue{}
5050
s.mockPDClient = pdClient
5151
s.mockCluster = mockCluster
5252
s.ctx, s.cancel = context.WithCancel(context.Background())

br/pkg/conn/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,7 @@ go_test(
5151
"//br/pkg/config",
5252
"//br/pkg/conn/util",
5353
"//br/pkg/pdutil",
54-
"//br/pkg/utils",
54+
"//br/pkg/utiltest",
5555
"//pkg/testkit/testsetup",
5656
"@com_github_docker_go_units//:go-units",
5757
"@com_github_pingcap_errors//:errors",

br/pkg/conn/conn_test.go

Lines changed: 26 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
// Copyright 2020 PingCAP, Inc. Licensed under Apache-2.0.
22

3-
package conn
3+
package conn_test
44

55
import (
66
"context"
@@ -15,9 +15,10 @@ import (
1515
"github.com/pingcap/failpoint"
1616
"github.com/pingcap/kvproto/pkg/metapb"
1717
kvconfig "github.com/pingcap/tidb/br/pkg/config"
18+
"github.com/pingcap/tidb/br/pkg/conn"
1819
"github.com/pingcap/tidb/br/pkg/conn/util"
1920
"github.com/pingcap/tidb/br/pkg/pdutil"
20-
"github.com/pingcap/tidb/br/pkg/utils"
21+
"github.com/pingcap/tidb/br/pkg/utiltest"
2122
"github.com/stretchr/testify/require"
2223
"go.uber.org/multierr"
2324
"google.golang.org/grpc/codes"
@@ -61,11 +62,9 @@ func TestGetAllTiKVStoresWithRetryCancel(t *testing.T) {
6162
},
6263
}
6364

64-
fpdc := utils.FakePDClient{
65-
Stores: stores,
66-
}
65+
fpdc := utiltest.NewFakePDClient(stores, false, nil)
6766

68-
_, err = GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
67+
_, err = conn.GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
6968
require.Error(t, err)
7069
errs := multierr.Errors(err)
7170
require.Equal(t, 1, len(errs))
@@ -109,11 +108,9 @@ func TestGetAllTiKVStoresWithUnknown(t *testing.T) {
109108
},
110109
}
111110

112-
fpdc := utils.FakePDClient{
113-
Stores: stores,
114-
}
111+
fpdc := utiltest.NewFakePDClient(stores, false, nil)
115112

116-
_, err = GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
113+
_, err = conn.GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
117114
require.Error(t, err)
118115
errs := multierr.Errors(err)
119116
require.Equal(t, 1, len(errs))
@@ -167,16 +164,14 @@ func TestCheckStoresAlive(t *testing.T) {
167164
},
168165
}
169166

170-
fpdc := utils.FakePDClient{
171-
Stores: stores,
172-
}
167+
fpdc := utiltest.NewFakePDClient(stores, false, nil)
173168

174-
kvStores, err := GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
169+
kvStores, err := conn.GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
175170
require.NoError(t, err)
176171
require.Len(t, kvStores, 2)
177172
require.Equal(t, stores[2:], kvStores)
178173

179-
err = checkStoresAlive(ctx, fpdc, util.SkipTiFlash)
174+
err = conn.CheckStoresAlive(ctx, fpdc, util.SkipTiFlash)
180175
require.NoError(t, err)
181176
}
182177

@@ -256,7 +251,7 @@ func TestGetAllTiKVStores(t *testing.T) {
256251
}
257252

258253
for _, testCase := range testCases {
259-
pdClient := utils.FakePDClient{Stores: testCase.stores}
254+
pdClient := utiltest.NewFakePDClient(testCase.stores, false, nil)
260255
stores, err := util.GetAllTiKVStores(context.Background(), pdClient, testCase.storeBehavior)
261256
if len(testCase.expectedError) != 0 {
262257
require.Error(t, err)
@@ -275,7 +270,7 @@ func TestGetConnOnCanceledContext(t *testing.T) {
275270
ctx, cancel := context.WithCancel(context.Background())
276271
cancel()
277272

278-
mgr := &Mgr{PdController: &pdutil.PdController{}}
273+
mgr := &conn.Mgr{PdController: &pdutil.PdController{}}
279274

280275
_, err := mgr.GetBackupClient(ctx, 42)
281276
require.Error(t, err)
@@ -309,9 +304,9 @@ func TestGetMergeRegionSizeAndCount(t *testing.T) {
309304
},
310305
content: []string{""},
311306
// no tikv detected in this case
312-
importNumGoroutines: DefaultImportNumGoroutines,
313-
regionSplitSize: DefaultMergeRegionSizeBytes,
314-
regionSplitKeys: DefaultMergeRegionKeyCount,
307+
importNumGoroutines: conn.DefaultImportNumGoroutines,
308+
regionSplitSize: conn.DefaultMergeRegionSizeBytes,
309+
regionSplitKeys: conn.DefaultMergeRegionKeyCount,
315310
},
316311
{
317312
stores: []*metapb.Store{
@@ -342,9 +337,9 @@ func TestGetMergeRegionSizeAndCount(t *testing.T) {
342337
"",
343338
},
344339
// no tikv detected in this case
345-
importNumGoroutines: DefaultImportNumGoroutines,
346-
regionSplitSize: DefaultMergeRegionSizeBytes,
347-
regionSplitKeys: DefaultMergeRegionKeyCount,
340+
importNumGoroutines: conn.DefaultImportNumGoroutines,
341+
regionSplitSize: conn.DefaultMergeRegionSizeBytes,
342+
regionSplitKeys: conn.DefaultMergeRegionKeyCount,
348343
},
349344
{
350345
stores: []*metapb.Store{
@@ -426,7 +421,7 @@ func TestGetMergeRegionSizeAndCount(t *testing.T) {
426421
pctx := context.Background()
427422
for _, ca := range cases {
428423
ctx, cancel := context.WithCancel(pctx)
429-
pdCli := utils.FakePDClient{Stores: ca.stores}
424+
pdCli := utiltest.NewFakePDClient(ca.stores, false, nil)
430425
require.Equal(t, len(ca.content), len(ca.stores))
431426
count := 0
432427
mockServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
@@ -448,12 +443,12 @@ func TestGetMergeRegionSizeAndCount(t *testing.T) {
448443
}
449444

450445
httpCli := mockServer.Client()
451-
mgr := &Mgr{PdController: &pdutil.PdController{}}
446+
mgr := &conn.Mgr{PdController: &pdutil.PdController{}}
452447
mgr.PdController.SetPDClient(pdCli)
453448
kvConfigs := &kvconfig.KVConfig{
454-
ImportGoroutines: kvconfig.ConfigTerm[uint]{Value: DefaultImportNumGoroutines, Modified: false},
455-
MergeRegionSize: kvconfig.ConfigTerm[uint64]{Value: DefaultMergeRegionSizeBytes, Modified: false},
456-
MergeRegionKeyCount: kvconfig.ConfigTerm[uint64]{Value: DefaultMergeRegionKeyCount, Modified: false},
449+
ImportGoroutines: kvconfig.ConfigTerm[uint]{Value: conn.DefaultImportNumGoroutines, Modified: false},
450+
MergeRegionSize: kvconfig.ConfigTerm[uint64]{Value: conn.DefaultMergeRegionSizeBytes, Modified: false},
451+
MergeRegionKeyCount: kvconfig.ConfigTerm[uint64]{Value: conn.DefaultMergeRegionKeyCount, Modified: false},
457452
}
458453
mgr.ProcessTiKVConfigs(ctx, kvConfigs, httpCli)
459454
require.EqualValues(t, ca.regionSplitSize, kvConfigs.MergeRegionSize.Value)
@@ -591,7 +586,7 @@ func TestIsLogBackupEnabled(t *testing.T) {
591586
pctx := context.Background()
592587
for _, ca := range cases {
593588
ctx, cancel := context.WithCancel(pctx)
594-
pdCli := utils.FakePDClient{Stores: ca.stores}
589+
pdCli := utiltest.NewFakePDClient(ca.stores, false, nil)
595590
require.Equal(t, len(ca.content), len(ca.stores))
596591
count := 0
597592
mockServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
@@ -613,7 +608,7 @@ func TestIsLogBackupEnabled(t *testing.T) {
613608
}
614609

615610
httpCli := mockServer.Client()
616-
mgr := &Mgr{PdController: &pdutil.PdController{}}
611+
mgr := &conn.Mgr{PdController: &pdutil.PdController{}}
617612
mgr.PdController.SetPDClient(pdCli)
618613
enable, err := mgr.IsLogBackupEnabled(ctx, httpCli)
619614
if ca.err {
@@ -655,7 +650,7 @@ func TestHandleTiKVAddress(t *testing.T) {
655650
},
656651
}
657652
for _, ca := range cases {
658-
addr, err := handleTiKVAddress(ca.store, ca.httpPrefix)
653+
addr, err := conn.HandleTiKVAddress(ca.store, ca.httpPrefix)
659654
require.Nil(t, err)
660655
require.Equal(t, ca.result, addr.String())
661656
}

br/pkg/conn/main_test.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,11 @@ import (
2121
"go.uber.org/goleak"
2222
)
2323

24+
var (
25+
CheckStoresAlive = checkStoresAlive
26+
HandleTiKVAddress = handleTiKVAddress
27+
)
28+
2429
func TestMain(m *testing.M) {
2530
opts := []goleak.Option{
2631
goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"),

br/pkg/gluetidb/glue.go

Lines changed: 0 additions & 132 deletions
Original file line numberDiff line numberDiff line change
@@ -235,135 +235,3 @@ func (gs *tidbSession) GetGlobalVariable(name string) (string, error) {
235235
func (gs *tidbSession) showCreatePlacementPolicy(policy *model.PolicyInfo) string {
236236
return executor.ConstructResultOfShowCreatePlacementPolicy(policy)
237237
}
238-
239-
// mockSession is used for test.
240-
type mockSession struct {
241-
se sessiontypes.Session
242-
globalVars map[string]string
243-
}
244-
245-
// GetSessionCtx implements glue.Glue
246-
func (s *mockSession) GetSessionCtx() sessionctx.Context {
247-
return s.se
248-
}
249-
250-
// Execute implements glue.Session.
251-
func (s *mockSession) Execute(ctx context.Context, sql string) error {
252-
return s.ExecuteInternal(ctx, sql)
253-
}
254-
255-
func (s *mockSession) ExecuteInternal(ctx context.Context, sql string, args ...any) error {
256-
ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnBR)
257-
rs, err := s.se.ExecuteInternal(ctx, sql, args...)
258-
if err != nil {
259-
return err
260-
}
261-
// Some of SQLs (like ADMIN RECOVER INDEX) may lazily take effect
262-
// when we are polling the result set.
263-
// At least call `next` once for triggering theirs side effect.
264-
// (Maybe we'd better drain all returned rows?)
265-
if rs != nil {
266-
//nolint: errcheck
267-
defer rs.Close()
268-
c := rs.NewChunk(nil)
269-
if err := rs.Next(ctx, c); err != nil {
270-
return nil
271-
}
272-
}
273-
return nil
274-
}
275-
276-
// CreateDatabase implements glue.Session.
277-
func (*mockSession) CreateDatabase(_ context.Context, _ *model.DBInfo) error {
278-
log.Fatal("unimplemented CreateDatabase for mock session")
279-
return nil
280-
}
281-
282-
// CreatePlacementPolicy implements glue.Session.
283-
func (*mockSession) CreatePlacementPolicy(_ context.Context, _ *model.PolicyInfo) error {
284-
log.Fatal("unimplemented CreateDatabase for mock session")
285-
return nil
286-
}
287-
288-
// CreateTables implements glue.BatchCreateTableSession.
289-
func (*mockSession) CreateTables(_ context.Context, _ map[string][]*model.TableInfo,
290-
_ ...ddl.CreateTableWithInfoConfigurier) error {
291-
log.Fatal("unimplemented CreateDatabase for mock session")
292-
return nil
293-
}
294-
295-
// CreateTable implements glue.Session.
296-
func (*mockSession) CreateTable(_ context.Context, _ model.CIStr,
297-
_ *model.TableInfo, _ ...ddl.CreateTableWithInfoConfigurier) error {
298-
log.Fatal("unimplemented CreateDatabase for mock session")
299-
return nil
300-
}
301-
302-
// Close implements glue.Session.
303-
func (s *mockSession) Close() {
304-
s.se.Close()
305-
}
306-
307-
// GetGlobalVariables implements glue.Session.
308-
func (s *mockSession) GetGlobalVariable(name string) (string, error) {
309-
if ret, ok := s.globalVars[name]; ok {
310-
return ret, nil
311-
}
312-
return "True", nil
313-
}
314-
315-
// MockGlue only used for test
316-
type MockGlue struct {
317-
se sessiontypes.Session
318-
GlobalVars map[string]string
319-
}
320-
321-
func (m *MockGlue) SetSession(se sessiontypes.Session) {
322-
m.se = se
323-
}
324-
325-
// GetDomain implements glue.Glue.
326-
func (*MockGlue) GetDomain(store kv.Storage) (*domain.Domain, error) {
327-
return nil, nil
328-
}
329-
330-
// CreateSession implements glue.Glue.
331-
func (m *MockGlue) CreateSession(store kv.Storage) (glue.Session, error) {
332-
glueSession := &mockSession{
333-
se: m.se,
334-
globalVars: m.GlobalVars,
335-
}
336-
return glueSession, nil
337-
}
338-
339-
// Open implements glue.Glue.
340-
func (*MockGlue) Open(path string, option pd.SecurityOption) (kv.Storage, error) {
341-
return nil, nil
342-
}
343-
344-
// OwnsStorage implements glue.Glue.
345-
func (*MockGlue) OwnsStorage() bool {
346-
return true
347-
}
348-
349-
// StartProgress implements glue.Glue.
350-
func (*MockGlue) StartProgress(ctx context.Context, cmdName string, total int64, redirectLog bool) glue.Progress {
351-
return nil
352-
}
353-
354-
// Record implements glue.Glue.
355-
func (*MockGlue) Record(name string, value uint64) {
356-
}
357-
358-
// GetVersion implements glue.Glue.
359-
func (*MockGlue) GetVersion() string {
360-
return "mock glue"
361-
}
362-
363-
// UseOneShotSession implements glue.Glue.
364-
func (m *MockGlue) UseOneShotSession(store kv.Storage, closeDomain bool, fn func(glue.Session) error) error {
365-
glueSession := &mockSession{
366-
se: m.se,
367-
}
368-
return fn(glueSession)
369-
}

br/pkg/gluetidb/mock/BUILD.bazel

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,18 @@
1+
load("@io_bazel_rules_go//go:def.bzl", "go_library")
2+
3+
go_library(
4+
name = "mock",
5+
srcs = ["mock.go"],
6+
importpath = "github.com/pingcap/tidb/br/pkg/gluetidb/mock",
7+
visibility = ["//visibility:public"],
8+
deps = [
9+
"//br/pkg/glue",
10+
"//pkg/ddl",
11+
"//pkg/domain",
12+
"//pkg/kv",
13+
"//pkg/parser/model",
14+
"//pkg/session/types",
15+
"//pkg/sessionctx",
16+
"@com_github_tikv_pd_client//:client",
17+
],
18+
)

0 commit comments

Comments
 (0)