-
Notifications
You must be signed in to change notification settings - Fork 9.8k
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
etcdserver: fix incorrect metrics generated when clients cancel watches #12196
Conversation
09281c6
to
bc50d7c
Compare
I've added unit tests for the error determination logic, but have a question about additional e2e tests (#12196 (comment)). I've done a little more digging and don't yet have any new insights as to whether it's possible to do more conclusive checks (#11375 (comment)) — maybe devising an e2e for the states in question would be useful in that regard. Even manual steps for inducing the more esoteric states would be useful as a jumping off point. |
@xiang90 In #11375 (comment) you seemed to indicate skepticism of this approach. Do you know of a specific alternative we should be pursuing instead? |
I'm going to circle back to the issue today to explore alternative fixes. |
So, looking at this with fresh eyes today, I observe a race condition here that explains the intermittent nature of the failure: https://github.com/etcd-io/etcd/blob/master/etcdserver/api/v3rpc/watch.go#L200 |
Using the following reproducer program connected to an etcd server with default options: package main
import (
"context"
"fmt"
"os"
"time"
"go.etcd.io/etcd/v3/clientv3"
)
func main() {
client, err := clientv3.New(clientv3.Config{
Endpoints: []string{"http://localhost:2379"},
})
if err != nil {
fmt.Printf("failed to create an etcd client: %s\n", err)
os.Exit(1)
}
ctx, cancel := context.WithCancel(context.Background())
wchan := client.Watch(ctx, "watcher", clientv3.WithRev(3))
go func() {
time.Sleep(2)
cancel()
}()
<-wchan
} The following portion of watch.go exhibits a race condition in stream error handling which seems to intermittently obfuscate the client/server nature of the error. Annotations inline: errc := make(chan error, 1)
// Ideally recvLoop would also use sws.wg to signal its completion
// but when stream.Context().Done() is closed, the stream's recv
// may continue to block since it uses a different context, leading to
// deadlock when calling sws.close().
go func() {
if rerr := sws.recvLoop(); rerr != nil {
if isClientCtxErr(stream.Context().Err(), rerr) {
sws.lg.Debug("failed to receive watch request from gRPC stream", zap.Error(rerr))
} else {
sws.lg.Warn("failed to receive watch request from gRPC stream", zap.Error(rerr))
streamFailures.WithLabelValues("receive", "watch").Inc()
}
errc <- rerr
}
}()
// RACE CONDITION: when the client cancels the watch, two things happen simultaneously:
//
// 1. The stream recv goroutine handles the client error and pushes it into the `errc` channel
// 2. The stream.Context().Done() channel receives a context cancellation error
//
// Depending on the ordering, the following `select` will either handle the error as a client
// cancellation (when `errc` is handled first) or a server error (when `stream` is handled first).
select {
case err = <-errc:
close(sws.ctrlStream)
case <-stream.Context().Done():
err = stream.Context().Err()
// the only server-side cancellation is noleader for now.
if err == context.Canceled {
err = rpctypes.ErrGRPCNoLeader
}
} |
9c103dd introduced an interceptor which wraps watch streams requiring a leader, causing those streams to be actively canceled when leader loss is detected. However, the error handling code assumes all stream context cancellations are from the interceptor. This assumption is broken when the context was canceled because of a client stream cancelation, triggering the race condition. Hopefully if I've gotten any of these details wrong, somebody will correct me. |
I think one of the central challenges here is the use of Need to think more about this. |
20447f0
to
1f41059
Compare
Alright, I think I found a way to fix this more sensibly by introducing a new context type the interceptor can use to definitively signal when the cancellation is from the server (e.g. the interceptor) vs. a normal closure. PTAL. |
1f41059
to
2d3fa2d
Compare
Tomorrow I can improve the test coverage here. |
2d3fa2d
to
f1d9aac
Compare
The interceptor changes are covered by the existing |
f1d9aac
to
a031968
Compare
cc @xiang90 |
@xiang90 @hexfusion who can review this? |
etcdserver/api/v3rpc/interceptor.go
Outdated
c.cancel() | ||
} | ||
|
||
func (c *leaderMonitoringContext) Err() error { |
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.
Can we document where this Err
func is called?
etcdserver/api/v3rpc/interceptor.go
Outdated
cctx, cancel := context.WithCancel(ss.Context()) | ||
ss = serverStreamWithCtx{ctx: cctx, cancel: &cancel, ServerStream: ss} | ||
cancelCtx, cancelFn := context.WithCancel(ss.Context()) | ||
monitorCtx := &leaderMonitoringContext{ |
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.
Do we really need this struct to wrap around the error? I am seeing the cancel func is only called in monitorLeader
(*ssWithCtx.cancel)()
. Maybe simpler to embed this error info in serverStreamWithCtx
and implement Err
method against it?
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.
I'm not sure I understand your proposal. The watchServer.Watch
method (in v3rpc/watch.go
) handles the context cancellation and has only the context itself and its embedded error to interpret (via Context.Err()
). The stdlib doesn't seem to provide any context implementations that allow errors to be injected, hence my wrapper.
Would you be able to try explaining in more detail what you had in mind?
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.
@gyuho friendly ping — I think what I have here solves the problem, and I'm not entirely clear on what the proposed alternative is. Can you please clarify?
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.
@ironcladlou Sorry for delay. I will take another look today.
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.
embedded error to interpret (via Context.Err()). The stdlib doesn't seem to provide any context implementations that allow errors to be injected, hence my wrapper.
Would combining these to serverStreamWithCtx
make this simpler with a field to note that cancellation error is for "no leader"?
func (ssc serverStreamWithCtx) Err() error {
c.lock.RLock()
defer c.lock.RUnlock()
if c.cancelReason != nil {
return c.cancelReason
}
return c.Context.Err()
}
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.
@gyuho okay, I think I now see where you were going with this suggestion. I spent some more time thinking about it and came up with a slightly new factoring which I hope is easier to understand.
I greatly simplified the context responsibility with a cancellableContext
struct which I hope now has very clear and self contained semantics. Then I made serverStreamWithCtx
depend on a concrete cancellableContext
and was able to move the specific cancellation error to the point it's actually generated instead of indirectly way out in the setup code like before.
I can still imagine a factoring where cancellableContext
is moved into serverStreamWithCtx
which itself would implement context.Context
, but I currently think with the new factoring the cancellableContext
is clear and self contained enough to keep it separate.
Let me know what you think — I really appreciate the feedback!
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.
It looks clear to me.
I would have some doubts if this 'alternative' context was a public class, but
as long it stays private implementation detail (as in this case), I think its good.
etcdserver/api/v3rpc/interceptor.go
Outdated
// the CancelForLeaderLoss() method is used to cancel the context. This is | ||
// so downstream context users can disambiguate the reason for the cancellation | ||
// which could be from the client (for example) or from this interceptor code. | ||
type leaderMonitoringContext struct { |
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.
I was suggesting these fields can be just moved to:
type serverStreamWithCtx struct {
...
?
f266e2f
to
ff5e5dd
Compare
Before this patch, a client which cancels the context for a watch results in the server generating a `rpctypes.ErrGRPCNoLeader` error that leads the recording of a gRPC `Unavailable` metric in association with the client watch cancellation. The metric looks like this: grpc_server_handled_total{grpc_code="Unavailable",grpc_method="Watch",grpc_service="etcdserverpb.Watch",grpc_type="bidi_stream"} So, the watch server has misidentified the error as a server error and then propagates the mistake to metrics, leading to a false indicator that the leader has been lost. This false signal then leads to false alerting. The commit 9c103dd introduced an interceptor which wraps watch streams requiring a leader, causing those streams to be actively canceled when leader loss is detected. However, the error handling code assumes all stream context cancellations are from the interceptor. This assumption is broken when the context was canceled because of a client stream cancelation. The core challenge is lack of information conveyed via `context.Context` which is shared by both the send and receive sides of the stream handling and is subject to cancellation by all paths (including the gRPC library itself). If any piece of the system cancels the shared context, there's no way for a context consumer to understand who cancelled the context or why. To solve the ambiguity of the stream interceptor code specifically, this patch introduces a custom context struct which the interceptor uses to expose a custom error through the context when the interceptor decides to actively cancel a stream. Now the consuming side can more safely assume a generic context cancellation can be propagated as a cancellation, and the server generated leader error is preserved and propagated normally without any special inference. When a client cancels the stream, there remains a race in the error handling code between the send and receive goroutines whereby the underlying gRPC error is lost in the case where the send path returns and is handled first, but this issue can be taken separately as no matter which paths wins, we can detect a generic cancellation. This is a replacement of etcd-io#11375. Fixes etcd-io#10289, etcd-io#9725, etcd-io#9576, etcd-io#9166
ff5e5dd
to
9571325
Compare
Hi guys, any update on this? Thanks in advance! |
@ironcladlou what do we need to do to progress this? |
@sferich888 I'm waiting for feedback on the latest changes (#12196 (comment)) and for somebody to make a merge decision. |
c.cancelReason = reason | ||
c.lock.Unlock() | ||
c.cancel() | ||
} |
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.
I wonder whether the first cancelation reason is not more important than the last.
In some sense all the following cancellations are being ignored if we are already Cancelling.
Is it protection in depth, or we really see multiple cancellations of the same context ?
@ironcladlou Please retrigger the tests on this change. Not "green" changes have lower chances to get maintainers attention (when scanning the PR list). |
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.
lgtm thx
@ironcladlou Can we highlight this in release 3.5 changelog and 3.4 if we need back ports? |
should we also fix the LeaseKeepAlive api metric which share the same logic as the old watch? |
Will this be backported/can it be backported to 3.4? |
Before this patch, a client which cancels the context for a watch results in the
server generating a
rpctypes.ErrGRPCNoLeader
error that leads the recording ofa gRPC
Unavailable
metric in association with the client watch cancellation.The metric looks like this:
So, the watch server has misidentified the error as a server error and then
propagates the mistake to metrics, leading to a false indicator that the leader
has been lost. This false signal then leads to false alerting.
The commit 9c103dd introduced an interceptor which wraps
watch streams requiring a leader, causing those streams to be actively canceled
when leader loss is detected.
However, the error handling code assumes all stream context cancellations are
from the interceptor. This assumption is broken when the context was canceled
because of a client stream cancelation.
The core challenge is lack of information conveyed via
context.Context
whichis shared by both the send and receive sides of the stream handling and is
subject to cancellation by all paths (including the gRPC library itself). If any
piece of the system cancels the shared context, there's no way for a context
consumer to understand who cancelled the context or why.
To solve the ambiguity of the stream interceptor code specifically, this patch
introduces a custom context struct which the interceptor uses to expose a custom
error through the context when the interceptor decides to actively cancel a
stream. Now the consuming side can more safely assume a generic context
cancellation can be propagated as a cancellation, and the server generated
leader error is preserved and propagated normally without any special inference.
When a client cancels the stream, there remains a race in the error handling
code between the send and receive goroutines whereby the underlying gRPC error
is lost in the case where the send path returns and is handled first, but this
issue can be taken separately as no matter which paths wins, we can detect a
generic cancellation.
This is a replacement of #11375.
Fixes #10289, #9725, #9576, #9166