-
Notifications
You must be signed in to change notification settings - Fork 3.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
sql: remote flows do not cancel promptly upon query cancellation #64916
Comments
The problem appears to be related to the problems we were looking at in ca6457e again. What happens in this situation is that flows are promptly cancelled via the "outbox watchdog" goroutine (the one in But, we do not cancel the flow's context when we get With the following tiny patch, the graph is fixed:
|
Here's another graph with 1024 concurrency, the last one was accidentally 512 concurrency. So I think this patch is an improvement, but it's interesting that we still have to dequeue flows just to set them up, call |
We definitely should find a way to do fast cancellation. Look what happens when the gateway completely swamps the cluster with a more heavy workload for a longer time (this is tpch query 1 with high concurrency). The queue grows and grows until the load is turned off. At that point, the system has to dequeue a flow, send a FlowStream RPC, and wait for either a I think we need some way of detecting flow cancellation actively. Imagine an RPC from the gateway that informs remote notes that flows xyz are cancelled. The remote nodes then can either filter the queue to remove dead flows, or at worst, keep the dead flow IDs in a map to use when flows are dequeued, so they can at least be fast-path finished. |
Yeah, something like this makes sense to me. We definitely need the participation of the gateway node (the one that called |
64940: colrpc: shut down outbox tree on a graceful termination of a stream r=yuzefovich a=jordanlewis Previously, if a query were gracefully cancelled that had related remote flows, those remote flows would not be actively cancelled. They would cancel themselves only when they next tried to write to the network and got an io.EOF. Now, the outbox "watchdog goroutine" will cancel the tree rooted in the outbox actively once it sees an io.EOF. Note that such behavior is reasonable because the inbox stream handler must have exited, so we don't need to concern ourselves with doing any other work; however, the flow context is not canceled since other trees planned on the same node might still be doing useful work on behalf of other streams. Addresses #64916. Release note (sql change): improve cancellation behavior for DistSQL flows. Co-authored-by: Jordan Lewis <[email protected]>
I have thought about implementing this eager cancellation mechanism of the scheduled flows. We will need to introduce another RPC to the DistSQL, and I think we can go two ways:
The first option is a lot simpler to implement, but it would obviously have higher overhead if multiple queries are canceled at once. I think I'll try implementing the second option. The complication there is splitting up the cancelFlowRequests on the gateway between different remote nodes and batching the flowIDs together. Another question for the second option is when do we actually perform the cancellation? I think I'll try an approach with limited number of "canceling worker" goroutines, each issue an RPC against a single node with all flowIDs for that node accumulated so far. Another question is when and how do we decide to try canceling a scheduled flow on the remote node. One alternative is to track which remote flows initiated FlowStream RPC and cancel those that haven't. (If the remote node initiated |
@jordanlewis I'm curious what were the parameters of the setup where you observed 13k flows queued up? I tried to use 3 node roachprod cluster with fourth server issuing the queries, imported TPCH data sets of scale factors of 10 and 1, and with concurrency=1024 of query 1, the cluster OOMs. |
I guess I'll just use machines with larger RAM, not the default ones. |
@yuzefovich I used a default setup, with I used 2 test experiments, one with query 1 and one with query 18. The experiment with the nice pyramid graph above was with query 1. One thing I'll note is that I did have a patch on the database that was not default that I assumed wasn't doing anything, but perhaps it did improve OOM behavior somewhat: #64906 (comment) Once I applied that patch and this cancellation patch, I was unable to easily OOM the database with query 1 concurrency=1024. But it still OOMed with query 18. |
Hm, that patch does seem to improve things, but I still get OOMs before the queueing of flows occurs (without the patch it is OOMing sooner). It's possible that my WIP on cancellation is to blame, but for now I'll just use machines with more RAM. |
I'm having hard time trying to manually reproduce the same scenario even after going to machines with 26GB of RAM and lower |
Ok, it is my patch to blame - can repro easily on master with the fetcher patch but without my WIP. |
Setup:
workload init tpch --data-loader=import
)select count(*) from tpch.lineitem
) at concurrency 1024 (workload run querybench --query-file=queries --tolerate-errors --verbose
)Observe custom chart of queued and active flows:
In this chart, the load was cancelled between 14:53 and 14:54, but it took over 5 minutes for the cluster to stop processing remote flows.
I would expect that when the load is cancelled, the remote nodes should be able to drop their flows on the floor very quickly.
The text was updated successfully, but these errors were encountered: