Skip to content

Commit

Permalink
lightning: "no leader" should be a retryable error (#46300) (#46317)
Browse files Browse the repository at this point in the history
close #45673
  • Loading branch information
ti-chi-bot committed Aug 23, 2023
1 parent d927657 commit b19b65c
Show file tree
Hide file tree
Showing 6 changed files with 15 additions and 3 deletions.
8 changes: 6 additions & 2 deletions br/pkg/lightning/backend/local/region_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -358,15 +358,19 @@ func (local *Backend) writeToTiKV(ctx context.Context, j *regionJob) error {
}
}

failpoint.Inject("NoLeader", func() {
log.FromContext(ctx).Warn("enter failpoint NoLeader")
leaderPeerMetas = nil
})

// if there is not leader currently, we don't forward the stage to wrote and let caller
// handle the retry.
if len(leaderPeerMetas) == 0 {
log.FromContext(ctx).Warn("write to tikv no leader",
logutil.Region(region), logutil.Leader(j.region.Leader),
zap.Uint64("leader_id", leaderID), logutil.SSTMeta(meta),
zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", totalSize))
return errors.Errorf("write to tikv with no leader returned, region '%d', leader: %d",
region.Id, leaderID)
return common.ErrNoLeader.GenWithStackByArgs(region.Id, leaderID)
}

takeTime := time.Since(begin)
Expand Down
1 change: 1 addition & 0 deletions br/pkg/lightning/common/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@ var (
ErrKVReadIndexNotReady = errors.Normalize("read index not ready", errors.RFCCodeText("Lightning:KV:ReadIndexNotReady"))
ErrKVIngestFailed = errors.Normalize("ingest tikv failed", errors.RFCCodeText("Lightning:KV:ErrKVIngestFailed"))
ErrKVRaftProposalDropped = errors.Normalize("raft proposal dropped", errors.RFCCodeText("Lightning:KV:ErrKVRaftProposalDropped"))
ErrNoLeader = errors.Normalize("write to tikv with no leader returned, region '%d', leader: %d", errors.RFCCodeText("Lightning:KV:ErrNoLeader"))

ErrUnknownBackend = errors.Normalize("unknown backend %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownBackend"))
ErrCheckLocalFile = errors.Normalize("cannot find local file for table: %s engineDir: %s", errors.RFCCodeText("Lightning:Restore:ErrCheckLocalFile"))
Expand Down
1 change: 1 addition & 0 deletions br/pkg/lightning/common/retry.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ func IsRetryableError(err error) bool {
var retryableErrorIDs = map[errors.ErrorID]struct{}{
ErrKVEpochNotMatch.ID(): {},
ErrKVNotLeader.ID(): {},
ErrNoLeader.ID(): {},
ErrKVRegionNotFound.ID(): {},
// common.ErrKVServerIsBusy is a little duplication with tmysql.ErrTiKVServerBusy
// it's because the response of sst.ingest gives us a sst.IngestResponse which doesn't contain error code,
Expand Down
1 change: 1 addition & 0 deletions br/tests/lightning_local_backend/config.toml
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
[lightning]
table-concurrency = 1
index-concurrency = 1

[checkpoint]
enable = true
Expand Down
2 changes: 1 addition & 1 deletion br/tests/lightning_local_backend/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ check_contains 'sum(c): 46'
run_sql 'DROP DATABASE cpeng;'
rm -f "/tmp/tidb_lightning_checkpoint_local_backend_test.pb"

export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=2*return("epochnotmatch")'
export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=2*return("epochnotmatch");github.com/pingcap/tidb/br/pkg/lightning/backend/local/NoLeader=1*return()'

run_lightning --backend local --enable-checkpoint=1 --log-file "$TEST_DIR/lightning-local.log" --config "tests/$TEST_NAME/config.toml"

Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -406,6 +406,11 @@ error = '''
raft proposal dropped
'''

["Lightning:KV:ErrNoLeader"]
error = '''
write to tikv with no leader returned, region '%d', leader: %d
'''

["Lightning:KV:NotLeader"]
error = '''
not leader
Expand Down

0 comments on commit b19b65c

Please sign in to comment.