Skip to content
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

Move CPU Bound Tasks off Tokio Threadpool #13692

Open
tustvold opened this issue Dec 8, 2024 · 24 comments
Open

Move CPU Bound Tasks off Tokio Threadpool #13692

tustvold opened this issue Dec 8, 2024 · 24 comments
Labels
enhancement New feature or request

Comments

@tustvold
Copy link
Contributor

tustvold commented Dec 8, 2024

Is your feature request related to a problem or challenge?

DataFusion currently performs CPU-bound work on the tokio runtime and this can lead to issues where IO tasks are starved and unable to adequately service the IO. Crucially this occurs long before the actual resources of the threadpool are exhausted, it is expected that if there is no CPU available that IO will be starved, what we want to avoid is IO getting throttled when there is resource available.

The accepted wisdom has been to run DataFusion in a separate threadpool and spawn IO off into a separate threadpool, as described here. #12393 seeks to document this, and by extension #13424 and #13690 to provide examples on how to achieve this.

Unfortunately this approach comes with a lot of both explicit and implicit complexity, and makes it hard to integrate DataFusion into existing codebases making use of tokio.

Describe the solution you'd like

The basic idea would be, rather than move the IO off to a separate runtime away from the CPU bound tasks, instead wrap the CPU bound tasks so that they can't starve the runtime.

Ultimately DF has three broad class of task:

  1. Fully synchronous code - e.g. optimisation, function evaluation, planning, most physical operators
  2. Mixed IO and CPU-bound - e.g. file IO, schema inference, etc...
  3. Pure IO - catalog

The vast majority of code in DF falls into the fully synchronous bucket (1.), and so it is a natural response to want to special case the less common IO (2. / 3.) instead of the more common CPU-bound code. However, I'd like to posit that conceptually and practically it is much simpler to dispatch the synchronous code than the asynchronous code, and that the items falling into the second bucket are the most complex operators in DataFusion.

This would yield some quite compelling advantages:

  • DataFusion could easily be integrated with existing tokio-based workloads
  • Avoids the many footguns of CPU-bound async tasks

This could be achieved using the native primitives

One way to achieve this would be to use block_in_place, as unlike spawn_blocking it does not require a 'static lifetime and can therefore borrow data from the invoking context.

That being said this does come with some risks:

  • It might become hard to determine what is sufficiently CPU-bound to warrant spawning
  • There would potentially be a lot of callsites that would need wrapping
  • It would be important to use block_in_place only at the granularity of a data batch, to ensure overheads are amortized
  • Applications relying on thread count to limit concurrency might need to put in place some additional concurrency controls (e.g. a semaphore)

An arguably better solution would be to dispatch the work using an actual CPU-threadpool such as rayon.

Describe alternatives you've considered

We could instead move IO off to a separate runtime

Additional context

No response

@adriangb
Copy link
Contributor

adriangb commented Dec 9, 2024

I considered something like this when looking into how to solve the problem and I think the sticking point is:

It might become hard to determine what is sufficiently CPU-bound to warrant spawning

It's relatively easy to, at a low level, know where IO calls are happening. In fact Tokio lets you configure the runtime to panic if you do IO at all (which makes it easy to be almost completely certain you didn't miss a spot).

@tustvold
Copy link
Contributor Author

tustvold commented Dec 9, 2024

Yeah, there are tools like tokio-console and the runtime metrics that could help to find problematic codepaths, but I'm not going to pretend it is trivial. I do think it may end up being simpler, and potentially a better long term solution, than dispatching the async work.

@crepererum
Copy link
Contributor

Couple of thoughts:

  • runtime requirements: block_in_place requires the multi-thread runtime. That's not a blocker, but we should clearly communicate that this means that DF can no longer run under the "same thread" runtime.
  • performance: Some experiments that I did suggested that block_in_place scales significantly worse that two runtimes (one for CPU work, one for IO). However this may not be the general case. I just had the impression that block_in_place is not particularity well optimized and in fact it requires the tokio scheduler to do some rather nasty thread switching.

I'm wondering if we should rather write our own async scheduler for the async compute graph that can better deal with the DF workload. For IO, we still can (and should) use tokio, but I question if shoehorning tokio into the CPU-bound workload is really worth it. And FWIW: such a scheduler can -- at least to some extend -- be pull OR push-based (requires some wiring, but not impossible).

@tustvold
Copy link
Contributor Author

tustvold commented Dec 9, 2024

The performance implications definitely concern me, I have a nagging suspicion block_in_place spawns a thread...

An arguably better solution would be to instead spawn the CPU bound work to a threadpool like rayon, but this would require any arguments to be 'static. However, this may not be as much of an issue for sync code as it is for async.... I'll update the ticket... Such a solution would have a number of compelling properties

I'm wondering if we should rather write our own async scheduler for the async compute graph that can better deal with the DF workload

This sounds a lot like #2199, I am not entirely sure such an approach is feasible at this point.

@alamb
Copy link
Contributor

alamb commented Dec 9, 2024

It's relatively easy to, at a low level, know where IO calls are happening. In fact Tokio lets you configure the runtime to panic if you do IO at all (which makes it easy to be almost completely certain you didn't miss a spot)

Yes, agree broadly speaking with @adriangb

Almost all of DataFusion's code is CPU bound and as @tustvold and others have observed before the locations where IO is done is (relatively) isolated in comparison.

I can try to take another shot at #13690 to more fully annotate IO in DataFusion.

@tustvold
Copy link
Contributor Author

tustvold commented Dec 9, 2024

I can try to take another shot at #13690 to more fully annotate IO in DataFusion.

IMO the key challenge is async is an abstraction designed for IO concurrency, not CPU parallelism. This is what ultimately makes it so difficult to disentangle in a coherent way. Whilst we can dispatch the "IO" off, the boundary inevitably ends up fuzzy, an async task doesn't have a single start and end, inevitably leading to complex shenanigans around lifetimes, buffering, etc...

I'm increasingly of the opinion that having CPU-bound futures fundamentally breaks their abstraction, and rather than fighting this, DF would be better served by avoiding such things in the first place. Instead it can use async / tokio for what it is designed for, and move the CPU bound work somewhere else. I don't actually think this would be that bad, most kernels are synchronous and could be offloaded without too much ceremony. It would also be largely mechanical, and something that could easily be actioned by a broad contributor base.

I'd be happy to write up an example of what this might look like if there is interest in taking such a direction

Edit: It is also potentially worth highlighting that this would naturally open the door to parallelism beyond that afforded by the plan, i.e. morsel driven parallelism.

@tustvold tustvold changed the title Use Tokio::block_in_place Move CPU Bound Tasks off Tokio Threadpool Dec 9, 2024
@alamb
Copy link
Contributor

alamb commented Dec 9, 2024

I'm increasingly of the opinion that having CPU-bound futures fundamentally breaks their abstraction, and rather than fighting this, DF would be better served by avoiding such things in the first place.

LOL I feel like this is not a new opinion (e.g. #2199) I agreed to disagree with you on this 😆

I realize the current use of async tasks is non ideal, but it seems to be working just fine for performance.

So, attempting to be pragmatic, I am going to spend my effort improving the current situation rather than trying to undertake a larger refactoring, unless there is something that is not possible (however non ideal) in the current design.

@tustvold
Copy link
Contributor Author

tustvold commented Dec 9, 2024

I realize the current use of async tasks is non ideal, but it seems to be working just fine for performance.

Yes the current model works perfectly fine when there is little to no IO and sufficient plan parallelism to exploit, I don't dispute this.

There is likely more going on here, but I can't help notice DF doesn't perform nearly as well on "cold" data

LOL I feel like this is not a new opinion (e.g. #2199) I agreed to disagree with you on this 😆

Sure I have made no secret of disliking DF's use of async, and that we have disagreed on this in the past, however, I guess my hope is that we might have reached critical mass of pain to actually do something about it. This is no longer me just moaning about a theoretical issue.

Ultimately moving the CPU bound work off tokio isn't that complex, and the DF community has successfully pulled off far more complex initiatives (e.g. StringView). Doing so would simplify integrating DF into existing codebases, eliminate whole classes of footgun, and allow for greater parallelism than is currently possible.

Edit:

unless there is something that is not possible (however non ideal) in the current design

I don't think there is a good story for intermixing IO with DF, and there is plenty of empirical evidence to this effect. We at Influx were aware of the need to run separate threadpools, and yet we still lost months to this class of issue. Blessing an approach we empirically know caused significant pain, in spite of significant investment to avoid the issue, seems unfortunate.

@crepererum
Copy link
Contributor

I realize the current use of async tasks is non ideal, but it seems to be working just fine for performance.

It works in an artificial benchmark that likely runs on an oversized machine. So like most benchmarks, it's a rather biased PoV.

Yes the current model works perfectly fine when there is little to no IO and sufficient plan parallelism to exploit, I don't dispute this.

💯

@alamb
Copy link
Contributor

alamb commented Dec 9, 2024

Ultimately moving the CPU bound work off tokio isn't that complex, and the DF community has successfully pulled off far more complex initiatives (e.g. StringView).

This is the core challenge in my mind. I would love to believe this project isn't that complex, but I have yet to see someone show how this would work practically with DataFusion (as well as trying myself). I am more than willing to coordinate this work once we have a pattern to apply.

Doing so would simplify integrating DF into existing codebases, eliminate whole classes of footgun, and allow for greater parallelism than is currently possible.

I agree with the first two points (more parallelism I am not sure). I would very much like this to happen and am willing to invest a significant amount of effort to to do. I just don't know how to do it.

@tustvold
Copy link
Contributor Author

tustvold commented Dec 9, 2024

I think moving to a push-based execution model as #2199 attempted is likely intractable at this stage. However, I think there is a compromise where just the kernels are dispatched, I can probably try to find some time to write up something more concrete in the next few days if there is interest

@djanderson
Copy link
Contributor

For my part, I'm going to spend the next few hours seeing if I can boil down an initial reproducer per the ask here. Hopefully it will help to show concretely where the current architecture falls over and test prototype approaches against.

@tustvold
Copy link
Contributor Author

tustvold commented Dec 9, 2024

That'd be very useful, Influx also had a reproducer but it uses customer data and I no longer have access to it having left, but Marco or Andrew can probably run it as well. Adding blocking sleeps at random points can be one way to make the problem more obvious

@alamb
Copy link
Contributor

alamb commented Dec 9, 2024

For my part, I'm going to spend the next few hours seeing if I can boil down an initial reproducer per the ask here. Hopefully it will help to show concretely where the current architecture falls over and test prototype approaches against.

This would be super helpful. There are many open source datasets on https://huggingface.co/datasets -- maybe we can find some suitably large ones to run queries against, and throttle the network bandwidth somehow (blocking IO sleeps as in an ObjectStore wrapper, as @tustvold suggests)

@tustvold
Copy link
Contributor Author

I managed to find my toy experiment from when I was investigating this at Influx, that I think quite nicely demonstrates the issue. I've pushed it up here - https://github.com/tustvold/io_stall

The key thing is that even with extremely low concurrency, well below the number of threads available to tokio, the presence of blocking tasks on the same runtime starts to negatively impact IO throughput

@tustvold
Copy link
Contributor Author

tustvold commented Dec 14, 2024

I've pushed a simple example to io_stall that glues together rayon and async_task to yield an async scheduler that is able to accommodate CPU bound tasks whilst not starving IO.

cargo run --release --bin tokio -- --cpu-duration 1s --concurrency 7
    Finished `release` profile [optimized] target(s) in 0.05s
     Running `target/release/tokio --cpu-duration 1s --concurrency 7`
Average duration of 1002 ms (IO 2 ms) over 1 samples, throughput 0.9978257 rps
Average duration of 1001 ms (IO 1 ms) over 7 samples, throughput 6.9909644 rps
Average duration of 1001 ms (IO 1 ms) over 7 samples, throughput 6.9897003 rps
Average duration of 1167 ms (IO 167 ms) over 6 samples, throughput 5.9959545 rps
Average duration of 1000 ms (IO 0 ms) over 6 samples, throughput 5.991002 rps
Average duration of 1334 ms (IO 334 ms) over 6 samples, throughput 5.993606 rps
Average duration of 1167 ms (IO 167 ms) over 6 samples, throughput 5.996025 rps
Average duration of 1167 ms (IO 167 ms) over 6 samples, throughput 5.994514 rps
Average duration of 1167 ms (IO 167 ms) over 6 samples, throughput 5.9959745 rps
Average duration of 1167 ms (IO 166 ms) over 6 samples, throughput 5.99648 rps
Average duration of 1167 ms (IO 167 ms) over 6 samples, throughput 5.9940777 rps
Average duration of 1200 ms (IO 200 ms) over 5 samples, throughput 4.996572 rps
Average duration of 1400 ms (IO 400 ms) over 5 samples, throughput 4.996495 rps
Average duration of 1400 ms (IO 400 ms) over 5 samples, throughput 4.99691 rps
Average duration of 1400 ms (IO 400 ms) over 5 samples, throughput 4.996111 rps
Average duration of 1400 ms (IO 400 ms) over 5 samples, throughput 4.99447 rps
Average duration of 1500 ms (IO 500 ms) over 4 samples, throughput 3.9966822 rps
Average duration of 1333 ms (IO 333 ms) over 6 samples, throughput 5.9941373 rps
Average duration of 1334 ms (IO 333 ms) over 6 samples, throughput 5.996268 rps
Average duration of 1167 ms (IO 167 ms) over 6 samples, throughput 5.9957566 rps
Average duration of 1166 ms (IO 166 ms) over 6 samples, throughput 5.9938416 rps
Average duration of 1167 ms (IO 166 ms) over 6 samples, throughput 5.9940543 rps

vs

cargo run --release --bin rayon -- --cpu-duration 1s --concurrency 7
   Compiling io_stall v0.1.0 (/home/raphael/repos/scratch/io_stall)
    Finished `release` profile [optimized] target(s) in 0.45s
     Running `target/release/rayon --cpu-duration 1s --concurrency 7`
Average duration of 1002 ms (IO 2 ms) over 1 samples, throughput 0.9976903 rps
Average duration of 1002 ms (IO 2 ms) over 7 samples, throughput 6.994286 rps
Average duration of 1000 ms (IO 0 ms) over 7 samples, throughput 6.9929976 rps
Average duration of 1000 ms (IO 0 ms) over 7 samples, throughput 6.9927454 rps
Average duration of 1000 ms (IO 0 ms) over 7 samples, throughput 6.994525 rps
Average duration of 1000 ms (IO 0 ms) over 7 samples, throughput 6.993674 rps
Average duration of 1000 ms (IO 0 ms) over 7 samples, throughput 6.993697 rps
Average duration of 1000 ms (IO 0 ms) over 7 samples, throughput 6.9924793 rps

I am sure there are ways to improve this, but I think this has some quite interesting properties, in particular:

  • Mostly a drop-in replacement for tokio::spawn
  • Less than 100 lines of code to maintain
  • Avoids IO starvation
  • Allows using rayon's very ergonomic parallelism options
  • Preserves the thread-locality originating from the way non-blocking operators are recursively "composed"

However, it is important to highlight that with this approach IO will still degrade poorly once CPU resources are saturated. Where there is a clear IO boundary, e.g. AsyncFileReader::get_bytes, it may still be worthwhile to spawn that as a dedicated tokio task so that it can run to completion without needing to "find time" on the rayon pool. However, this can be done as an optimisation if people run into such issues.

TLDR this avoids the major issue where concurrency nose dives long before CPU resources are saturated, without requiring complex shenanigans, and opening the door to greater intra-operator parallelism (e.g. using par_sort).

Edit: The first version of this had incorrect numbers for the tokio threadpool, the benchmark was incorrect, this has now been fixed

@alamb
Copy link
Contributor

alamb commented Dec 15, 2024

I've pushed a simple example to io_stall that glues together rayon and async_task to yield an async scheduler that is able to accommodate CPU bound tasks whilst not starving IO.

I hope to review this over the next day or two. Thank you

@djanderson
Copy link
Contributor

djanderson commented Jan 7, 2025

@tustvold, I gave this a thorough read today and plan to test out the approach soon.

The only thing I'm a little flaky on is the use of the call to use_current_thread() in the Rayon ThreadPoolBuilder here. The docs for that method seem pretty nuanced and don't make total sense to me, so I was wondering if this was a small optimization (re-use the existing thread because why-not) or is there was something deeper there with how the task spawned on Rayon is allowed to interact with the tokio runtime?

@peter-toth
Copy link
Contributor

peter-toth commented Jan 8, 2025

This is a very interresting issue.
I was trying to repro the results of the above experiement, but got matching results with rayon and tokio. Maybe rayon is slightly faster on my M3:

% cargo run --release --bin tokio -- --cpu-duration 1s --concurrency 7
    Finished `release` profile [optimized] target(s) in 0.46s
     Running `target/release/tokio --cpu-duration 1s --concurrency 7`
Average duration of 1061 ms (IO 57 ms) over 1 samples, throughput 0.94205743 rps
Average duration of 1060 ms (IO 57 ms) over 7 samples, throughput 6.741966 rps
Average duration of 1039 ms (IO 35 ms) over 7 samples, throughput 6.7168765 rps
Average duration of 1041 ms (IO 37 ms) over 7 samples, throughput 6.7122035 rps
Average duration of 1039 ms (IO 37 ms) over 7 samples, throughput 6.7568874 rps
Average duration of 1039 ms (IO 36 ms) over 7 samples, throughput 6.7319846 rps
Average duration of 1037 ms (IO 34 ms) over 7 samples, throughput 6.7261376 rps
Average duration of 1042 ms (IO 38 ms) over 7 samples, throughput 6.720298 rps
^C
% cargo run --release --bin rayon -- --cpu-duration 1s --concurrency 7
    Finished `release` profile [optimized] target(s) in 1.89s
     Running `target/release/rayon --cpu-duration 1s --concurrency 7`
Average duration of 1042 ms (IO 37 ms) over 1 samples, throughput 0.9594442 rps
Average duration of 1044 ms (IO 41 ms) over 7 samples, throughput 6.8305006 rps
Average duration of 1028 ms (IO 27 ms) over 7 samples, throughput 6.8629923 rps
Average duration of 1023 ms (IO 22 ms) over 7 samples, throughput 6.8896527 rps
Average duration of 1019 ms (IO 18 ms) over 7 samples, throughput 6.9374223 rps
Average duration of 1027 ms (IO 22 ms) over 7 samples, throughput 6.8580756 rps
Average duration of 1018 ms (IO 16 ms) over 7 samples, throughput 6.847353 rps
Average duration of 1018 ms (IO 16 ms) over 7 samples, throughput 6.8783445 rps
^C

Honestly, I don't get why would be significant difference between the 2 as both apps seem to work the same way.
We have the main thread that spawns tasks and those tasks are excuted either on the 8 threads of tokio or the 7 threads of rayon (more on this later). In the rayon app the tokio worker theads don't do anything, do they? So I would explain the slight discrepancy with the different work stealing logic of tokio and rayon.

Where I do see difference is --concurrency 8+:

% cargo run --release --bin tokio -- --cpu-duration 1s --concurrency 8
    Finished `release` profile [optimized] target(s) in 0.25s
     Running `target/release/tokio --cpu-duration 1s --concurrency 8`
Average duration of 1064 ms (IO 59 ms) over 1 samples, throughput 0.93955696 rps
Average duration of 1072 ms (IO 69 ms) over 8 samples, throughput 7.634239 rps
Average duration of 1045 ms (IO 43 ms) over 8 samples, throughput 7.6972647 rps
Average duration of 1040 ms (IO 37 ms) over 8 samples, throughput 7.66677 rps
Average duration of 1039 ms (IO 36 ms) over 8 samples, throughput 7.762669 rps
Average duration of 1037 ms (IO 34 ms) over 8 samples, throughput 7.6279187 rps
Average duration of 1044 ms (IO 40 ms) over 8 samples, throughput 7.680044 rps
Average duration of 1042 ms (IO 39 ms) over 8 samples, throughput 7.695585 rps
^C
% cargo run --release --bin rayon -- --cpu-duration 1s --concurrency 8
    Finished `release` profile [optimized] target(s) in 1.48s
     Running `target/release/rayon --cpu-duration 1s --concurrency 8`
Average duration of 1036 ms (IO 31 ms) over 1 samples, throughput 0.96456635 rps
Average duration of 1198 ms (IO 196 ms) over 7 samples, throughput 6.9807143 rps
Average duration of 1167 ms (IO 166 ms) over 7 samples, throughput 6.969797 rps
Average duration of 1158 ms (IO 156 ms) over 7 samples, throughput 6.9852505 rps
Average duration of 1145 ms (IO 143 ms) over 7 samples, throughput 6.9620543 rps
Average duration of 1150 ms (IO 148 ms) over 7 samples, throughput 6.962559 rps
Average duration of 1148 ms (IO 146 ms) over 7 samples, throughput 6.959907 rps
^C

But that's because of the aforementioned .use_current_thread() initialization of rayon treadpool which causes the main thread to be part of the rayon pool, but work stealing is not initialized there. Removing that line makes the 2 match again.

Edit: Maybe a different advantage of rayon over tokio is that it offers structured concurency via scoped spawning and so we can avoid the 'static lifetime requirement of the tasks. I'm not sure tokio has a similar API.

@tustvold
Copy link
Contributor Author

tustvold commented Jan 8, 2025

if this was a small optimization (re-use the existing thread because why-not) or is there was something deeper there with how the task spawned on Rayon is allowed to interact with the tokio runtime?

It was a quick demo cobbled together in a couple of hours, I wouldn't read too much into what it is doing 😅. In this case I probably just did what was easiest

Where I do see difference is --concurrency 8+:

Once the concurrency exceeds the pool size starvation is inevitable, and isn't the issue people have been running into.

The issue is tokio starving at low concurrency, I believe to do with the way it polls its reactor.

Unfortunately the reproducer had a bug, which I fixed, but now it isn't as obvious, perhaps it never was a good reproducer. What probably needs to happen is someone to try this on one of the real workloads that had this issue and see if it helps.

Unfortunately I have no more time to dedicate to this

Maybe a different advantage of rayon over tokio is that it offers structured concurency via scoped spawning and so we can avoid the 'static lifetime requirement of the tasks. I'm not sure tokio has a similar API.

1000% this, it also has a load of parallel operations like sorts out of the box

@stuhood
Copy link

stuhood commented Jan 8, 2025

(take with a grain of salt because I haven't worked with block_in_place inside of hot loops)

The performance implications definitely concern me, I have a nagging suspicion block_in_place spawns a thread...

block_in_place doesn't spawn a thread for the work that it scopes. As mentioned in the description, it can't: it doesn't place any lifetime/Send bounds on the closure that is running, and so the closure cannot be moved to another thread.

But it is possible that using block_in_place in enough positions will cause the tokio runtime to need to spin up more threads on the runtime (subject to the worker_threads and max_blocking_threads limits). To the extent that you saw a slowdown with block_in_place, I expect that the tokio maintainers would like to know about it: because last I checked, it was used internally in a few different tokio codepaths to wrap syscalls which didn't have non-blocking variants.

Additionally, as mentioned in its docstring, block_in_place can remove parallelism if it isn't used in concert with additional calls to spawn: for example, if you have a call to join like:

let (res1, res2) =
  join!(
    async {
        task::block_in_place(...);
    },
    async {
       // something async
       ...
    },
  );

... then the two async blocks may end up executing sequentially, because the entire task running the join is blocked. To fix it, you would spawn more tasks.

@crepererum
Copy link
Contributor

RE #13692 (comment) :

My concern here is that this thread switch (i.e. the tokio worker gets a new thread from the pool) is potentially expensive, esp. in a hot loop. It's also a potential synchronization point, because the thread pool is shared across all workers. So while this may be Ok if you do that occasionally, I don't think this is gonna yield good results if you do that e.g. for every record batch.

@stuhood
Copy link

stuhood commented Jan 9, 2025

My concern here is that this thread switch (i.e. the tokio worker gets a new thread from the pool) is potentially expensive, esp. in a hot loop. It's also a potential synchronization point, because the thread pool is shared across all workers. So while this may be Ok if you do that occasionally, I don't think this is gonna yield good results if you do that e.g. for every record batch.

Sorry, but to be 1000% clear: block_in_place does not context switch. It cannot. The closure it runs cannot be moved to another thread, because it doesn't require Send/etc. That generally makes it more convenient to use as well.

The reason that additional threads might be spawned is not for the task which is running a line of code which hit a block_in_place: rather, so that other tasks that are currently running on the executor do not starve. But of course, there might be other overheads in the implementation of block_in_place (and as mentioned: the tokio maintainers would love to hear about those).

@alamb
Copy link
Contributor

alamb commented Jan 25, 2025

Here is another PR with an example showing how to move CPU work to another threadpool:

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement New feature or request
Projects
None yet
Development

No branches or pull requests

7 participants