Fixed unexpected cancelled watch with WatchID=0.#14296
Conversation
|
I found some comments that said watchID starts at 0, I don't know if there are any other effects. |
|
Thanks @kafuu-chino for the PR, but I did not get what's the issue you are fixing. The PR is also causing some test failures. |
|
@ahrtr
func (wr *watchRequest) toPB() *pb.WatchRequest {
req := &pb.WatchCreateRequest{
StartRevision: wr.rev,
Key: []byte(wr.key),
RangeEnd: []byte(wr.end),
ProgressNotify: wr.progressNotify,
Filters: wr.filters,
PrevKv: wr.prevKV,
Fragment: wr.fragment,
}
cr := &pb.WatchRequest_CreateRequest{CreateRequest: req}
return &pb.WatchRequest{RequestUnion: cr}
}
// AutoWatchID is the watcher ID passed in WatchStream.Watch when no
// user-provided ID is available. If pass, an ID will automatically be assigned.
const AutoWatchID WatchID = 0Due to if id == AutoWatchID {
for ws.watchers[ws.nextID] != nil {
ws.nextID++
}
id = ws.nextID
ws.nextID++
}
if !sws.isWatchPermitted(creq) {
wr := &pb.WatchResponse{
Header: sws.newResponseHeader(sws.watchStream.Rev()),
WatchId: creq.WatchId, // = 0
Canceled: true,
Created: true,
CancelReason: rpctypes.ErrGRPCPermissionDenied.Error(),
}
select {
case sws.ctrlStream <- wr:
continue
case <-sws.closec:
return nil
}
}
case c, ok := <-sws.ctrlStream:
if !ok {
return
}
if err := sws.gRPCStream.Send(c); err != nil {
if isClientCtxErr(sws.gRPCStream.Context().Err(), err) {
sws.lg.Debug("failed to send watch control response to gRPC stream", zap.Error(err))
} else {
ws.lg.Warn("failed to send watch control response to gRPC stream", zap.Error(err))
streamFailures.WithLabelValues("send", "watch").Inc()
}
return
}
// track id creation
wid := mvcc.WatchID(c.WatchId)
if c.Canceled {
delete(ids, wid)
continue
}
case pbresp := <-w.respc:
if cur == nil || pbresp.Created || pbresp.Canceled {
cur = pbresp
} else if cur != nil && cur.WatchId == pbresp.WatchId {
// merge new events
cur.Events = append(cur.Events, pbresp.Events...)
// update "Fragment" field; last response with "Fragment" == false
cur.Fragment = pbresp.Fragment
}
switch {
case pbresp.Created: |
|
I looked at the commit history and restored |
| wr := &pb.WatchResponse{ | ||
| Header: sws.newResponseHeader(sws.watchStream.Rev()), | ||
| WatchId: creq.WatchId, | ||
| WatchId: -1, |
There was a problem hiding this comment.
Probably we should check the value of creq.WatchId, how about the logic something like below?
watchId := creq.WatchId
if watchId == int64(mvcc.AutoWatchID) {
watchId = -1
}
Another choice is to make watchId starting with 1 instead of 0. But we need to fix some test cases, and it also change the default behavior, accordingly might have some impact on client applications which depend on the watch.
I manually verified that this PR indeed can fix the issue #12385.
But the overall design looks weird to me. The existing watch can continue to work even the simple token expired, because the client can retry the authentication. But creating new watch stream using exactly the same client doesn't work, it's a little strange.
It seems that we need to get the overall design clarified firstly. WDYT @mitake
There was a problem hiding this comment.
I set watchId starting with 1 at the beginning, all issues are resolved and do not conflict with AutoWatchID but test failed. If can fix test, I think it is better to start with 1.
I haven't found a place to set creq.WatchId yet, so it should always be mvcc.autowatchid?
There was a problem hiding this comment.
I think it's better to use -1 as an invalid watch ID for preserving the current behavior of watch API like what current PR does. How about introducing a new const like AutoWatchID?
Let me check the behavior introduced by this PR in a few days. I only tested a case which don't rely on it.
There was a problem hiding this comment.
I haven't found a place to set creq.WatchId yet, so it should always be mvcc.autowatchid?
Yes, it's basically true. In theory, users do not necessarily use the etcd client SDK. When they construct the wathRequest themselves, then watchId might not be 0 anymore. But the possibility should be very low, and it isn't the recommended way. So probably we can assume that it's always mvcc.AutoWatchID for any pb.WatchRequest_CreateRequest.
I think it's better to use -1 as an invalid watch ID for preserving the current behavior of watch API like what current PR does.
Based on point above (watchId always be mvcc.AutoWatchID for any pb.WatchRequest_CreateRequest), it makes sense to always use -1 as an invalid watch ID
There was a problem hiding this comment.
In theory, users do not necessarily use the etcd client SDK. When they construct the wathRequest themselves, then watchId might not be 0 anymore. But the possibility should be very low, and it isn't the recommended way.
Yeah using non standard SDK is technically possible... but I think it's good to exclude it because it's quite irregular as you point out.
There was a problem hiding this comment.
@mitake
I only found two places to return both Canceled and Created are true and can trigger the assert.
- https://github.com/etcd-io/etcd/blob/main/server/etcdserver/api/v3rpc/watch.go#L271
- https://github.com/etcd-io/etcd/blob/main/server/etcdserver/api/v3rpc/watch.go#L308
What do you think which cause assert triggerred? Or i missed something and can you help to supplement?
I agree with :
Passing
InvalidWatchIDto delete() or looking it up from ids isn't harmful.
Because it will not affect the normal watch.
Does c.CancelReason == "" equal to c.Canceled == false? I think it means the watch will not be deleted and the WatchId will be a normal watch id.
In my opinion, the purpose of this assert is that when a watch is newly created and an error is encountered ( both Canceled and Created are true), do not cause other normal watches to be deleted (WatchId MUST BE InvalidWatchID), it's hard to detect when it happens.
There was a problem hiding this comment.
@kafuu-chino In the case of TestKVCompaction, https://github.com/etcd-io/etcd/blob/main/server/etcdserver/api/v3rpc/watch.go#L308 is causing the assert. The condition is c.Created == true && c.Canceled == false && c.WatchId == 0 && c.CancelReason == "".
On the second thought, the condition should be like (c.Canceled || c.Created) && wid != mvcc.InvalidWatchID? If it's false, the branches https://github.com/etcd-io/etcd/blob/main/server/etcdserver/api/v3rpc/watch.go#L468-L472 will be executed and unintentional operations against invalid watch ID will happen (although it's not harmful).
It's great if you can try to run the test case too: TESTCASE=TestKVCompact TEST_OPTS="PASSES='integration'" make test
There was a problem hiding this comment.
@mitake
The assert condition really wrong and I've got the condition backwards.
If condition like (c.Canceled || c.Created) && wid != mvcc.InvalidWatchID, there will be problems here: https://github.com/etcd-io/etcd/blob/main/server/etcdserver/api/v3rpc/watch.go#L342
c.Created == falseandc.Canceled == false- then
(c.Canceled || c.Created) == false - and
(c.Canceled || c.Created) && wid != mvcc.InvalidWatchID== false condition == falseand assert will be trigger
How about !(c.Canceled && c.Created) || wid == mvcc.InvalidWatchID?
There was a problem hiding this comment.
@kafuu-chino Sorry the above my condition is very wrong... Especially if it includes && wid != mvcc.InvalidWatchID, it cannot handle an invalid wid always. I guess I pasted a wrong version :(
I think your condition is correct. Sorry for the confusion.
There was a problem hiding this comment.
Let me review the entire PR, I think I can reply sometime tomorrow.
|
@kafuu-chino Could you squash the 2 commits into one, and change the commit title like this? |
d109b1b to
e881993
Compare
|
@mitake Sure, I've squashed and rebase to the latest commit. |
| wr := &pb.WatchResponse{ | ||
| Header: sws.newResponseHeader(sws.watchStream.Rev()), | ||
| WatchId: creq.WatchId, | ||
| WatchId: -1, |
There was a problem hiding this comment.
I think
WatchResponsewithCreated && Canceledand valid ID is possible in other paths, e.g. https://github.com/etcd-io/etcd/blob/main/server/etcdserver/api/v3rpc/watch.go#L308
Could you update the assert condition for checkingCancelReason? @kafuu-chino
The Watch always returns -1 when there is an error. Please replace all the -1 with the const InvalidWatchID. cc @kafuu-chino
|
@kafuu-chino The change looks good to me. But please rebase this PR instead of merging main into your dev branch. |
|
@ahrtr |
d5d243a to
4cc222c
Compare
mitake
left a comment
There was a problem hiding this comment.
Overall LGTM, could you rebase on the latest main branch correctly? The commit 9fba81a comes from the PR #14377 so it shouldn't be included in this PR.
server/etcdserver/api/v3rpc/watch.go, especially serverWatchStream.recvLoop() is hard to test with unit tests. Do you think it's possible to add integration or e2e test?
|
@mitake
I need to read the tests package code first and it's take some time to think test code. |
6a652e3 to
3e9b0aa
Compare
|
@kafuu-chino I'm writing a test case in ff9c661 for #14322 |
There was a problem hiding this comment.
LGTM
Thank you @kafuu-chino
The only concern is that there is no new test case for the code change. But it's OK if @mitake or @kafuu-chino can address the test case in a separate PR.
Leave it to @mitake to approve & merge this PR.
|
@kafuu-chino @ahrtr I finalized the test for my PR in 94fd161 I think it can be extended for covering the change in this PR. Let me think about it. Probably I'll be able to share my update sometime early next week. |
|
@kafuu-chino On the second thought, I think this change is essentially not limited to auth token TTL. I wrote below integration test and checked that the test can fail in the main branch and run successfully with your branch. Could you check it's a suitable test case for your change? If it's ok, could you include it in this PR? (path is |
|
@mitake Ok, I'll try the test later. |
|
@mitake |
|
@kafuu-chino Could you rebase this PR based on the latest main? My PR introduced a conflict in the integration test file, sorry. |
|
@mitake |
mitake
left a comment
There was a problem hiding this comment.
LGTM, thanks a lot @kafuu-chino ! It's great if @ahrtr can cross check the test case.
| testutil.AssertNil(t, watchResponse.Err()) | ||
| } | ||
|
|
||
| func TestV3AuthWatchErrorAndWatchId0(t *testing.T) { |
There was a problem hiding this comment.
The test case is OK, but I do not any relationship between this case and the issue this PR is trying to fix.
There was a problem hiding this comment.
Actually auth token TTL expiration isn't the only one reason which can cause unexpected cancelling watch with ID=0. Other errors caused by isWatchPermitted() https://github.com/etcd-io/etcd/pull/14296/files#diff-79ca72cbf269ee36941406748b6078905b26447428c7179f70e543669cf42ec3R272 can result the same issue. So I used the different error auth.ErrPermissionDenied for checking the corrected behavior. By using this error, the test case can avoid sleeping for waiting auth token TTL and it's better for making it deterministically.
There was a problem hiding this comment.
Thanks @mitake for the explanation.
Actually auth token TTL expiration isn't the only one reason which can cause unexpected cancelling watch with ID=0.
I follow this issue intermittently, so I might lose some context. I agree with you on your point, but I don't think it verifies the scenario of the original issue #12385 ? My understand is that the client should can successfully Watch even when the auth token is expired, so we need to add a case to verify it. Please correct me if I am wrong.
There was a problem hiding this comment.
My understand is that the client should can successfully Watch even when the auth token is expired, so we need to add a case to verify it.
Correct, and this behavior is checked by the test case added in another PR: #14322 The test case is https://github.com/etcd-io/etcd/pull/14322/files#diff-c42aac07498140d89b902c1e02edb73076615172896ba431ec99319636432e1dR501 (TestV3AuthWatchAndTokenExpire). So the test case added in this PR (TestV3AuthWatchErrorAndWatchId0) focuses only on the issue happens to watch with ID=0. I think the 2 test cases are enough for covering the issues reported in #12385 (actually there are 2 related but independent issues). How do you think @ahrtr ?
There was a problem hiding this comment.
Thanks @mitake for the clarification, which makes sense.
Signed-off-by: Kafuu Chino <[email protected]> add test
Codecov Report✅ All modified and coverable lines are covered by tests. Additional details and impacted files@@ Coverage Diff @@
## main #14296 +/- ##
==========================================
- Coverage 75.35% 75.24% -0.11%
==========================================
Files 457 457
Lines 37190 37193 +3
==========================================
- Hits 28023 27986 -37
- Misses 7408 7442 +34
- Partials 1759 1765 +6
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. 🚀 New features to boost your workflow:
|
|
Overall look good to me. Thanks! |
|
@kafuu-chino Thanks a lot for working on this! Is it possible for you to open PRs which backport this change to 3.4 and 3.5 branches? |
|
@kafuu-chino please let me know if you don’t have time to backport the change to release-3.4 and 3.5. Probably I’ll be able to help. |
Sorry, I just got back from my vacation, I'll handle it today. |
etcdserver: Fixed unexpected cancelled watch with WatchID=0.
I ran into the same problem from #12385, I found that the error will cause the watch with
WatchID=0to be invalid,so I set WatchID to start at 1 to fix it.First of all
WatchID=0is semantic, it'sAutoWatchID.Secondly when token expired, make a new watch request to server side,
serverWatchStream.isWatchPermittedreturns false, then sendWatchResponsetoserverWatchStream.ctrlStreamwithCanceled=true, howeverWatchResponse.WatchID=creq.WatchId, defaultcreq.WatchId=0, so a watch withWatchID=0will be unexpectedly cancelled on the server side and client can never receive messages withWatchID=0.