-
Notifications
You must be signed in to change notification settings - Fork 2.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Retry VStream errors with exponential backoff and switch tablets for repeated errors #16536
Closed
Closed
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
ab08158
Change retry error handling of vstream_manager to select a new tablet…
twthorn 293f586
Use LastError & exponential backoff
twthorn fe75145
Fail immediately for partial journal events
twthorn c303404
If we run out of tablets, surface the previous error
twthorn File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
File renamed without changes.
File renamed without changes.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,6 +26,7 @@ import ( | |
"time" | ||
|
||
"golang.org/x/exp/maps" | ||
grpcbackoff "google.golang.org/grpc/backoff" | ||
|
||
"vitess.io/vitess/go/stats" | ||
"vitess.io/vitess/go/vt/discovery" | ||
|
@@ -36,6 +37,7 @@ import ( | |
"vitess.io/vitess/go/vt/topo" | ||
"vitess.io/vitess/go/vt/topo/topoproto" | ||
"vitess.io/vitess/go/vt/vterrors" | ||
"vitess.io/vitess/go/vt/vterrors/backoff" | ||
|
||
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" | ||
querypb "vitess.io/vitess/go/vt/proto/query" | ||
|
@@ -46,9 +48,11 @@ import ( | |
|
||
// vstreamManager manages vstream requests. | ||
type vstreamManager struct { | ||
resolver *srvtopo.Resolver | ||
toposerv srvtopo.Server | ||
cell string | ||
resolver *srvtopo.Resolver | ||
toposerv srvtopo.Server | ||
cell string | ||
maxTimeInError time.Duration | ||
baseRetryDelay time.Duration | ||
|
||
vstreamsCreated *stats.CountersWithMultiLabels | ||
vstreamsLag *stats.GaugesWithMultiLabels | ||
|
@@ -61,6 +65,9 @@ const maxSkewTimeoutSeconds = 10 * 60 | |
// for a vstream | ||
const tabletPickerContextTimeout = 90 * time.Second | ||
|
||
// Default max time a tablet with the same error should be retried before retrying another. | ||
const defaultMaxTimeInError = 5 * time.Minute | ||
|
||
// vstream contains the metadata for one VStream request. | ||
type vstream struct { | ||
// mu protects parts of vgtid, the semantics of a send, and journaler. | ||
|
@@ -122,6 +129,9 @@ type vstream struct { | |
ts *topo.Server | ||
|
||
tabletPickerOptions discovery.TabletPickerOptions | ||
|
||
lastError *vterrors.LastError | ||
backoffStrategy backoff.Strategy | ||
} | ||
|
||
type journalEvent struct { | ||
|
@@ -134,9 +144,11 @@ func newVStreamManager(resolver *srvtopo.Resolver, serv srvtopo.Server, cell str | |
exporter := servenv.NewExporter(cell, "VStreamManager") | ||
|
||
return &vstreamManager{ | ||
resolver: resolver, | ||
toposerv: serv, | ||
cell: cell, | ||
resolver: resolver, | ||
toposerv: serv, | ||
cell: cell, | ||
maxTimeInError: defaultMaxTimeInError, | ||
baseRetryDelay: grpcbackoff.DefaultConfig.BaseDelay, | ||
vstreamsCreated: exporter.NewCountersWithMultiLabels( | ||
"VStreamsCreated", | ||
"Number of vstreams created", | ||
|
@@ -183,10 +195,26 @@ func (vsm *vstreamManager) VStream(ctx context.Context, tabletType topodatapb.Ta | |
CellPreference: flags.GetCellPreference(), | ||
TabletOrder: flags.GetTabletOrder(), | ||
}, | ||
lastError: initLastError(vsm.maxTimeInError), | ||
backoffStrategy: initBackOffStrategy(vsm.baseRetryDelay), | ||
} | ||
return vs.stream(ctx) | ||
} | ||
|
||
func initLastError(maxTimeInError time.Duration) *vterrors.LastError { | ||
return vterrors.NewLastError("VStreamManager", maxTimeInError) | ||
} | ||
|
||
func initBackOffStrategy(retryDelay time.Duration) backoff.Strategy { | ||
config := grpcbackoff.Config{ | ||
BaseDelay: retryDelay, | ||
Multiplier: grpcbackoff.DefaultConfig.Multiplier, | ||
Jitter: grpcbackoff.DefaultConfig.Jitter, | ||
MaxDelay: grpcbackoff.DefaultConfig.MaxDelay, | ||
} | ||
return backoff.Get("exponential", config) | ||
} | ||
|
||
// resolveParams provides defaults for the inputs if they're not specified. | ||
func (vsm *vstreamManager) resolveParams(ctx context.Context, tabletType topodatapb.TabletType, vgtid *binlogdatapb.VGtid, | ||
filter *binlogdatapb.Filter, flags *vtgatepb.VStreamFlags) (*binlogdatapb.VGtid, *binlogdatapb.Filter, *vtgatepb.VStreamFlags, error) { | ||
|
@@ -483,8 +511,9 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha | |
// It will be closed when all journal events converge. | ||
var journalDone chan struct{} | ||
ignoreTablets := make([]*topodatapb.TabletAlias, 0) | ||
var prevErr error | ||
|
||
errCount := 0 | ||
backoffIndex := 0 | ||
for { | ||
select { | ||
case <-ctx.Done(): | ||
|
@@ -524,11 +553,18 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha | |
if err != nil { | ||
return tabletPickerErr(err) | ||
} | ||
if len(tp.GetMatchingTablets(ctx)) == 0 { | ||
tperr := vterrors.Wrapf(prevErr, "zero matching tablets for %s tablet for VStream in %s/%s within the %s cell(s)", | ||
vs.tabletType.String(), sgtid.GetKeyspace(), sgtid.GetShard(), strings.Join(cells, ",")) | ||
log.Errorf("%v", tperr) | ||
return tperr | ||
} | ||
// Create a child context with a stricter timeout when picking a tablet. | ||
// This will prevent hanging in the case no tablets are found. | ||
tpCtx, tpCancel := context.WithTimeout(ctx, tabletPickerContextTimeout) | ||
defer tpCancel() | ||
tablet, err := tp.PickForStreaming(tpCtx) | ||
|
||
if err != nil { | ||
return tabletPickerErr(err) | ||
} | ||
|
@@ -585,8 +621,8 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha | |
} | ||
var vstreamCreatedOnce sync.Once | ||
err = tabletConn.VStream(ctx, req, func(events []*binlogdatapb.VEvent) error { | ||
// We received a valid event. Reset error count. | ||
errCount = 0 | ||
// We received a valid event. Reset backoff index. | ||
backoffIndex = 0 | ||
|
||
labels := []string{sgtid.Keyspace, sgtid.Shard, req.Target.TabletType.String()} | ||
|
||
|
@@ -708,49 +744,25 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha | |
err = vterrors.Errorf(vtrpcpb.Code_UNKNOWN, "vstream ended unexpectedly") | ||
} | ||
|
||
retry, ignoreTablet := vs.shouldRetry(err) | ||
if !retry { | ||
log.Errorf("vstream for %s/%s error: %v", sgtid.Keyspace, sgtid.Shard, err) | ||
return err | ||
} | ||
if ignoreTablet { | ||
vs.lastError.Record(err) | ||
prevErr = err | ||
|
||
if vs.lastError.ShouldRetry() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should be done in the |
||
log.Infof("Retrying tablet, count: %d, alias: %v, hostname: %s", backoffIndex, tablet.GetAlias(), tablet.GetHostname()) | ||
retryDelay := vs.backoffStrategy.Backoff(backoffIndex) | ||
backoffIndex++ | ||
time.Sleep(retryDelay) | ||
} else { | ||
log.Infof("Adding tablet to ignore list, alias: %v, hostname: %s", tablet.GetAlias(), tablet.GetHostname()) | ||
ignoreTablets = append(ignoreTablets, tablet.GetAlias()) | ||
vs.lastError = initLastError(vs.vsm.maxTimeInError) | ||
backoffIndex = 0 | ||
} | ||
|
||
errCount++ | ||
// Retry, at most, 3 times if the error can be retried. | ||
if errCount >= 3 { | ||
log.Errorf("vstream for %s/%s had three consecutive failures: %v", sgtid.Keyspace, sgtid.Shard, err) | ||
return err | ||
} | ||
log.Infof("vstream for %s/%s error, retrying: %v", sgtid.Keyspace, sgtid.Shard, err) | ||
} | ||
} | ||
|
||
// shouldRetry determines whether we should exit immediately or retry the vstream. | ||
// The first return value determines if the error can be retried, while the second | ||
// indicates whether the tablet with which the error occurred should be omitted | ||
// from the candidate list of tablets to choose from on the retry. | ||
// | ||
// An error should be retried if it is expected to be transient. | ||
// A tablet should be ignored upon retry if it's likely another tablet will not | ||
// produce the same error. | ||
func (vs *vstream) shouldRetry(err error) (bool, bool) { | ||
errCode := vterrors.Code(err) | ||
|
||
if errCode == vtrpcpb.Code_FAILED_PRECONDITION || errCode == vtrpcpb.Code_UNAVAILABLE { | ||
return true, false | ||
} | ||
|
||
// If there is a GTIDSet Mismatch on the tablet, omit it from the candidate | ||
// list in the TabletPicker on retry. | ||
if errCode == vtrpcpb.Code_INVALID_ARGUMENT && strings.Contains(err.Error(), "GTIDSet Mismatch") { | ||
return true, true | ||
} | ||
|
||
return false, false | ||
} | ||
|
||
// sendAll sends a group of events together while holding the lock. | ||
func (vs *vstream) sendAll(ctx context.Context, sgtid *binlogdatapb.ShardGtid, eventss [][]*binlogdatapb.VEvent) error { | ||
vs.mu.Lock() | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That seems nice. I had never even noticed it. I feel like it should probably be in
go/vt/grpcbackoff
instead though (withgrpcbackoff
as the package name too) as isn't really error related and it is grpc related — really wrapping/extendinggoogle.golang.org/grpc/backoff
.