etcd-io / etcd

Distributed reliable key-value store for the most critical data of a distributed system
https://etcd.io
Apache License 2.0
47.42k stars 9.73k forks source link

clientv3: Canceling `Watch()` doesn't send `&pb.WatchCancelRequest` #9416

Closed yudai closed 4 years ago

yudai commented 6 years ago

Summary

Canceling a context that has been used to invoke Watch() is supposed to stop subscribing events for the Watch(), meaning that canceling should release resources in the server side.

However, clientv3 doesn't send a &pb.WatchCancelRequest message to the server when a substream is canceled, which leads resource leak in the side until the parent watch stream itself is closed.

Issue

Tested with the master branch (00b84235df).

watchGrpcStream serves watch substreams in serveSubstream and accepts context canceling for them there as well (using ws.initReq.ctx). When ws.initReq.ctx is canceled, it eventually sends the ws to w.closingc, and the ws is handled by the watchGrpcStream.run(). Then it calls w.closeSubstream(ws) and the substream is removed from internal w.substreams. (the client side is fine, it releases the resources for substreams)

However, this canceling procedure for substreams doesn't send any cancel messages to the server. The server therefore doesn't know a substream is canceled by the client at this moment.

When the watched key of a substream is actively updated, an event to the key eventually resolve this inconsistency here.

On the other hand, when the key is not updated at all, the resources for the substream in the server side kept forever until the watch stream itself is canceled or the client is dropped.

This behavior can be an issue when there are long-running clients that dynamically watch/unwatch different keys that are not actively updated after unwatching.

Reproducing

You can reproduce the situation by the following code.

package main

import "github.com/coreos/etcd/clientv3"

import (
    "context"
    "fmt"
    "log"
    "time"
)

func main() {
    cli, err := clientv3.New(clientv3.Config{
        Endpoints:   []string{"localhost:2379"},
        DialTimeout: 5 * time.Second,
    })
    if err != nil {
        log.Fatal(err)
    }
    log.Printf("Start")

    cli.Watch(context.Background(), "/") // this watch keeps the stream alive

    num := 1000
    for i := 0; i < num; i++ {
        ctx, cancel := context.WithCancel(context.Background())
        key := fmt.Sprintf("/%d", i)
        cli.Watch(ctx, key)
        cancel()
    }

    log.Printf("Done")

    time.Sleep(time.Hour)
        // etcd_debugging_mvcc_watch_stream_total 1
        // etcd_debugging_mvcc_watcher_total 1001   <- expected to be 1
}

Proposal

I think we can simply send a cancel request when w.closingc receives a closing stream.

https://github.com/coreos/etcd/blob/release-3.2/clientv3/watch.go#L513

Adding something like this?

        case ws := <-w.closingc:
            cr := &pb.WatchRequest_CancelRequest{
                CancelRequest: &pb.WatchCancelRequest{
                    WatchId: ws.id,
                },
            }
            req := &pb.WatchRequest{RequestUnion: cr}
            wc.Send(req)
gyuho commented 6 years ago

I think this is a valid use case for multiplexed watch stream requests + gRPC proxy. Right now, only way to cancel mvcc watchers is closing the whole watch stream.

What do you think? @heyitsanthony @xiang90

heyitsanthony commented 6 years ago

@yudai that's how it works today. A server-side mechanism to LRU evict watches in the v3rpc layer might be easier/cover more cases than reworking the client's cancel path.

yudai commented 6 years ago

Thanks for the comments. Please correct me if I misunderstand something below.

I think it's fine and working as expected that a mvcc.WatchStream is kept running when there is a corresponding watch stream in the client side. We need to close the whole watch stream to close mvcc.WatchStream.

The issue I wanted to mention was that mvcc.WatchStream.Cancel(id WatchID) is not called properly when you cancel a context for a clientv3.Watch() call (which creates a substream). mvcc.WatchStream.Cancel(id WatchID) releases mvcc.watcher (I'm a bit confused. this seems a way to cancel mvcc watchers without closing the whole watch stream), but since we don't call it, instances of mvcc.watcher piles up and not going to be freed, until you close the master watch stream.

In the v3rpc layer, v3rpc.serverWatchStream calls this mvcc.WatchStream.Cancel(id WatchID) properly when it receives *pb.WatchRequest_CancelRequest(https://github.com/coreos/etcd/blob/master/etcdserver/api/v3rpc/watch.go#L265-L268).

I therefore think we already have a proper implementation for substream canceling in the server side, and the only missing part is in the client side. The client library simply doesn't send *pb.WatchRequest_CancelRequest when users cancel contexts.

I assume this issue could be critical for gRPC proxy as well. I actually realized this issue when I was processing watch requests in our private gRPC proxy. Since clients don't send substream cancel requests, our proxy cannot release its internal resources for watch substreams as long as watch streams are alive. I think that the same thing should happen for gRPC proxy.

The mvcc layer also keeps unnecessary mvcc.watchers and other resources for context-canceled watch substreams as long as the watch stream for them is alive, as you can see etcd_debugging_mvcc_watcher_total in the reproducing code above. If a client repeats calling clientv3.Watch() and canceling it (it uses only one watch stream and generates many substreams), this can happen in any environments.

In my opinion, it would be better to make the client send cancel requests when users cancel contexts. It's pretty confusing for users that canceling contexts is not always canceling watch. Sending canceling requests on context canceling would be more straight forward and complies better with the defined protocol.

yudai commented 6 years ago

Created reproduction code:

package main

import "github.com/coreos/etcd/clientv3"

import (
  "context"
  "log"
  "time"
)

func main() {
  cli, err := clientv3.New(clientv3.Config{
    Endpoints:   []string{"localhost:2379"},
    DialTimeout: 5 * time.Second,
  })
  if err != nil {
    log.Fatal(err)
  }
  log.Printf("Start")

  cli.Watch(context.Background(), "/")

  for {
    ctx, cancel := context.WithCancel(context.Background())
    cli.Watch(ctx, "/")
    cancel()
  }

  log.Printf("Done")
}

You can confirm etcd_debugging_mvcc_watcher_total keeps increasing and never decrease with this code (as long as you don't put something to /).

gyuho commented 6 years ago

@yudai @heyitsanthony I will run some benchmarks to measure the overhead of canceling each watcher on closing channel receive, from client-side. Or investigate if it can be easily handled in storage layer.

scotwells commented 5 years ago

We seem to be experiencing a similar issue with our current production setup running on etcd v3.2.24. The number of open watches on the server continues to climb until we hit between ~23k and ~32k open watchers. Once the number of etcd open watches flat lines, etcd will fail to send watches out to clients and we see a sharp rise in Slow Watchers.

While diving into the code, we noticed that the watch request on the etcd server was not closed until the client got an event from the server after the client context was canceled. As an attempted workaround, we configured the clients to use WithProgressNotify. This change seemed to only delay the pile up, instead of resolving the issue.

Has there been any progress on determining if there is significant overhead in canceling a watch request when the context gets canceled on the client side?

Code executed when the client context is canceled

https://github.com/etcd-io/etcd/blob/e06761ed79c0ddcc411e300de171ada63bb5bef2/clientv3/watch.go#L513-L520

Code executed when an event is received from the server after the context is closed

https://github.com/etcd-io/etcd/blob/e06761ed79c0ddcc411e300de171ada63bb5bef2/clientv3/watch.go#L489-L496

Number of Open Watches vs Slow Watchers on the server image

stale[bot] commented 4 years ago

This issue has been automatically marked as stale because it has not had recent activity. It will be closed after 21 days if no further activity occurs. Thank you for your contributions.

fcvarela commented 4 years ago

@xiang90 Any chance this is actively being worked on? This has caused us production issues several times before and we'll invest the time to fix it if no one else is working on it.

spzala commented 4 years ago

@xiang90 Any chance this is actively being worked on? This has caused us production issues several times before and we'll invest the time to fix it if no one else is working on it. Reopening the issue per your comment, thanks @fcvarela

jackkleeman commented 4 years ago

I will run some benchmarks to measure the overhead of canceling each watcher on closing channel receive, from client-side

Can there be an overhead? Once a watch is closed on a client, we will eventually send a cancel request when the next message arrives. So, there are two cases by cancelling proactively:

  1. We end up cancelling earlier, as we don't wait for a message. Arguably this saves resources.
  2. We cancel where we otherwise wouldn't have, as no message would ever come. This is the case that is causing @monzo and presumably others production issues, and I don't know if leaving streams open indefinitely should be considered a performance improvement over closing them when no longer required.

I'm working on a PR to do what others above have suggested.