br: download sst again with no-crypt parameters when met decryption error for downloadSST (#32847)
close pingcap/tidb#32423
This commit is contained in:
@ -160,6 +160,11 @@ func (bc *Client) GetGCTTL() int64 {
|
||||
return bc.gcTTL
|
||||
}
|
||||
|
||||
// GetStorageBackend gets storage backupend field in client.
|
||||
func (bc *Client) GetStorageBackend() *backuppb.StorageBackend {
|
||||
return bc.backend
|
||||
}
|
||||
|
||||
// GetStorage gets storage for this backup.
|
||||
func (bc *Client) GetStorage() storage.ExternalStorage {
|
||||
return bc.storage
|
||||
@ -499,7 +504,7 @@ func WriteBackupDDLJobs(metaWriter *metautil.MetaWriter, store kv.Storage, lastB
|
||||
func (bc *Client) BackupRanges(
|
||||
ctx context.Context,
|
||||
ranges []rtree.Range,
|
||||
req backuppb.BackupRequest,
|
||||
request backuppb.BackupRequest,
|
||||
concurrency uint,
|
||||
metaWriter *metautil.MetaWriter,
|
||||
progressCallBack func(ProgressUnit),
|
||||
@ -518,10 +523,12 @@ func (bc *Client) BackupRanges(
|
||||
eg, ectx := errgroup.WithContext(ctx)
|
||||
for id, r := range ranges {
|
||||
id := id
|
||||
sk, ek := r.StartKey, r.EndKey
|
||||
req := request
|
||||
req.StartKey, req.EndKey = r.StartKey, r.EndKey
|
||||
|
||||
workerPool.ApplyOnErrorGroup(eg, func() error {
|
||||
elctx := logutil.ContextWithField(ectx, logutil.RedactAny("range-sn", id))
|
||||
err := bc.BackupRange(elctx, sk, ek, req, metaWriter, progressCallBack)
|
||||
err := bc.BackupRange(elctx, req, metaWriter, progressCallBack)
|
||||
if err != nil {
|
||||
// The error due to context cancel, stack trace is meaningless, the stack shall be suspended (also clear)
|
||||
if errors.Cause(err) == context.Canceled {
|
||||
@ -529,7 +536,6 @@ func (bc *Client) BackupRanges(
|
||||
} else {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
}
|
||||
return nil
|
||||
})
|
||||
@ -541,7 +547,6 @@ func (bc *Client) BackupRanges(
|
||||
// Returns an array of files backed up.
|
||||
func (bc *Client) BackupRange(
|
||||
ctx context.Context,
|
||||
startKey, endKey []byte,
|
||||
req backuppb.BackupRequest,
|
||||
metaWriter *metautil.MetaWriter,
|
||||
progressCallBack func(ProgressUnit),
|
||||
@ -550,13 +555,13 @@ func (bc *Client) BackupRange(
|
||||
defer func() {
|
||||
elapsed := time.Since(start)
|
||||
logutil.CL(ctx).Info("backup range finished", zap.Duration("take", elapsed))
|
||||
key := "range start:" + hex.EncodeToString(startKey) + " end:" + hex.EncodeToString(endKey)
|
||||
key := "range start:" + hex.EncodeToString(req.StartKey) + " end:" + hex.EncodeToString(req.EndKey)
|
||||
if err != nil {
|
||||
summary.CollectFailureUnit(key, err)
|
||||
}
|
||||
}()
|
||||
logutil.CL(ctx).Info("backup started",
|
||||
logutil.Key("startKey", startKey), logutil.Key("endKey", endKey),
|
||||
logutil.Key("startKey", req.StartKey), logutil.Key("endKey", req.EndKey),
|
||||
zap.Uint64("rateLimit", req.RateLimit),
|
||||
zap.Uint32("concurrency", req.Concurrency))
|
||||
|
||||
@ -566,14 +571,8 @@ func (bc *Client) BackupRange(
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
req.StartKey = startKey
|
||||
req.EndKey = endKey
|
||||
req.StorageBackend = bc.backend
|
||||
|
||||
push := newPushDown(bc.mgr, len(allStores))
|
||||
|
||||
var results rtree.RangeTree
|
||||
results, err = push.pushBackup(ctx, req, allStores, progressCallBack)
|
||||
results, err := push.pushBackup(ctx, req, allStores, progressCallBack)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -581,10 +580,7 @@ func (bc *Client) BackupRange(
|
||||
|
||||
// Find and backup remaining ranges.
|
||||
// TODO: test fine grained backup.
|
||||
err = bc.fineGrainedBackup(
|
||||
ctx, startKey, endKey, req.StartVersion, req.EndVersion, req.CompressionType, req.CompressionLevel,
|
||||
req.RateLimit, req.Concurrency, req.IsRawKv, req.CipherInfo, results, progressCallBack)
|
||||
if err != nil {
|
||||
if err := bc.fineGrainedBackup(ctx, req, results, progressCallBack); err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -593,8 +589,8 @@ func (bc *Client) BackupRange(
|
||||
|
||||
if req.IsRawKv {
|
||||
logutil.CL(ctx).Info("raw ranges backed up",
|
||||
logutil.Key("startKey", startKey),
|
||||
logutil.Key("endKey", endKey),
|
||||
logutil.Key("startKey", req.StartKey),
|
||||
logutil.Key("endKey", req.EndKey),
|
||||
zap.String("cf", req.Cf))
|
||||
} else {
|
||||
logutil.CL(ctx).Info("time range backed up",
|
||||
@ -656,15 +652,7 @@ func (bc *Client) findRegionLeader(ctx context.Context, key []byte, isRawKv bool
|
||||
|
||||
func (bc *Client) fineGrainedBackup(
|
||||
ctx context.Context,
|
||||
startKey, endKey []byte,
|
||||
lastBackupTS uint64,
|
||||
backupTS uint64,
|
||||
compressType backuppb.CompressionType,
|
||||
compressLevel int32,
|
||||
rateLimit uint64,
|
||||
concurrency uint32,
|
||||
isRawKv bool,
|
||||
cipherInfo *backuppb.CipherInfo,
|
||||
req backuppb.BackupRequest,
|
||||
rangeTree rtree.RangeTree,
|
||||
progressCallBack func(ProgressUnit),
|
||||
) error {
|
||||
@ -692,7 +680,7 @@ func (bc *Client) fineGrainedBackup(
|
||||
bo := tikv.NewBackoffer(ctx, backupFineGrainedMaxBackoff)
|
||||
for {
|
||||
// Step1, check whether there is any incomplete range
|
||||
incomplete := rangeTree.GetIncompleteRange(startKey, endKey)
|
||||
incomplete := rangeTree.GetIncompleteRange(req.StartKey, req.EndKey)
|
||||
if len(incomplete) == 0 {
|
||||
return nil
|
||||
}
|
||||
@ -713,9 +701,9 @@ func (bc *Client) fineGrainedBackup(
|
||||
go func(boFork *tikv.Backoffer) {
|
||||
defer wg.Done()
|
||||
for rg := range retry {
|
||||
backoffMs, err :=
|
||||
bc.handleFineGrained(ctx, boFork, rg, lastBackupTS, backupTS,
|
||||
compressType, compressLevel, rateLimit, concurrency, isRawKv, cipherInfo, respCh)
|
||||
subReq := req
|
||||
subReq.StartKey, subReq.EndKey = rg.StartKey, rg.EndKey
|
||||
backoffMs, err := bc.handleFineGrained(ctx, boFork, subReq, respCh)
|
||||
if err != nil {
|
||||
errCh <- err
|
||||
return
|
||||
@ -855,37 +843,14 @@ func OnBackupResponse(
|
||||
func (bc *Client) handleFineGrained(
|
||||
ctx context.Context,
|
||||
bo *tikv.Backoffer,
|
||||
rg rtree.Range,
|
||||
lastBackupTS uint64,
|
||||
backupTS uint64,
|
||||
compressType backuppb.CompressionType,
|
||||
compressionLevel int32,
|
||||
rateLimit uint64,
|
||||
concurrency uint32,
|
||||
isRawKv bool,
|
||||
cipherInfo *backuppb.CipherInfo,
|
||||
req backuppb.BackupRequest,
|
||||
respCh chan<- *backuppb.BackupResponse,
|
||||
) (int, error) {
|
||||
leader, pderr := bc.findRegionLeader(ctx, rg.StartKey, isRawKv)
|
||||
leader, pderr := bc.findRegionLeader(ctx, req.StartKey, req.IsRawKv)
|
||||
if pderr != nil {
|
||||
return 0, errors.Trace(pderr)
|
||||
}
|
||||
storeID := leader.GetStoreId()
|
||||
|
||||
req := backuppb.BackupRequest{
|
||||
ClusterId: bc.clusterID,
|
||||
StartKey: rg.StartKey, // TODO: the range may cross region.
|
||||
EndKey: rg.EndKey,
|
||||
StartVersion: lastBackupTS,
|
||||
EndVersion: backupTS,
|
||||
StorageBackend: bc.backend,
|
||||
RateLimit: rateLimit,
|
||||
Concurrency: concurrency,
|
||||
IsRawKv: isRawKv,
|
||||
CompressionType: compressType,
|
||||
CompressionLevel: compressionLevel,
|
||||
CipherInfo: cipherInfo,
|
||||
}
|
||||
lockResolver := bc.mgr.GetLockResolver()
|
||||
client, err := bc.mgr.GetBackupClient(ctx, storeID)
|
||||
if err != nil {
|
||||
@ -906,7 +871,7 @@ func (bc *Client) handleFineGrained(
|
||||
// Handle responses with the same backoffer.
|
||||
func(resp *backuppb.BackupResponse) error {
|
||||
response, shouldBackoff, err1 :=
|
||||
OnBackupResponse(storeID, bo, backupTS, lockResolver, resp)
|
||||
OnBackupResponse(storeID, bo, req.EndVersion, lockResolver, resp)
|
||||
if err1 != nil {
|
||||
return err1
|
||||
}
|
||||
|
||||
@ -6,6 +6,7 @@ import (
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
@ -259,6 +260,40 @@ func (importer *FileImporter) SetRawRange(startKey, endKey []byte) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// getKeyRangeForFiles gets the maximum range on files.
|
||||
func (importer *FileImporter) getKeyRangeForFiles(
|
||||
files []*backuppb.File,
|
||||
rewriteRules *RewriteRules,
|
||||
) ([]byte, []byte, error) {
|
||||
var (
|
||||
startKey, endKey []byte
|
||||
start, end []byte
|
||||
err error
|
||||
)
|
||||
|
||||
for _, f := range files {
|
||||
if importer.isRawKvMode {
|
||||
start, end = f.GetStartKey(), f.GetEndKey()
|
||||
} else {
|
||||
start, end, err = rewriteFileKeys(f, rewriteRules)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Trace(err)
|
||||
}
|
||||
}
|
||||
|
||||
if len(startKey) == 0 || bytes.Compare(start, startKey) < 0 {
|
||||
startKey = start
|
||||
}
|
||||
if len(endKey) == 0 || bytes.Compare(endKey, end) < 0 {
|
||||
endKey = end
|
||||
}
|
||||
}
|
||||
|
||||
log.Debug("rewrite file keys", logutil.Files(files),
|
||||
logutil.Key("startKey", startKey), logutil.Key("endKey", endKey))
|
||||
return startKey, endKey, nil
|
||||
}
|
||||
|
||||
// Import tries to import a file.
|
||||
// All rules must contain encoded keys.
|
||||
func (importer *FileImporter) Import(
|
||||
@ -270,32 +305,14 @@ func (importer *FileImporter) Import(
|
||||
) error {
|
||||
start := time.Now()
|
||||
log.Debug("import file", logutil.Files(files))
|
||||
|
||||
// Rewrite the start key and end key of file to scan regions
|
||||
var startKey, endKey []byte
|
||||
if importer.isRawKvMode {
|
||||
startKey = files[0].StartKey
|
||||
endKey = files[0].EndKey
|
||||
} else {
|
||||
for _, f := range files {
|
||||
start, end, err := rewriteFileKeys(f, rewriteRules)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
if len(startKey) == 0 || bytes.Compare(startKey, start) > 0 {
|
||||
startKey = start
|
||||
}
|
||||
if bytes.Compare(endKey, end) < 0 {
|
||||
endKey = end
|
||||
}
|
||||
}
|
||||
startKey, endKey, err := importer.getKeyRangeForFiles(files, rewriteRules)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
log.Debug("rewrite file keys",
|
||||
logutil.Files(files),
|
||||
logutil.Key("startKey", startKey),
|
||||
logutil.Key("endKey", endKey))
|
||||
|
||||
err := utils.WithRetry(ctx, func() error {
|
||||
err = utils.WithRetry(ctx, func() error {
|
||||
tctx, cancel := context.WithTimeout(ctx, importScanRegionTime)
|
||||
defer cancel()
|
||||
// Scan regions covered by the file range
|
||||
@ -311,35 +328,7 @@ func (importer *FileImporter) Import(
|
||||
for _, regionInfo := range regionInfos {
|
||||
info := regionInfo
|
||||
// Try to download file.
|
||||
downloadMetas := make([]*import_sstpb.SSTMeta, 0, len(files))
|
||||
remainFiles := files
|
||||
errDownload := utils.WithRetry(ctx, func() error {
|
||||
var e error
|
||||
for i, f := range remainFiles {
|
||||
var downloadMeta *import_sstpb.SSTMeta
|
||||
if importer.isRawKvMode {
|
||||
downloadMeta, e = importer.downloadRawKVSST(ctx, info, f, cipher, apiVersion)
|
||||
} else {
|
||||
downloadMeta, e = importer.downloadSST(ctx, info, f, rewriteRules, cipher)
|
||||
}
|
||||
failpoint.Inject("restore-storage-error", func(val failpoint.Value) {
|
||||
msg := val.(string)
|
||||
log.Debug("failpoint restore-storage-error injected.", zap.String("msg", msg))
|
||||
e = errors.Annotate(e, msg)
|
||||
})
|
||||
failpoint.Inject("restore-gRPC-error", func(_ failpoint.Value) {
|
||||
log.Warn("the connection to TiKV has been cut by a neko, meow :3")
|
||||
e = status.Error(codes.Unavailable, "the connection to TiKV has been cut by a neko, meow :3")
|
||||
})
|
||||
if e != nil {
|
||||
remainFiles = remainFiles[i:]
|
||||
return errors.Trace(e)
|
||||
}
|
||||
downloadMetas = append(downloadMetas, downloadMeta)
|
||||
}
|
||||
|
||||
return nil
|
||||
}, utils.NewDownloadSSTBackoffer())
|
||||
downloadMetas, errDownload := importer.download(ctx, info, files, rewriteRules, cipher, apiVersion)
|
||||
if errDownload != nil {
|
||||
for _, e := range multierr.Errors(errDownload) {
|
||||
switch errors.Cause(e) { // nolint:errorlint
|
||||
@ -364,67 +353,10 @@ func (importer *FileImporter) Import(
|
||||
logutil.ShortError(errDownload))
|
||||
return errors.Trace(errDownload)
|
||||
}
|
||||
log.Debug("download file done", zap.String("file-sample", files[0].Name), zap.Stringer("take", time.Since(start)),
|
||||
logutil.Key("start", files[0].StartKey),
|
||||
logutil.Key("end", files[0].EndKey),
|
||||
)
|
||||
ingestResp, errIngest := importer.ingestSSTs(ctx, downloadMetas, info)
|
||||
ingestRetry:
|
||||
for errIngest == nil {
|
||||
errPb := ingestResp.GetError()
|
||||
if errPb == nil {
|
||||
// Ingest success
|
||||
break ingestRetry
|
||||
}
|
||||
switch {
|
||||
case errPb.NotLeader != nil:
|
||||
// If error is `NotLeader`, update the region info and retry
|
||||
var newInfo *RegionInfo
|
||||
if newLeader := errPb.GetNotLeader().GetLeader(); newLeader != nil {
|
||||
newInfo = &RegionInfo{
|
||||
Leader: newLeader,
|
||||
Region: info.Region,
|
||||
}
|
||||
} else {
|
||||
// Slow path, get region from PD
|
||||
newInfo, errIngest = importer.metaClient.GetRegion(
|
||||
ctx, info.Region.GetStartKey())
|
||||
if errIngest != nil {
|
||||
break ingestRetry
|
||||
}
|
||||
// do not get region info, wait a second and continue
|
||||
if newInfo == nil {
|
||||
log.Warn("get region by key return nil", logutil.Region(info.Region))
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
}
|
||||
log.Debug("ingest sst returns not leader error, retry it",
|
||||
logutil.Region(info.Region),
|
||||
zap.Stringer("newLeader", newInfo.Leader))
|
||||
|
||||
if !checkRegionEpoch(newInfo, info) {
|
||||
errIngest = errors.Trace(berrors.ErrKVEpochNotMatch)
|
||||
break ingestRetry
|
||||
}
|
||||
ingestResp, errIngest = importer.ingestSSTs(ctx, downloadMetas, newInfo)
|
||||
case errPb.EpochNotMatch != nil:
|
||||
// TODO handle epoch not match error
|
||||
// 1. retry download if needed
|
||||
// 2. retry ingest
|
||||
errIngest = errors.Trace(berrors.ErrKVEpochNotMatch)
|
||||
break ingestRetry
|
||||
case errPb.KeyNotInRegion != nil:
|
||||
errIngest = errors.Trace(berrors.ErrKVKeyNotInRegion)
|
||||
break ingestRetry
|
||||
default:
|
||||
// Other errors like `ServerIsBusy`, `RegionNotFound`, etc. should be retryable
|
||||
errIngest = errors.Annotatef(berrors.ErrKVIngestFailed, "ingest error %s", errPb)
|
||||
break ingestRetry
|
||||
}
|
||||
}
|
||||
|
||||
if errIngest != nil {
|
||||
log.Debug("download file done",
|
||||
zap.String("file-sample", files[0].Name), zap.Stringer("take", time.Since(start)),
|
||||
logutil.Key("start", files[0].StartKey), logutil.Key("end", files[0].EndKey))
|
||||
if errIngest := importer.ingest(ctx, info, downloadMetas); errIngest != nil {
|
||||
log.Error("ingest file failed",
|
||||
logutil.Files(files),
|
||||
logutil.SSTMetas(downloadMetas),
|
||||
@ -433,12 +365,12 @@ func (importer *FileImporter) Import(
|
||||
return errors.Trace(errIngest)
|
||||
}
|
||||
}
|
||||
|
||||
log.Debug("ingest file done", zap.String("file-sample", files[0].Name), zap.Stringer("take", time.Since(start)))
|
||||
for _, f := range files {
|
||||
summary.CollectSuccessUnit(summary.TotalKV, 1, f.TotalKvs)
|
||||
summary.CollectSuccessUnit(summary.TotalBytes, 1, f.TotalBytes)
|
||||
}
|
||||
|
||||
return nil
|
||||
}, utils.NewImportSSTBackoffer())
|
||||
return errors.Trace(err)
|
||||
@ -452,6 +384,60 @@ func (importer *FileImporter) setDownloadSpeedLimit(ctx context.Context, storeID
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
func (importer *FileImporter) download(
|
||||
ctx context.Context,
|
||||
regionInfo *RegionInfo,
|
||||
files []*backuppb.File,
|
||||
rewriteRules *RewriteRules,
|
||||
cipher *backuppb.CipherInfo,
|
||||
apiVersion kvrpcpb.APIVersion,
|
||||
) ([]*import_sstpb.SSTMeta, error) {
|
||||
var (
|
||||
downloadMetas = make([]*import_sstpb.SSTMeta, 0, len(files))
|
||||
remainFiles = files
|
||||
)
|
||||
errDownload := utils.WithRetry(ctx, func() error {
|
||||
var e error
|
||||
for i, f := range remainFiles {
|
||||
var downloadMeta *import_sstpb.SSTMeta
|
||||
if importer.isRawKvMode {
|
||||
downloadMeta, e = importer.downloadRawKVSST(ctx, regionInfo, f, cipher, apiVersion)
|
||||
} else {
|
||||
downloadMeta, e = importer.downloadSST(ctx, regionInfo, f, rewriteRules, cipher)
|
||||
}
|
||||
|
||||
failpoint.Inject("restore-storage-error", func(val failpoint.Value) {
|
||||
msg := val.(string)
|
||||
log.Debug("failpoint restore-storage-error injected.", zap.String("msg", msg))
|
||||
e = errors.Annotate(e, msg)
|
||||
})
|
||||
failpoint.Inject("restore-gRPC-error", func(_ failpoint.Value) {
|
||||
log.Warn("the connection to TiKV has been cut by a neko, meow :3")
|
||||
e = status.Error(codes.Unavailable, "the connection to TiKV has been cut by a neko, meow :3")
|
||||
})
|
||||
if isDecryptSstErr(e) {
|
||||
log.Info("fail to decrypt when download sst, try again with no-crypt", logutil.File(f))
|
||||
if importer.isRawKvMode {
|
||||
downloadMeta, e = importer.downloadRawKVSST(ctx, regionInfo, f, nil, apiVersion)
|
||||
} else {
|
||||
downloadMeta, e = importer.downloadSST(ctx, regionInfo, f, rewriteRules, nil)
|
||||
}
|
||||
}
|
||||
|
||||
if e != nil {
|
||||
remainFiles = remainFiles[i:]
|
||||
return errors.Trace(e)
|
||||
}
|
||||
|
||||
downloadMetas = append(downloadMetas, downloadMeta)
|
||||
}
|
||||
|
||||
return nil
|
||||
}, utils.NewDownloadSSTBackoffer())
|
||||
|
||||
return downloadMetas, errDownload
|
||||
}
|
||||
|
||||
func (importer *FileImporter) downloadSST(
|
||||
ctx context.Context,
|
||||
regionInfo *RegionInfo,
|
||||
@ -592,6 +578,67 @@ func (importer *FileImporter) downloadRawKVSST(
|
||||
return &sstMeta, nil
|
||||
}
|
||||
|
||||
func (importer *FileImporter) ingest(
|
||||
ctx context.Context,
|
||||
info *RegionInfo,
|
||||
downloadMetas []*import_sstpb.SSTMeta,
|
||||
) error {
|
||||
for {
|
||||
ingestResp, errIngest := importer.ingestSSTs(ctx, downloadMetas, info)
|
||||
if errIngest != nil {
|
||||
return errors.Trace(errIngest)
|
||||
}
|
||||
|
||||
errPb := ingestResp.GetError()
|
||||
switch {
|
||||
case errPb == nil:
|
||||
return nil
|
||||
case errPb.NotLeader != nil:
|
||||
// If error is `NotLeader`, update the region info and retry
|
||||
var newInfo *RegionInfo
|
||||
if newLeader := errPb.GetNotLeader().GetLeader(); newLeader != nil {
|
||||
newInfo = &RegionInfo{
|
||||
Leader: newLeader,
|
||||
Region: info.Region,
|
||||
}
|
||||
} else {
|
||||
for {
|
||||
// Slow path, get region from PD
|
||||
newInfo, errIngest = importer.metaClient.GetRegion(
|
||||
ctx, info.Region.GetStartKey())
|
||||
if errIngest != nil {
|
||||
return errors.Trace(errIngest)
|
||||
}
|
||||
if newInfo != nil {
|
||||
break
|
||||
}
|
||||
// do not get region info, wait a second and GetRegion() again.
|
||||
log.Warn("get region by key return nil", logutil.Region(info.Region))
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
}
|
||||
|
||||
if !checkRegionEpoch(newInfo, info) {
|
||||
return errors.Trace(berrors.ErrKVEpochNotMatch)
|
||||
}
|
||||
log.Debug("ingest sst returns not leader error, retry it",
|
||||
logutil.Region(info.Region),
|
||||
zap.Stringer("newLeader", newInfo.Leader))
|
||||
info = newInfo
|
||||
case errPb.EpochNotMatch != nil:
|
||||
// TODO handle epoch not match error
|
||||
// 1. retry download if needed
|
||||
// 2. retry ingest
|
||||
return errors.Trace(berrors.ErrKVEpochNotMatch)
|
||||
case errPb.KeyNotInRegion != nil:
|
||||
return errors.Trace(berrors.ErrKVKeyNotInRegion)
|
||||
default:
|
||||
// Other errors like `ServerIsBusy`, `RegionNotFound`, etc. should be retryable
|
||||
return errors.Annotatef(berrors.ErrKVIngestFailed, "ingest error %s", errPb)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (importer *FileImporter) ingestSSTs(
|
||||
ctx context.Context,
|
||||
sstMetas []*import_sstpb.SSTMeta,
|
||||
@ -629,3 +676,9 @@ func (importer *FileImporter) ingestSSTs(
|
||||
resp, err := importer.importClient.MultiIngest(ctx, leader.GetStoreId(), req)
|
||||
return resp, errors.Trace(err)
|
||||
}
|
||||
|
||||
func isDecryptSstErr(err error) bool {
|
||||
return err != nil &&
|
||||
strings.Contains(err.Error(), "Engine Engine") &&
|
||||
strings.Contains(err.Error(), "Corruption: Bad table magic number")
|
||||
}
|
||||
|
||||
@ -323,6 +323,7 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig
|
||||
StartVersion: cfg.LastBackupTS,
|
||||
EndVersion: backupTS,
|
||||
RateLimit: cfg.RateLimit,
|
||||
StorageBackend: client.GetStorageBackend(),
|
||||
Concurrency: defaultBackupConcurrency,
|
||||
CompressionType: cfg.CompressionType,
|
||||
CompressionLevel: cfg.CompressionLevel,
|
||||
|
||||
@ -201,10 +201,13 @@ func RunBackupRaw(c context.Context, g glue.Glue, cmdName string, cfg *RawKvConf
|
||||
|
||||
req := backuppb.BackupRequest{
|
||||
ClusterId: client.GetClusterID(),
|
||||
StartKey: backupRange.StartKey,
|
||||
EndKey: backupRange.StartKey,
|
||||
StartVersion: 0,
|
||||
EndVersion: 0,
|
||||
RateLimit: cfg.RateLimit,
|
||||
Concurrency: cfg.Concurrency,
|
||||
StorageBackend: client.GetStorageBackend(),
|
||||
IsRawKv: true,
|
||||
Cf: cfg.CF,
|
||||
CompressionType: cfg.CompressionType,
|
||||
@ -213,7 +216,7 @@ func RunBackupRaw(c context.Context, g glue.Glue, cmdName string, cfg *RawKvConf
|
||||
}
|
||||
metaWriter := metautil.NewMetaWriter(client.GetStorage(), metautil.MetaFileSize, false, &cfg.CipherInfo)
|
||||
metaWriter.StartWriteMetasAsync(ctx, metautil.AppendDataFile)
|
||||
err = client.BackupRange(ctx, backupRange.StartKey, backupRange.EndKey, req, metaWriter, progressCallBack)
|
||||
err = client.BackupRange(ctx, req, metaWriter, progressCallBack)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -17,9 +17,10 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
// importSSTRetryTimes specifies the retry time. Its longest time is about 90s-100s.
|
||||
importSSTRetryTimes = 16
|
||||
importSSTWaitInterval = 10 * time.Millisecond
|
||||
importSSTMaxWaitInterval = 1 * time.Second
|
||||
importSSTWaitInterval = 40 * time.Millisecond
|
||||
importSSTMaxWaitInterval = 10 * time.Second
|
||||
|
||||
downloadSSTRetryTimes = 8
|
||||
downloadSSTWaitInterval = 1 * time.Second
|
||||
|
||||
79
br/tests/br_crypter2/run.sh
Executable file
79
br/tests/br_crypter2/run.sh
Executable file
@ -0,0 +1,79 @@
|
||||
#!/bin/sh
|
||||
#
|
||||
# Copyright 2022 PingCAP, Inc.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
set -eu
|
||||
DB="$TEST_NAME"
|
||||
TABLE="usertable"
|
||||
DB_COUNT=3
|
||||
|
||||
function create_db_with_table(){
|
||||
for i in $(seq $DB_COUNT); do
|
||||
run_sql "CREATE DATABASE $DB${i};"
|
||||
go-ycsb load mysql -P tests/$TEST_NAME/workload -p mysql.host=$TIDB_IP -p mysql.port=$TIDB_PORT -p mysql.user=root -p mysql.db=$DB${i}
|
||||
done
|
||||
}
|
||||
|
||||
function drop_db(){
|
||||
for i in $(seq $DB_COUNT); do
|
||||
run_sql "DROP DATABASE $DB${i};"
|
||||
done
|
||||
}
|
||||
|
||||
function check_db_row(){
|
||||
for i in $(seq $DB_COUNT); do
|
||||
row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}')
|
||||
done
|
||||
|
||||
fail=false
|
||||
for i in $(seq $DB_COUNT); do
|
||||
if [ "${row_count_ori[i]}" != "${row_count_new[i]}" ];then
|
||||
fail=true
|
||||
echo "TEST: [$TEST_NAME] fail on database $DB${i}"
|
||||
fi
|
||||
echo "database $DB${i} [original] row count: ${row_count_ori[i]}, [after br] row count: ${row_count_new[i]}"
|
||||
done
|
||||
|
||||
if $fail; then
|
||||
echo "TEST: [$TEST_NAME] failed!"
|
||||
exit 1
|
||||
fi
|
||||
}
|
||||
|
||||
# Create dbs with table
|
||||
create_db_with_table
|
||||
|
||||
# Get the original row count from dbs
|
||||
for i in $(seq $DB_COUNT); do
|
||||
row_count_ori[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}')
|
||||
done
|
||||
|
||||
# Test backup/restore with crypt for br
|
||||
CRYPTER_METHOD=aes128-ctr
|
||||
CRYPTER_KEY="0123456789abcdef0123456789abcdef"
|
||||
|
||||
export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/noop-backup=100*return(1)"
|
||||
run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \
|
||||
--use-backupmeta-v2=true --check-requirements=false --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_KEY
|
||||
|
||||
drop_db
|
||||
|
||||
run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \
|
||||
--check-requirements=false --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_KEY
|
||||
|
||||
check_db_row
|
||||
|
||||
# Drop dbs finally
|
||||
drop_db
|
||||
12
br/tests/br_crypter2/workload
Normal file
12
br/tests/br_crypter2/workload
Normal file
@ -0,0 +1,12 @@
|
||||
recordcount=1000
|
||||
operationcount=0
|
||||
workload=core
|
||||
|
||||
readallfields=true
|
||||
|
||||
readproportion=0
|
||||
updateproportion=0
|
||||
scanproportion=0
|
||||
insertproportion=0
|
||||
|
||||
requestdistribution=uniform
|
||||
Reference in New Issue
Block a user