-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
sql: distsql plans against unavailable node #23601
Comments
Good catch! Ping #21882 |
The decommissioned node shouldn't have any leases and for non-DistSQL requests, this would be discovered and the lease holder cache cleared. Perhaps DistSQL is failing to close the loop back to DistSender to tell it to eject from the lease holder cache and so it's stuck trying again and again. |
PS the title suggests that this is specific to decommissioning, but it seems that the bug is rather that DistSQL plans against unavailable nodes. It shouldn't do that in the first place; I don't think there's anything special that decommissioning the nodes will do. That just makes sure they cease to have leases and data, so that there is no reason to ever plan on them unless there's a bug like described in my last post. Consider s/decommissioning/unavailable/ in the title. |
Done, thanks. The error goes away if I force local execution via I just confirmed that this issue is not present in v2.0-alpha.20180129, so it's at least a somewhat recent regression. |
I wonder if the behavior change as of #22658 merely exposed a previously existing problem. It seems to me that a problem in which DistSQL plans on unavailable nodes would have existed both before and after that PR. |
I was thinking something similar. I wouldn't be surprised if DistSQL was already trying to plan on the unavailable node, but prior to that commit it would reliably figure out that it was unavailable and adjust the plan. I'll see if I can test out that hypothesis via tracing. |
Yep, this seems to be the case. Every time the query is run I see this in the trace:
So why is DistSQL trying to plan on an unavailable node which is no longer a leaseholder or replica for any ranges? Presumably some cache is never getting updated. |
@tschottdorf's first instinct looks correct: |
Summing up some findings on this. I think this issue comes down to the intersection of a few things:
The sum effect is that the DistSQL planner will try and plan against an unavailable node which is no longer a leaseholder. If its connection is in an error state, the planner will plan on a different node and the query will succeed. But if it's in the "not connected" state, the planner will go ahead and plan on it, which errors out on execution. How do we end up with a stale leaseholder cache? My theory is that DistSender needs to be smarter about updating the cache. Currently it only updates if it receives a cockroach/pkg/kv/dist_sender.go Lines 1339 to 1343 in 747d488
Receiving a successful response from the leaseholder has no effect on the cache, even if we thought someone else was the leaseholder. We rely on the non-leaseholders to report this info, and they may very well have stale caches too. So I think the action items here are:
|
Even if you fix this it does look like it's only going to make the execution path failure rarer. You still need to recover from the execution path seeing an unhealthy node. related issue: #15637 |
moving to 2.1 since this is not a regression |
Good point about the execution path. But I'd personally still consider this behavior a regression. When a node becomes unavailable, distsql queries which would previously have included that node start erroring out ~50% of the time. That's not the case in v1.1.5. |
I agree with @solongordon that this is a regression. It's not acceptable to require retries half of the time after you decommission a node, which is supposed to be an ordinary operational command. Fixing all 3 of these things probably can't happen by 2.0, though. Is there a simple fix that we can apply to prevent the user-visible issue, at least? |
A simple fix would be to change DistSQL not to plan against nodes whose RPC connection is in the |
I suppose one problem with that is that those connections wouldn't ever be resurrected in the case of a transient network failure, then, unless there's an external actor trying to resurrect them. |
Yeah, checking for connection health ( ISTM that the most fundamental badness here is that we're not evicting the lease holders. Fixing that would also fix many of these errors but while it's probably not technically hard, perhaps there's some refactoring involved. (iirc the leaseholder cache lives above the code that tries the other replicas). For the planning layer, |
CC @andreimatei |
My bad for not having a test for whatever happened here. But these tests are hard to write... |
Seconded. This is also a performance issue, as in #23543 |
When exactly do you suggest to do evictions from the leaseholders cache?
I'm not sure I follow this. If we manage to read from a range, either through the local engine of DistSQL, the correct leaseholder for that should make it to the cache. In the DistSQL case, the gateway gets info for all the mis-planned ranges (planned on a node other than the leaseholder). |
Hm, that's not the behavior I'm seeing. The leaseholder cache can remain stale even after successful reads for the range, both with local and distsql execution. I think that's related to the fact that |
If true that the cache remains stale, that's a bug I think. You're saying that the |
Yes, though I think it's a bit more specific than that. |
The recent PR cockroachdb#22658 introduced a regression in `(*rpcContext).ConnHealth` which caused DistSQL to continue planning on unavailable nodes for about an hour (`ttlNodeDescriptorGossip`) if the leaseholder cache happened to not be updated by other non-DistSQL requests. Instead, consult node liveness and avoid planning on dead nodes. This reduces the problem to a <10s window. The defunct `ConnHealth` mechanism still protects against planning in some of cases (supposedly due to a once-per-second reconnection policy) and is retained for that reason, with issue cockroachdb#23829 filed to decide its future. NB: I'm not putting a release note since this was introduced after 1.1. We released it in a beta, though, so it may be worth calling out there. Touches cockroachdb#23601. (Not fixing it because this issue should only close when there's a roachtest). Release note (bug fix): NB: this fixes a regression introduced in 2.0-beta, and not present in 1.1: Avoid planning DistSQL errors against unavailable nodes.
The recent PR cockroachdb#22658 introduced a regression in `(*rpcContext).ConnHealth` which caused DistSQL to continue planning on unavailable nodes for about an hour (`ttlNodeDescriptorGossip`) if the leaseholder cache happened to not be updated by other non-DistSQL requests. Instead, consult node liveness and avoid planning on dead nodes. This reduces the problem to a <10s window. The defunct `ConnHealth` mechanism still protects against planning in some of cases (supposedly due to a once-per-second reconnection policy) and is retained for that reason, with issue cockroachdb#23829 filed to decide its future. NB: I'm not putting a release note since this was introduced after 1.1. We released it in a beta, though, so it may be worth calling out there. Touches cockroachdb#23601. (Not fixing it because this issue should only close when there's a roachtest). Release note (bug fix): NB: this fixes a regression introduced in 2.0-beta, and not present in 1.1: Avoid planning DistSQL errors against unavailable nodes.
23916: backport-2.0: distsql: consult liveness during physical planning r=tschottdorf a=tschottdorf Backport 1/1 commits from #23834. /cc @cockroachdb/release --- The recent PR #22658 introduced a regression in `(*rpcContext).ConnHealth` which caused DistSQL to continue planning on unavailable nodes for about an hour (`ttlNodeDescriptorGossip`) if the leaseholder cache happened to not be updated by other non-DistSQL requests. Instead, consult node liveness and avoid planning on dead nodes. This reduces the problem to a <10s window. The defunct `ConnHealth` mechanism still protects against planning in some of cases (supposedly due to a once-per-second reconnection policy) and is retained for that reason, with issue #23829 filed to decide its future. NB: I'm not putting a release note since this was introduced after 1.1. We released it in a beta, though, so it may be worth calling out there. Touches #23601. (Not fixing it because this issue should only close when there's a roachtest). Release note (bug fix): NB: this fixes a regression introduced in 2.0-beta, and not present in 1.1: Avoid planning DistSQL errors against unavailable nodes.
Is this still an issue? Should we revisit after #23834? |
Looks like this is all set from a DistSQL perspective thanks to #23834. However it looks like the stale leaseholder cache fix (#23885) never got merged, which could cause unnecessary performance hits when a node goes down. Sending back to @tschottdorf in case he wants to wrap that up before closing this issue. |
This addresses a situation in which we would not evict a stale leaseholder for a long time. Consider the replica [s1,s2,s3] and s1 is down but is the cached leaseholder, while s2 is the actual lease holder. The RPC layer will try s1, get an RPC error, try s2 and succeed. Since there is no NotLeaseHolderError involved, the cache would not get updated, and so every request pays the overhead of trying s1 first. Fixes cockroachdb#23601. Release note: None
23885: kv: evict leaseholder on RPC error r=solongoron a=tschottdorf This addresses a situation in which we would not evict a stale leaseholder for a long time. Consider the replicas [s1,s2,s3] and s1 is down but is the cached leaseholder, while s2 is the actual lease holder. The RPC layer will try s1, get an RPC error, try s2 and succeed. Since there is no NotLeaseHolderError involved, the cache would not get updated, and so every request pays the overhead of trying s1 first. WIP because needs testing. Touches #23601. Release note (bug fix): Improve request routing during node outages. 28609: opt: Make additional perf improvements r=andy-kimball a=andy-kimball Make several more fixes: 1. Do not qualify column names in metadata, since that requires expensive string formatting up-front (also cleanup the factoring of this code, which had gotten messy). 2. Inline Metadata into Memo. 3. Inline logicalPropsBuilder into the Memo. Together, these changes improve KV perf from: ``` Phases/kv-read/OptBuild 18.4µs ± 1% ``` to: ``` Phases/kv-read/OptBuild 17.8µs ± 1% ``` 28661: storage: don't include RHS data in merge trigger r=bdarnell,tschottdorf a=benesch Now that we require ranges to be collocated during a merge and the RHS replicas to be up-to-date before the merge commits, we no longer need to include a snapshot of the RHS in the merge trigger. We know that the copy of the data that already exists in the local store is perfectly up-to-date. So, stop sending the data in the merge trigger. Release note: None 28689: sqlbase: avoid using SERIAL in system tables r=knz a=knz Needed for #28575. We'll soon want special behavior for SERIAL. We can't afford the definition of system tables to be subject to a discussion about what SERIAL means. So this patch ensures system tables don't use SERIAL. Release note: None Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com> Co-authored-by: Andrew Kimball <andyk@cockroachlabs.com> Co-authored-by: Nikhil Benesch <nikhil.benesch@gmail.com> Co-authored-by: Raphael 'kena' Poss <knz@cockroachlabs.com>
After stopping one node in a four node cluster, I noticed that distsql inconsistently continues to include that node in physical plans, which causes errors in query execution. This is reproducible on a local cluster.
Now if you repeatedly run
SELECT * FROM d.t
on node 1, about half the time it will give the correct result, but the other half it will throw an rpc error:And if you repeatedly request the distsql explain plan, about half the time it will incorrectly include the stopped node:
The behavior persists after decommissioning the stopped node.
From looking at session traces, it looks to me like distsql always tries to plan against the stopped node, but there is a race condition where sometimes it discovers that it is unhealthy while planning and adjusts, but sometimes it doesn't and fails upon execution.
The text was updated successfully, but these errors were encountered: