Skip to content

Commit 03dae47

Browse files
authored
Merge pull request pingcap#5 from lichunzhu/refactorCodeDir
LGTM
2 parents 94c454c + 6a912e7 commit 03dae47

Some content is hidden

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

84 files changed

+2231
-2158
lines changed

br/pkg/backup/client.go

Lines changed: 18 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,17 @@ import (
2121
"github.com/pingcap/kvproto/pkg/metapb"
2222
"github.com/pingcap/log"
2323
filter "github.com/pingcap/tidb-tools/pkg/table-filter"
24+
"github.com/tikv/client-go/v2/oracle"
25+
"github.com/tikv/client-go/v2/tikv"
26+
"github.com/tikv/client-go/v2/txnkv/txnlock"
27+
pd "github.com/tikv/pd/client"
28+
"go.uber.org/zap"
29+
"golang.org/x/sync/errgroup"
30+
"google.golang.org/grpc/codes"
31+
"google.golang.org/grpc/status"
32+
2433
"github.com/pingcap/tidb/br/pkg/conn"
34+
connutil "github.com/pingcap/tidb/br/pkg/conn/util"
2535
berrors "github.com/pingcap/tidb/br/pkg/errors"
2636
"github.com/pingcap/tidb/br/pkg/logutil"
2737
"github.com/pingcap/tidb/br/pkg/metautil"
@@ -30,22 +40,16 @@ import (
3040
"github.com/pingcap/tidb/br/pkg/storage"
3141
"github.com/pingcap/tidb/br/pkg/summary"
3242
"github.com/pingcap/tidb/br/pkg/utils"
33-
"github.com/pingcap/tidb/distsql"
43+
"github.com/pingcap/tidb/br/pkg/utils/utildb"
44+
"github.com/pingcap/tidb/br/pkg/utils/utilpool"
45+
"github.com/pingcap/tidb/distsql/request"
3446
"github.com/pingcap/tidb/kv"
3547
"github.com/pingcap/tidb/meta"
3648
"github.com/pingcap/tidb/meta/autoid"
3749
"github.com/pingcap/tidb/parser/model"
3850
"github.com/pingcap/tidb/util"
3951
"github.com/pingcap/tidb/util/codec"
4052
"github.com/pingcap/tidb/util/ranger"
41-
"github.com/tikv/client-go/v2/oracle"
42-
"github.com/tikv/client-go/v2/tikv"
43-
"github.com/tikv/client-go/v2/txnkv/txnlock"
44-
pd "github.com/tikv/pd/client"
45-
"go.uber.org/zap"
46-
"golang.org/x/sync/errgroup"
47-
"google.golang.org/grpc/codes"
48-
"google.golang.org/grpc/status"
4953
)
5054

5155
// ClientMgr manages connections needed by backup.
@@ -226,7 +230,7 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) {
226230
ranges = ranger.FullIntRange(false)
227231
}
228232

229-
kvRanges, err := distsql.TableHandleRangesToKVRanges(nil, []int64{tblID}, tbl.IsCommonHandle, ranges, nil)
233+
kvRanges, err := request.TableHandleRangesToKVRanges(nil, []int64{tblID}, tbl.IsCommonHandle, ranges, nil)
230234
if err != nil {
231235
return nil, errors.Trace(err)
232236
}
@@ -236,7 +240,7 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) {
236240
continue
237241
}
238242
ranges = ranger.FullRange()
239-
idxRanges, err := distsql.IndexRangesToKVRanges(nil, tblID, index.ID, ranges, nil)
243+
idxRanges, err := request.IndexRangesToKVRanges(nil, tblID, index.ID, ranges, nil)
240244
if err != nil {
241245
return nil, errors.Trace(err)
242246
}
@@ -443,7 +447,7 @@ func (bc *Client) BackupRanges(
443447
}
444448

445449
// we collect all files in a single goroutine to avoid thread safety issues.
446-
workerPool := utils.NewWorkerPool(concurrency, "Ranges")
450+
workerPool := utilpool.NewWorkerPool(concurrency, "Ranges")
447451
eg, ectx := errgroup.WithContext(ctx)
448452
for id, r := range ranges {
449453
id := id
@@ -484,7 +488,7 @@ func (bc *Client) BackupRange(
484488
zap.Uint32("concurrency", req.Concurrency))
485489

486490
var allStores []*metapb.Store
487-
allStores, err = conn.GetAllTiKVStoresWithRetry(ctx, bc.mgr.GetPDClient(), conn.SkipTiFlash)
491+
allStores, err = conn.GetAllTiKVStoresWithRetry(ctx, bc.mgr.GetPDClient(), connutil.SkipTiFlash)
488492
if err != nil {
489493
return errors.Trace(err)
490494
}
@@ -758,7 +762,7 @@ func OnBackupResponse(
758762
return nil, 0, errors.Annotatef(berrors.ErrKVClusterIDMismatch, "%v on storeID: %d", resp.Error, storeID)
759763
default:
760764
// UNSAFE! TODO: use meaningful error code instead of unstructured message to find failed to write error.
761-
if utils.MessageIsRetryableStorageError(resp.GetError().GetMsg()) {
765+
if utildb.MessageIsRetryableStorageError(resp.GetError().GetMsg()) {
762766
log.Warn("backup occur storage error", zap.String("error", resp.GetError().GetMsg()))
763767
// back off 3000ms, for S3 is 99.99% available (i.e. the max outage time would less than 52.56mins per year),
764768
// this time would be probably enough for s3 to resume.

br/pkg/backup/push.go

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,11 +12,14 @@ import (
1212
"github.com/pingcap/failpoint"
1313
backuppb "github.com/pingcap/kvproto/pkg/brpb"
1414
"github.com/pingcap/kvproto/pkg/metapb"
15+
1516
berrors "github.com/pingcap/tidb/br/pkg/errors"
1617
"github.com/pingcap/tidb/br/pkg/logutil"
1718
"github.com/pingcap/tidb/br/pkg/redact"
1819
"github.com/pingcap/tidb/br/pkg/rtree"
1920
"github.com/pingcap/tidb/br/pkg/utils"
21+
"github.com/pingcap/tidb/br/pkg/utils/utildb"
22+
2023
"go.uber.org/zap"
2124
)
2225

@@ -159,7 +162,7 @@ func (push *pushDown) pushBackup(
159162
logutil.CL(ctx).Error("backup occur cluster ID error", zap.Reflect("error", v))
160163
return res, errors.Annotatef(berrors.ErrKVClusterIDMismatch, "%v", errPb)
161164
default:
162-
if utils.MessageIsRetryableStorageError(errPb.GetMsg()) {
165+
if utildb.MessageIsRetryableStorageError(errPb.GetMsg()) {
163166
logutil.CL(ctx).Warn("backup occur storage error", zap.String("error", errPb.GetMsg()))
164167
continue
165168
}

br/pkg/backup/schema.go

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -12,17 +12,19 @@ import (
1212
"github.com/pingcap/errors"
1313
backuppb "github.com/pingcap/kvproto/pkg/brpb"
1414
"github.com/pingcap/log"
15+
"go.uber.org/zap"
16+
"golang.org/x/sync/errgroup"
17+
1518
"github.com/pingcap/tidb/br/pkg/checksum"
1619
"github.com/pingcap/tidb/br/pkg/glue"
1720
"github.com/pingcap/tidb/br/pkg/logutil"
1821
"github.com/pingcap/tidb/br/pkg/metautil"
1922
"github.com/pingcap/tidb/br/pkg/summary"
2023
"github.com/pingcap/tidb/br/pkg/utils"
24+
"github.com/pingcap/tidb/br/pkg/utils/utilpool"
2125
"github.com/pingcap/tidb/kv"
2226
"github.com/pingcap/tidb/parser/model"
2327
"github.com/pingcap/tidb/statistics/handle"
24-
"go.uber.org/zap"
25-
"golang.org/x/sync/errgroup"
2628
)
2729

2830
const (
@@ -81,7 +83,7 @@ func (ss *Schemas) BackupSchemas(
8183
ctx = opentracing.ContextWithSpan(ctx, span1)
8284
}
8385

84-
workerPool := utils.NewWorkerPool(concurrency, "Schemas")
86+
workerPool := utilpool.NewWorkerPool(concurrency, "Schemas")
8587
errg, ectx := errgroup.WithContext(ctx)
8688
startAll := time.Now()
8789
op := metautil.AppendSchema

br/pkg/conn/conn.go

Lines changed: 17 additions & 65 deletions
Original file line numberDiff line numberDiff line change
@@ -16,14 +16,6 @@ import (
1616
backuppb "github.com/pingcap/kvproto/pkg/brpb"
1717
"github.com/pingcap/kvproto/pkg/metapb"
1818
"github.com/pingcap/log"
19-
berrors "github.com/pingcap/tidb/br/pkg/errors"
20-
"github.com/pingcap/tidb/br/pkg/glue"
21-
"github.com/pingcap/tidb/br/pkg/logutil"
22-
"github.com/pingcap/tidb/br/pkg/pdutil"
23-
"github.com/pingcap/tidb/br/pkg/utils"
24-
"github.com/pingcap/tidb/br/pkg/version"
25-
"github.com/pingcap/tidb/domain"
26-
"github.com/pingcap/tidb/kv"
2719
"github.com/tikv/client-go/v2/tikv"
2820
"github.com/tikv/client-go/v2/txnkv/txnlock"
2921
pd "github.com/tikv/pd/client"
@@ -34,6 +26,17 @@ import (
3426
"google.golang.org/grpc/credentials"
3527
"google.golang.org/grpc/keepalive"
3628
"google.golang.org/grpc/status"
29+
30+
"github.com/pingcap/tidb/br/pkg/conn/util"
31+
berrors "github.com/pingcap/tidb/br/pkg/errors"
32+
"github.com/pingcap/tidb/br/pkg/glue"
33+
"github.com/pingcap/tidb/br/pkg/logutil"
34+
"github.com/pingcap/tidb/br/pkg/pdutil"
35+
"github.com/pingcap/tidb/br/pkg/utils"
36+
"github.com/pingcap/tidb/br/pkg/utils/utildb"
37+
"github.com/pingcap/tidb/br/pkg/version"
38+
"github.com/pingcap/tidb/domain"
39+
"github.com/pingcap/tidb/kv"
3740
)
3841

3942
const (
@@ -115,68 +118,17 @@ type Mgr struct {
115118
ownsStorage bool
116119
}
117120

118-
// StoreBehavior is the action to do in GetAllTiKVStores when a non-TiKV
119-
// store (e.g. TiFlash store) is found.
120-
type StoreBehavior uint8
121-
122-
const (
123-
// ErrorOnTiFlash causes GetAllTiKVStores to return error when the store is
124-
// found to be a TiFlash node.
125-
ErrorOnTiFlash StoreBehavior = 0
126-
// SkipTiFlash causes GetAllTiKVStores to skip the store when it is found to
127-
// be a TiFlash node.
128-
SkipTiFlash StoreBehavior = 1
129-
// TiFlashOnly caused GetAllTiKVStores to skip the store which is not a
130-
// TiFlash node.
131-
TiFlashOnly StoreBehavior = 2
132-
)
133-
134-
// GetAllTiKVStores returns all TiKV stores registered to the PD client. The
135-
// stores must not be a tombstone and must never contain a label `engine=tiflash`.
136-
func GetAllTiKVStores(
137-
ctx context.Context,
138-
pdClient pd.Client,
139-
storeBehavior StoreBehavior,
140-
) ([]*metapb.Store, error) {
141-
// get all live stores.
142-
stores, err := pdClient.GetAllStores(ctx, pd.WithExcludeTombstone())
143-
if err != nil {
144-
return nil, errors.Trace(err)
145-
}
146-
147-
// filter out all stores which are TiFlash.
148-
j := 0
149-
for _, store := range stores {
150-
isTiFlash := false
151-
if version.IsTiFlash(store) {
152-
if storeBehavior == SkipTiFlash {
153-
continue
154-
} else if storeBehavior == ErrorOnTiFlash {
155-
return nil, errors.Annotatef(berrors.ErrPDInvalidResponse,
156-
"cannot restore to a cluster with active TiFlash stores (store %d at %s)", store.Id, store.Address)
157-
}
158-
isTiFlash = true
159-
}
160-
if !isTiFlash && storeBehavior == TiFlashOnly {
161-
continue
162-
}
163-
stores[j] = store
164-
j++
165-
}
166-
return stores[:j], nil
167-
}
168-
169121
func GetAllTiKVStoresWithRetry(ctx context.Context,
170122
pdClient pd.Client,
171-
storeBehavior StoreBehavior,
123+
storeBehavior util.StoreBehavior,
172124
) ([]*metapb.Store, error) {
173125
stores := make([]*metapb.Store, 0)
174126
var err error
175127

176-
errRetry := utils.WithRetry(
128+
errRetry := utildb.WithRetry(
177129
ctx,
178130
func() error {
179-
stores, err = GetAllTiKVStores(ctx, pdClient, storeBehavior)
131+
stores, err = util.GetAllTiKVStores(ctx, pdClient, storeBehavior)
180132
failpoint.Inject("hint-GetAllTiKVStores-error", func(val failpoint.Value) {
181133
if val.(bool) {
182134
logutil.CL(ctx).Debug("failpoint hint-GetAllTiKVStores-error injected.")
@@ -201,9 +153,9 @@ func GetAllTiKVStoresWithRetry(ctx context.Context,
201153

202154
func checkStoresAlive(ctx context.Context,
203155
pdclient pd.Client,
204-
storeBehavior StoreBehavior) error {
156+
storeBehavior util.StoreBehavior) error {
205157
// Check live tikv.
206-
stores, err := GetAllTiKVStores(ctx, pdclient, storeBehavior)
158+
stores, err := util.GetAllTiKVStores(ctx, pdclient, storeBehavior)
207159
if err != nil {
208160
log.Error("fail to get store", zap.Error(err))
209161
return errors.Trace(err)
@@ -231,7 +183,7 @@ func NewMgr(
231183
tlsConf *tls.Config,
232184
securityOption pd.SecurityOption,
233185
keepalive keepalive.ClientParameters,
234-
storeBehavior StoreBehavior,
186+
storeBehavior util.StoreBehavior,
235187
checkRequirements bool,
236188
needDomain bool,
237189
) (*Mgr, error) {

br/pkg/conn/conn_test.go

Lines changed: 16 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -9,11 +9,13 @@ import (
99
"github.com/pingcap/errors"
1010
"github.com/pingcap/failpoint"
1111
"github.com/pingcap/kvproto/pkg/metapb"
12-
"github.com/pingcap/tidb/br/pkg/pdutil"
1312
"github.com/stretchr/testify/require"
1413
pd "github.com/tikv/pd/client"
1514
"google.golang.org/grpc/codes"
1615
"google.golang.org/grpc/status"
16+
17+
"github.com/pingcap/tidb/br/pkg/conn/util"
18+
"github.com/pingcap/tidb/br/pkg/pdutil"
1719
)
1820

1921
type fakePDClient struct {
@@ -60,7 +62,7 @@ func TestGetAllTiKVStoresWithRetryCancel(t *testing.T) {
6062
stores: stores,
6163
}
6264

63-
_, err := GetAllTiKVStoresWithRetry(ctx, fpdc, SkipTiFlash)
65+
_, err := GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
6466
require.Error(t, err)
6567
require.Equal(t, codes.Canceled, status.Code(errors.Cause(err)))
6668
}
@@ -100,7 +102,7 @@ func TestGetAllTiKVStoresWithUnknown(t *testing.T) {
100102
stores: stores,
101103
}
102104

103-
_, err := GetAllTiKVStoresWithRetry(ctx, fpdc, SkipTiFlash)
105+
_, err := GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
104106
require.Error(t, err)
105107
require.Equal(t, codes.Unknown, status.Code(errors.Cause(err)))
106108
}
@@ -155,12 +157,12 @@ func TestCheckStoresAlive(t *testing.T) {
155157
stores: stores,
156158
}
157159

158-
kvStores, err := GetAllTiKVStoresWithRetry(ctx, fpdc, SkipTiFlash)
160+
kvStores, err := GetAllTiKVStoresWithRetry(ctx, fpdc, util.SkipTiFlash)
159161
require.NoError(t, err)
160162
require.Len(t, kvStores, 2)
161163
require.Equal(t, stores[2:], kvStores)
162164

163-
err = checkStoresAlive(ctx, fpdc, SkipTiFlash)
165+
err = checkStoresAlive(ctx, fpdc, util.SkipTiFlash)
164166
require.NoError(t, err)
165167
}
166168

@@ -169,38 +171,38 @@ func TestGetAllTiKVStores(t *testing.T) {
169171

170172
testCases := []struct {
171173
stores []*metapb.Store
172-
storeBehavior StoreBehavior
174+
storeBehavior util.StoreBehavior
173175
expectedStores map[uint64]int
174176
expectedError string
175177
}{
176178
{
177179
stores: []*metapb.Store{
178180
{Id: 1},
179181
},
180-
storeBehavior: SkipTiFlash,
182+
storeBehavior: util.SkipTiFlash,
181183
expectedStores: map[uint64]int{1: 1},
182184
},
183185
{
184186
stores: []*metapb.Store{
185187
{Id: 1},
186188
},
187-
storeBehavior: ErrorOnTiFlash,
189+
storeBehavior: util.ErrorOnTiFlash,
188190
expectedStores: map[uint64]int{1: 1},
189191
},
190192
{
191193
stores: []*metapb.Store{
192194
{Id: 1},
193195
{Id: 2, Labels: []*metapb.StoreLabel{{Key: "engine", Value: "tiflash"}}},
194196
},
195-
storeBehavior: SkipTiFlash,
197+
storeBehavior: util.SkipTiFlash,
196198
expectedStores: map[uint64]int{1: 1},
197199
},
198200
{
199201
stores: []*metapb.Store{
200202
{Id: 1},
201203
{Id: 2, Labels: []*metapb.StoreLabel{{Key: "engine", Value: "tiflash"}}},
202204
},
203-
storeBehavior: ErrorOnTiFlash,
205+
storeBehavior: util.ErrorOnTiFlash,
204206
expectedError: "cannot restore to a cluster with active TiFlash stores.*",
205207
},
206208
{
@@ -212,7 +214,7 @@ func TestGetAllTiKVStores(t *testing.T) {
212214
{Id: 5, Labels: []*metapb.StoreLabel{{Key: "else", Value: "tikv"}, {Key: "engine", Value: "tiflash"}}},
213215
{Id: 6, Labels: []*metapb.StoreLabel{{Key: "else", Value: "tiflash"}, {Key: "engine", Value: "tikv"}}},
214216
},
215-
storeBehavior: SkipTiFlash,
217+
storeBehavior: util.SkipTiFlash,
216218
expectedStores: map[uint64]int{1: 1, 3: 1, 4: 1, 6: 1},
217219
},
218220
{
@@ -224,7 +226,7 @@ func TestGetAllTiKVStores(t *testing.T) {
224226
{Id: 5, Labels: []*metapb.StoreLabel{{Key: "else", Value: "tikv"}, {Key: "engine", Value: "tiflash"}}},
225227
{Id: 6, Labels: []*metapb.StoreLabel{{Key: "else", Value: "tiflash"}, {Key: "engine", Value: "tikv"}}},
226228
},
227-
storeBehavior: ErrorOnTiFlash,
229+
storeBehavior: util.ErrorOnTiFlash,
228230
expectedError: "cannot restore to a cluster with active TiFlash stores.*",
229231
},
230232
{
@@ -236,14 +238,14 @@ func TestGetAllTiKVStores(t *testing.T) {
236238
{Id: 5, Labels: []*metapb.StoreLabel{{Key: "else", Value: "tikv"}, {Key: "engine", Value: "tiflash"}}},
237239
{Id: 6, Labels: []*metapb.StoreLabel{{Key: "else", Value: "tiflash"}, {Key: "engine", Value: "tikv"}}},
238240
},
239-
storeBehavior: TiFlashOnly,
241+
storeBehavior: util.TiFlashOnly,
240242
expectedStores: map[uint64]int{2: 1, 5: 1},
241243
},
242244
}
243245

244246
for _, testCase := range testCases {
245247
pdClient := fakePDClient{stores: testCase.stores}
246-
stores, err := GetAllTiKVStores(context.Background(), pdClient, testCase.storeBehavior)
248+
stores, err := util.GetAllTiKVStores(context.Background(), pdClient, testCase.storeBehavior)
247249
if len(testCase.expectedError) != 0 {
248250
require.Error(t, err)
249251
require.Regexp(t, testCase.expectedError, err.Error())

0 commit comments

Comments
 (0)