-
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
distsqlrun: eliminate queuing in flowScheduler and tune max_running_flows #34229
Comments
Prior to this PR when SetupFlowRequests in excess of `max_running_flows` were received they would be queued for processing. This queueing was unbounded and had no notion of a "full" queue. While the queue was well intentioned and may have led to desirable behavior in bursty, short-lived workloads, in cases of high volumes of longer running flows the system would observe arbitrarily long queuing delays. These delays would ultimately result propagate to the user in the form of a timeout setting up the other side of the connection. This error was both delayed and difficult for customers to interpret. The immediate action is to remove the queuing behavior and increase the flow limit. It is worth noting that this new limit is still arbitrary and unjustified except to say that it's larger than the previous limit by a factor of 2. After this PR when the `max_running_flows` limit is hit, clients will receive a more meaningful error faster which hopefully will guide them to adjusting the setting when appropriate or scaling out. This commit could use a unit test to ensure that we do indeed see the newly introduced error and I intend to write one soon. The machinery around setting up a unit test seemed much more daunting than this change but I wanted to get this out just to start a discussion about whether this is the right thing to do and if not, what better approaches y'all might see. Fixes cockroachdb#34229. Release note: None
Prior to this PR when SetupFlowRequests in excess of `max_running_flows` were received they would be queued for processing. This queueing was unbounded and had no notion of a "full" queue. While the queue was well intentioned and may have led to desirable behavior in bursty, short-lived workloads, in cases of high volumes of longer running flows the system would observe arbitrarily long queuing delays. These delays would ultimately result propagate to the user in the form of a timeout setting up the other side of the connection. This error was both delayed and difficult for customers to interpret. The immediate action is to remove the queuing behavior and increase the flow limit. It is worth noting that this new limit is still arbitrary and unjustified except to say that it's larger than the previous limit by a factor of 2. After this PR when the `max_running_flows` limit is hit, clients will receive a more meaningful error faster which hopefully will guide them to adjusting the setting when appropriate or scaling out. This commit could use a unit test to ensure that we do indeed see the newly introduced error and I intend to write one soon. The machinery around setting up a unit test seemed much more daunting than this change but I wanted to get this out just to start a discussion about whether this is the right thing to do and if not, what better approaches y'all might see. Fixes cockroachdb#34229. Release note: None
@lunevalex |
@sumeerbhola we trying to clean up the KV Backlog to make it useful for planning. To help with that we declared a limited bankruptcy on issues that have not ben touched by anyone in over two years. All issues that this happened to are marked with X-stale and can be found here https://github.com/cockroachdb/cockroach/issues?q=+is%3Aissue+project%3Acockroachdb%2Fcockroach%2F22+label%3AX-stale+. I'll re-open this back as it looks like we may be looking into this as part of the admission control work. |
71787: flowinfra: make max_running_flows default depend on the number of CPUs r=yuzefovich a=yuzefovich We think that it makes sense to scale the default value for `max_running_flows` based on how beefy the machines are, so we make it a multiple of the number of available CPU cores. We do so in a backwards-compatible fashion by treating the positive values of `sql.distsql.max_running_flows` as absolute values (the previous meaning) and the negative values as multiples of the number of the CPUs. The choice of 128 as the default multiple is driven by the old default value of 500 and is such that if we have 4 CPUs, then we'll get the value of 512, pretty close to the old default. Informs: #34229. Release note (ops change): The meaning of `sql.distsql.max_running_flows` cluster setting has been extended so that when the value is negative, it would be multiplied by the number of CPUs on the node to get the maximum number of concurrent remote flows on the node. The default value is -128, meaning that on a 4 CPU machine we will have up to 512 concurrent remote DistSQL flows, but on a 8 CPU machine up to 1024. The previous default was 500. 71832: server: avoid range local scan for status/{ranges,raft} endpoint r=nvanbenschoten a=arulajmani See individual commits for details. Co-authored-by: Yahor Yuzefovich <[email protected]> Co-authored-by: arulajmani <[email protected]>
@nvanbenschoten had some questions about
|
Yes, admission control dynamically adjusts cpu slots (that gate both KV and SQL work), so requests that are stalled due to unavailability should not affect utilization of cpu (and IO, since storage write tokens are also dynamically adjusted). And yes, we don't have a separate queue per range.
Transactions that have begun acquiring locks will have requests assigned There are some deficiencies in admission control in relation to this issue:
|
Before doing archeology I wrote: If I had to guess, it came from a time with much less memory monitoring and a hope to avoid both OOMs and starvation which killed node liveness. When I then went and looked at the history, and it was even more speculative. It was introduced by @RaduBerinde in the same commit as the |
I think the time is ripe for removing this queueing in the flowScheduler altogether. (Note that the whole discussion about the flowScheduler is only relevant to the remote DistSQL flows - all flows that are scheduled on the gateway of distributed plans as well as all non-distributed plans don't go through the flow scheduler.) My reasoning is that we now have an actual admission control in place, so this queueing - which served as a form of admission control previously - is no longer needed. I've reviewed most of the issues linked to this one, and I believe this comment from Peter has the succinct explanation for why we had this queueing:
The actual admission control now directly (mostly) takes care of
Sumeer also pointed out some deficiencies in admission control in relation to this issue:
It doesn't seem like a blocker to me for removing this flow scheduler queueing since already implemented admission control queues do kick in.
Neither the admission control nor the queueing in the flow scheduler prevent the nodes from OOMing, however, over the years we have become much better with the memory accounting which serves as the pushback on the RAM usage when exceeding In 22.1 (as well as in a backport to 21.2) we already increased the default value of In order to see how CRDB would behave when overloaded with remote DistSQL flows I ran Here is what I got with the flow scheduler's queueing in place:
and with that queueing eliminated:
As we can see the overall throughput increased at the expense of increase in the tail latencies while reducing the average latency. I've done similar experiments with concurrencies of 4096 and 8192 and obtained similar results. Here are some of the relevant metrics of the runs with 16384 concurrency: To me the numbers and the graphs show that the admission control does its job well enough for us to remove the queueing in the flow scheduler, but I'm curious to hear from others. Thoughts? cc @sumeerbhola @ajwerner @irfansharif cc @cockroachdb/sql-queries |
84925: vendor: bump Pebble to 89b7bc729bd1 r=nicktrav a=jbowens ``` 89b7bc72 db: lazily construct combined iterator 4adbba57 internal/base: add SeekLTFlags 97fbe8e2 internal/base: replace trySeekUsingNext with SeekGEFlags 63d55279 lint: use `go run path@version` syntax for lint binaries ``` Release note: None 84932: distsql: reserve 10KiB memory on admission of remote flow r=yuzefovich a=yuzefovich This commit makes it so that we reserve 10KiB of memory when scheduling each remote DistSQL flow. The idea is that a read-only remote flow (since we don't distribute writes) will need to account for some memory anyway, so we might as well make a small reservation upfront. This would serve as a form of admission control "based" on the RAM usage. Informs: #34229. Release note: None 84944: dev: v45 r=irfansharif a=sjbarag ./dev's version wasn't properly incremented in PR 84766 [1] due to a last-minute rebase, in which someone else increased the version as well. Bump dev's version to force a rebuild. [1] #84766 Release note: None Co-authored-by: Jackson Owens <[email protected]> Co-authored-by: Yahor Yuzefovich <[email protected]> Co-authored-by: Sean Barag <[email protected]>
Looks fine to me. It is interesting that the latter half of the run where there is no |
We decided to take a conservative approach with removing this queueing behavior. In 22.2 we will introduce a cluster setting that controls whether the queueing is enabled or not (disabled by default) so that we have an escape hatch in case we missed some cases in this discussion, and then in 23.1 we'll remove the queueing altogether. Also, #84932 has already added a small memory reservation whenever a new remote DistSQL flow is being scheduled which improves our stance on the RAM usage stability. |
84865: kvserver: always return NLHE on lease acquisition timeouts r=nvanbenschoten a=erikgrinaker In ab74b97 we added internal timeouts for lease acquisitions. These were wrapped in `RunWithTimeout()`, as mandated for context timeouts. However, this would mask the returned `NotLeaseHolderError` as a `TimeoutError`, preventing the DistSender from retrying it and instead propagating it out to the client. Additionally, context cancellation errors from the actual RPC call were never wrapped as a `NotLeaseHolderError` in the first place. This ended up only happening in a very specific scenario where the outer timeout added to the client context did not trigger, but the inner timeout for the coalesced request context did trigger while the lease request was in flight. Accidentally, the outer `RunWithTimeout()` call did not return the `roachpb.Error` from the closure but instead passed it via a captured variable, bypassing the error wrapping. This patch replaces the `RunWithTimeout()` calls with regular `context.WithTimeout()` calls to avoid the error wrapping, and returns a `NotLeaseHolderError` from `requestLease()` if the RPC request fails and the context was cancelled (presumably causing the error). Another option would be to extract an NLHE from the error chain, but this would require correct propagation of the structured error chain across RPC boundaries, so out of an abundance of caution and with an eye towards backports, we instead choose to return a bare `NotLeaseHolderError`. The empty lease in the returned error prevents the DistSender from updating its caches on context cancellation. Resolves #84258. Resolves #85115. Release note (bug fix): Fixed a bug where clients could sometimes receive errors due to lease acquisition timeouts of the form `operation "storage.pendingLeaseRequest: requesting lease" timed out after 6s`. 84946: distsql: make the number of DistSQL runners dynamic r=yuzefovich a=yuzefovich **distsql: make the number of DistSQL runners dynamic** This commit improves the infrastructure around a pool of "DistSQL runners" that are used for issuing SetupFlow RPCs in parallel. Previously, we had a hard-coded number of 16 goroutines which was probably insufficient in many cases. This commit makes it so that we use the default value of `4 x N(cpus)` to make it proportional to how beefy the node is (under the expectation that the larger the node is, the more distributed queries it will be handling). The choice of the four as the multiple was made so that we get the previous default on machines with 4 CPUs. Additionally, this commit introduces a mechanism to dynamically adjust the number of runners based on a cluster setting. Whenever the setting is reduced, some of the workers are stopped, if the setting is increased, then new workers are spun up accordingly. This coordinator listens on two channels: one about the server quescing, and another about the new target pool size. Whenever a new target size is received, the coordinator will spin up / shut down one worker at a time until that target size is achieved. The worker, however, doesn't access the server quescing channel and, instead, relies on the coordinator to tell it to exit (either by closing the channel when quescing or sending a single message when the target size is decreased). Fixes: #84459. Release note: None **distsql: change the flow setup code a bit** Previously, when setting up a distributed plan, we would wait for all SetupFlow RPCs to come back before setting up the flow on the gateway. Most likely (in the happy scenario) all those RPCs would be successful, so we can parallelize the happy path a bit by setting up the local flow while the RPCs are in-flight which is what this commit does. This seems especially beneficial given the change in the previous commit to increase the number of DistSQL runners for beefy machines - we are now more likely to issue SetupFlow RPCs asynchronously. Release note: None 85091: flowinfra: disable queueing mechanism of the flow scheduler by default r=yuzefovich a=yuzefovich This commit disables the queueing mechanism of the flow scheduler as part of the effort to remove that queueing altogether during 23.1 release cycle. To get there though we choose a conservative approach of introducing a cluster setting that determines whether the queueing is enabled or not, and if it is disabled, then we effectively a treating `sql.distsql.max_running_flows` limit as infinite. By default, the queueing is now disabled since recent experiments have shown that the admission control does a good job of protecting the nodes from the influx of remote flows. Addresses: #34229. Release note: None 85134: sql: allow NULL in create view definition r=mgartner a=rafiss fixes #84000 Release note (sql change): CREATE VIEW statements can now have a constant NULL column definition. The resulting column is of type TEXT. 85178: kvserver: record batch requests with no gateway r=kvoli a=kvoli Previously, batch requests with no `GatewayNodeID` would not be accounted for on the QPS of a replica. By extension, the store QPS would also not aggregate this missing QPS over replicas it holds. This patch introduces tracking for all requests, regardless of the `GatewayNodeID`. This was done to as follow the workload lease transfers consider the per-locality counts, therefore untagged localities were not useful. This has since been updated to ignore filter out localities directly, so it is not necessary to exclude them anymore. `leaseholderStats`, which previously tracked the QPS, and `writeStats` tracking the mvcc keys written, have also been removed. They are duplicated in `batchRequest` and `writeKeys` respectively, within the `loadStats` of a replica. resolves #85157 Release note: None 85355: sql: improve physical planning of window functions r=yuzefovich a=yuzefovich **sql: remove shouldNotDistribute recommendation** It doesn't seem to be used much. Release note: None **sql: improve physical planning of window functions** This commit improves the physical planning of window functions in several ways. First, the optimizer is updated so that all window functions with a PARTITION BY clause are constructed first followed by the remaining window functions without PARTITION BY. This is needed by the execution which can only evaluate functions with PARTITION BY in the distributed fashion - as a result of this change, we are now more likely to get partial distributed execution (previously things depended on the order in which window functions were mentioned in the query). Second, the physical planner now thinks that we "should distribute" the plan if it finds at least one window function with PARTITION BY clause. Previously, we didn't make any recommendation about the distribution based on the presence of the window functions (i.e. we relied on the rest of the plan to do so), but they can be quite computation-intensive, so whenever we can distribute the execution, we should do so. Additionally, this commit removes some of the code in the physical planner which tries to find window functions with the same PARTITION BY and ORDER BY clauses - that code has been redundant for long time given that the optimizer does that too. Release note: None 85366: sql,logictest,descidgen: abstract descriptor ID generation, make deterministic in logictests r=ajwerner a=ajwerner The first commit adds an interface for descriptor ID generation and propagates the interface from the ExecCfg into the EvalContext. There are some minor refactoring to avoid propagating an ExecCfg further up the stack by making the parameters more specific. The second commit adds a testing knob to use a transactional implementation in the EvalContext. Fixes #37751 Fixes #69226 85406: schemachanger: check explain diagrams during rollback test r=postamar a=postamar This commit enriches the declarative schema changer integration tests by making data-driven EXPLAIN output assertions easier to add as a complement to otherwise unrelated tests. In particular, this commit improves the rollback test to check the explained rollback plan for each post-commit revertible stage. This should make it easier to debug bad rule definitions which otherwise would manifest themselves as causing the schema change to hang during the rollback. Release note: None 85414: colflow: fix a recent flake r=yuzefovich a=yuzefovich In 0866ddc we merged a change that relied on the assumption that the allocator passed to the parallel unordered synchronizer was not used by anyone else, but this assumption was broken in a test and is now fixed. Fixes: #85360. Release note: None Co-authored-by: Erik Grinaker <[email protected]> Co-authored-by: Yahor Yuzefovich <[email protected]> Co-authored-by: Rafi Shamim <[email protected]> Co-authored-by: Austen McClernon <[email protected]> Co-authored-by: Andrew Werner <[email protected]> Co-authored-by: Marius Posta <[email protected]>
Describe the problem
In the face of a large number of flows today's distsqlrun.flowSchedule begins queuing requests for processing. This queueing leads to delays in processing incoming flow requests. At a certain point these requests will be delayed long enough that the upstream node expecting a connection will time out (default 10s) which results in the client receiving an inscrutable
no inboud stream connection
error. See #27746. While some amount of queuing might be acceptable, today there is no limit on queuing or on how long requests might be delayed. Rather than work through exactly what the queuing behavior ought to be, this issue suggests we should reject incoming requests which today might be queued. This will lead to faster failure with a more reasonable error.To Reproduce
Run a workload which creates a large number of distsql flows.
Expected behavior
The client should receive an error related to resource exhaustion.
Additional context
Wrapped up in this issue is the question of what is the right number of max_running_flows before rejection occurs. Coming up with a more sophisticated acceptance control mechanism is out of scope for this issue but it is important that this change not break existing workloads.
Add any other context about the problem here.
Jira issue: CRDB-4659
The text was updated successfully, but these errors were encountered: