cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
29.88k stars 3.77k forks source link

raft: cancel commands in pending queue #3299

Closed bdarnell closed 8 years ago

bdarnell commented 8 years ago

Sometimes a command may be proposed to raft but become moot before it is forwarded to the leader and committed. This is especially true of leader lease requests, which have a short expiration and are sometimes retried frequently (#3296). We should be able to cancel these requests while they are in the queue so they don't get replayed when the node connects to the cluster.

Replica.proposeRaftCommand creates a context.Context with a deadline. We should pass this context through Store.ProposeRaftCommand and into multiraft, and then in the loops over g.pending skip anything with an expired Context.

xiang90 commented 8 years ago

We should be able to cancel these requests while they are in the queue so they don't get replayed when the node connects to the cluster.

I feel it is hard to cancel 100% of timeout requests. For example if there is a request that already goes into raft log but raft loses leader before it can commit, then we cannot really cancel and have to wait it commit or abort (potentially overwrite by the new leader with a higher term) until the cluster recovers.

bdarnell commented 8 years ago

Yeah, this doesn't have to be 100% accurate. We have some logic to re-propose commands that may have been dropped by raft during a leadership transition; this issue is just to stop re-proposing commands that will have no effect when applied because their internal deadline has expired.

xiang90 commented 8 years ago

Got it. Thanks!

tbg commented 8 years ago

Note also recent work in #4876 cleaned up the behavior a little bit, but this is still going to be mildly tricky (though doable). When looking at a cancelled context from within Raft, we need to check (*Replica).pendingCmds and only if there's no entry we can abandon it). The reason is that we must not leave a client waiting, and a client waits as long as there's an entry in pendingCmds (see (*Replica).tryAbandon). Just something to keep in mind.

bdarnell commented 8 years ago

@tschottdorf I think we're talking about two different things; this issue is about checking for cancelled contexts in reproprosePendingCmdsLocked for commands in pendingCmds.

bdarnell commented 8 years ago

Moving from beta to 1.0 now that #5032 has fixed the major problems here.

tamird commented 8 years ago

Visiting for triage.

This seems to be relatively well handled as of #4876, which caused the client (which listens on the context's Done() channel) to eagerly remove the pending command from (*Replica).pendingCmds.

The only thing that can be done here is to check ctx.Done() before calling (*Replica).proposeRaftCommand in (*Replica).addWriteCmd. For example, if (*Replica).beginCmds had a channel-based API, this would all compose nicely.

cc @nvanbenschoten @petermattis for ideas on how such an API can be achieved.

petermattis commented 8 years ago

@tamird I'm not sure what you're thinking of here, nor am I terribly familiar with this code. Mind expanding on what you're imagining?

tamird commented 8 years ago

(*Replica).addWriteCmd calls (*Replica).beginCmds, which blocks. If, instead of a blocking method, we had a channel, we could write:

select {
case (*Replica).beginCmds<-&ba:
case <-ctx.Done():
}

This would achieve the bail-before-proposing goal described in this issue.

petermattis commented 8 years ago

I don't see anything easy to do here. beginCmds blocks because of the CommandQueue API which uses a sync.WaitGroup. Your suggestion couldn't work as-is because beginCmds needs to know about the keys used by the request. We could have beginCmds return a channel which is closed (becomes readable) when the command is ready to run, but I think that would require beginCmds to create a goroutine.

tbg commented 8 years ago

The more trivial fix for the remainder of this issue is to do a non-blocking select on ctx.Done() before proposing the Raft command (in addWriteCmd after all the waiting). For the command queue, what's really lacking is transparency - it would be a nice gimmick to have the context cancel waiting in the command queue, but if that happens then what we'd really need is information about what it was waiting for, or generally how that structure as a whole is performing under the current load. It's not clear to me what the right "digest" of this information would be (and performance considerations also limit what we want to do). A start is log.Traceing the number in the WaitGroup, and perhaps exposing the number of items in the whole structure (or depth of the three, or whatever makes sense) via (*Replica).State().

BTW, if the hope with early cancellation inside of the command queue is to release commands waiting on ours early, I think that's not going to work - the command queue crucially uses the fact that a command always waits for its prerequisites, because that allows a newly inserted command to wait only on some, and not all, of its overlapping predecessors. So even if the command queue realized that a context had been cancelled, it couldn't do that much interesting stuff with it, I think.

tbg commented 8 years ago

Oh, but then there's also the argument that goroutines are cheap - context.Context uses them pretty carelessly (any context.WithTimeout has one attached to it). Not suggesting that we need to use them here (see above), but maybe we can actually afford one (more) of them per request.

nvanbenschoten commented 8 years ago

This is mostly just repeating what Toby said, but I don't think cancellation inside the command queue is easily possible because of the linear ordering we compose through WaitGroups between commands. Once cmdQ.add is called, we need to wait for the corresponding wg.Wait() before it is safe to end the command with r.endCmds.

That said, hypothetically if WaitGroups played nicely with channels, we could do something like the following inside of Replica.beginCmds to at least avoid blocking the caller when a context is cancelled:

r.mu.Lock()
r.mu.cmdQ.getWait(readOnly, &wg, spans...)
cmd = r.mu.cmdQ.add(readOnly, spans...)
r.mu.Unlock()
select {
  case <-wg.Wait(): // pretend this works for the sake of the example
  case <-ctx.Done():
    // Launch goroutine to continue waiting in the background
    go func() {
      wg.Wait()
      r.mu.cmdQ.remove(cmd)
    }()
    return nil
}
...

I'm not sure this is necessary though. Is it often that case that only a single command is cancelled? If not, then there's a good chance all prerequisite commands will hit the check from #4876 and shortly after the waiting will unblock. If this is the case (or even if it's not), then the best thing to do for now might just be to

do a non-blocking select on ctx.Done() before proposing the Raft command (in addWriteCmd after all the waiting)

tamird commented 8 years ago

Here's a prototype of the channel-based API, no extra goroutines necessary: https://github.com/cockroachdb/cockroach/compare/master...tamird:cmdq-chan

As everyone's pointed out, doing the select is currently unsafe, but perhaps that too can be solved.