mirror of https://github.com/tikv/client-go.git
avoid too frequent logs (#1100)
Signed-off-by: zyguan <zhongyangguan@gmail.com>
This commit is contained in:
parent
61c486af13
commit
dbea404a84
|
|
@ -733,16 +733,19 @@ func (state *accessFollower) next(bo *retry.Backoffer, selector *replicaSelector
|
|||
leaderUnreachable := leader.store.getLivenessState() != reachable
|
||||
leaderExhausted := state.IsLeaderExhausted(leader)
|
||||
leaderInvalid := leaderEpochStale || leaderUnreachable || leaderExhausted
|
||||
if len(state.option.labels) > 0 {
|
||||
logutil.Logger(bo.GetCtx()).Warn("unable to find stores with given labels",
|
||||
if len(state.option.labels) > 0 && !state.option.leaderOnly {
|
||||
logutil.Logger(bo.GetCtx()).Warn("unable to find a store with given labels",
|
||||
zap.Uint64("region", selector.region.GetID()),
|
||||
zap.Bool("leader-epoch-stale", leaderEpochStale),
|
||||
zap.Bool("leader-unreachable", leaderUnreachable),
|
||||
zap.Bool("leader-exhausted", leaderExhausted),
|
||||
zap.Bool("stale-read", state.isStaleRead),
|
||||
zap.Any("labels", state.option.labels))
|
||||
}
|
||||
if leaderInvalid || leader.deadlineErrUsingConfTimeout {
|
||||
logutil.Logger(bo.GetCtx()).Warn("unable to find valid leader",
|
||||
zap.Uint64("region", selector.region.GetID()),
|
||||
zap.Bool("epoch-stale", leaderEpochStale),
|
||||
zap.Bool("unreachable", leaderUnreachable),
|
||||
zap.Bool("exhausted", leaderExhausted),
|
||||
zap.Bool("kv-timeout", leader.deadlineErrUsingConfTimeout),
|
||||
zap.Bool("stale-read", state.isStaleRead))
|
||||
// In stale-read, the request will fallback to leader after the local follower failure.
|
||||
// If the leader is also unavailable, we can fallback to the follower and use replica-read flag again,
|
||||
// The remote follower not tried yet, and the local follower can retry without stale-read flag.
|
||||
|
|
|
|||
Loading…
Reference in New Issue