Skip to content

Conversation

@seddonm1
Copy link
Contributor

@andygrove

I found an interesting defect where the final partition of the RepartitionExec::execute thread spawner was consistently not being spawned via tokio::spawn. This meant that RepartitionStream::poll_next was sitting waiting forever for data that never arrived. I am unable to reproduce via DataFusion tests.

It looks like a race condition where the JoinHandle was not being awaited and something strange going on with the internals of tokio like lazy evaluation?

This PR fixes the problem.

@github-actions
Copy link

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @seddonm1 -- It looks like you need to run cargo fmt to get CI to pass but otherwise this looks like a good change to me. I wonder if there is any way to get a reproducer / test case for this?

Copy link
Member

@andygrove andygrove left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice catch @seddonm1. I ran into something similar in Ballista recently. It is always important to collect the results of these spawn operations.

@seddonm1
Copy link
Contributor Author

@alamb reformatted.

This is super weird as I have been running rustup update frequently (within last few days) and before I reran it this time cargo fmt did not format this code. It must be very fast moving.

@alamb
Copy link
Contributor

alamb commented Feb 19, 2021

https://github.com/apache/arrow/pull/9523/checks?check_run_id=1930131314 seems to have failed with something related to authentication. I am going re-run that check to see if it was an intermittent failure


==========================================================
Testing file auth:basic_proto
==========================================================
Traceback (most recent call last):
################# FAILURES #################
  File "/arrow/dev/archery/archery/integration/util.py", line 139, in run_cmd
FAILED TEST: auth:basic_proto Rust producing,  C++ consuming
    output = subprocess.check_output(cmd, stderr=subprocess.STDOUT)

  File "/opt/conda/envs/arrow/lib/python3.8/subprocess.py", line 411, in check_output
1 failures
    return run(*popenargs, stdout=PIPE, timeout=timeout, check=True,
  File "/opt/conda/envs/arrow/lib/python3.8/subprocess.py", line 512, in run
    raise CalledProcessError(retcode, process.args,
subprocess.CalledProcessError: Command '['/build/cpp/debug/flight-test-integration-client', '-host', 'localhost', '-port=38301', '-scenario', 'auth:basic_proto']' died with <Signals.SIGABRT: 6>.

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/arrow/dev/archery/archery/integration/runner.py", line 308, in _run_flight_test_case
    consumer.flight_request(port, **client_args)
  File "/arrow/dev/archery/archery/integration/tester_cpp.py", line 116, in flight_request
    run_cmd(cmd)
  File "/arrow/dev/archery/archery/integration/util.py", line 148, in run_cmd
    raise RuntimeError(sio.getvalue())
RuntimeError: Command failed: /build/cpp/debug/flight-test-integration-client -host localhost -port=38301 -scenario auth:basic_proto
With output:

@alamb
Copy link
Contributor

alamb commented Feb 19, 2021

The integration test failed the same way on this unrelated PR as well: https://github.com/apache/arrow/pull/9529/checks?check_run_id=1935707651

@seddonm1
Copy link
Contributor Author

@alamb is this good to go now?

@alamb
Copy link
Contributor

alamb commented Feb 21, 2021

yes -- thank you @seddonm1 -- I just had to re-run the integration test several times to get it to pass. I filed https://issues.apache.org/jira/browse/ARROW-11717 to track that, btw

@alamb alamb closed this in def1965 Feb 21, 2021
@edrevo
Copy link
Contributor

edrevo commented Feb 25, 2021

@andygrove / @seddonm1, I think we should revert this: RepartitionExec will now load everything into memory, since we are now .awaiting untill the input stream has been completely consumed, which moves all of the data into memory.

Maybe it would be worth understanding the root cause of the original bug to understand what is going on with the last partition. I had a similar behavior with ballista recently and it was due to https://issues.apache.org/jira/browse/ARROW-11784

@seddonm1
Copy link
Contributor Author

@edrevo We should revert if there is an issue.

This change puts the await per input partition so I thought it would wait until that partition is fully processed with the maximum limited by the tokio::spawn rules thus limiting memory?

@alamb
Copy link
Contributor

alamb commented Feb 25, 2021

I reviewed the code -- and I think @edrevo is right (basically that the call to execute won't return now until after all the streams have been finished). I wonder if the right fix is really https://issues.apache.org/jira/browse/ARROW-11784 / #9574

I can get a revert PR up probably tomorrow. @seddonm1 is there any chance you can try your test case that was hanging with the fix for #9574 to see if that fixes your issue?

@edrevo
Copy link
Contributor

edrevo commented Feb 25, 2021

Unfortunately, it looks like #9574 isn't the fix for this: I've reverted this change in my branch with that fix and it does in fact still hang in the last partition.

@seddonm1
Copy link
Contributor Author

@edrevo @alamb Yes. I have read through that code again and agree. It will be blocking.

Let me see if I can do some more debugging as my code is definitely reproducing the issue consistently.

@edrevo
Copy link
Contributor

edrevo commented Feb 25, 2021

By the way, I am trying to repartition 100GB of data and with this change I get OOMKilled (unfortunately for me I don't have 100GB of RAM 😂 ) and without this change the memory usage is normal (but it hangs)

@alamb
Copy link
Contributor

alamb commented Feb 25, 2021

Thanks @edrevo and @seddonm1 ! I am sorry to hear DataFusion is abusing your system :)

Let me know if I can help

@seddonm1
Copy link
Contributor Author

@edrevo you have inspired me to work this out

@andygrove
Copy link
Member

🍿 I am watching this with interest. Let me if/how I can help too. Once this works and the hash partitioning PR is merged then Ballista can implement "shuffled hash join" and that solves one of the main issues limiting scalability today.

@edrevo
Copy link
Contributor

edrevo commented Feb 25, 2021

Ok, after looking at the code for a while, I'm going to throw in a theory: the problem lies with https://github.com/apache/arrow/blob/master/rust/datafusion/src/physical_plan/repartition.rs#L216

That line will block the thread until there is data coming in. This is not a good idea in Tokio, since if there are enough partitions, that would end up blocking all of the threads in the threadpool. I believe this should be .try_recv and we should map Err(TryRecvError::Empty) to Poll::Pending.

What is my working theory? The downstream ExecutionPlan will request progress from as many partitions as it can. If these requests happen fast enough, they might block the tokio threadpool, and the task which is in charge of feeding data to these channels cannot progress: deadlock.

I don't have time to make the change an test it out, but if someone wants to pick this up and see if it solves the issue please go ahead. If not, I'll try it out tomorrow.

Cheers!

@seddonm1
Copy link
Contributor Author

@edrevo

My working hypothesis is that this is a race condition in spawning tokio tasks before the execute() exits. I am able to reproduce so just need to do some testing. TBC

@seddonm1
Copy link
Contributor Author

@edrevo @andygrove @alamb

Got it!

My summary of what was happening is that the last thread was never being spawned as we didn't provide an opportunity for tokio to spawn it when running from within another tokio thread.

By adding tokio::task::yield_now().await; in place of this join_handle.await... the tokio runtime is able to still spawn the task.

Can you please test #9580?

@edrevo
Copy link
Contributor

edrevo commented Feb 26, 2021

I've fiddled around with the code and I'm now convinced the root cause is what I mentioned in #9523 (comment)

It would probably be a good idea to audit all of the usages of crossbeam in the project, because crossbeam + tokio don't mix well. See https://tokio.rs/tokio/tutorial/channels#tokios-channel-primitives, where they mention that crossbeam is for use outside of async contexts:

There are also channels for use outside of asynchronous Rust, such as std::sync::mpsc and crossbeam::channel. These channels wait for messages by blocking the thread, which is not allowed in asynchronous code.

(emphasis is mine)

I have a local change where I'm using tokio's mpsc channel and there's no deadlock. I haven't done any formal measurements, but with that change the repartition of the 100GBs uses flat memory and is able to use x1.5 the amount of CPUs that it used before. I can also see the output files growing at a larger rate. This is probably because tokio's threads aren't blocked anymore and tokio's scheduler can schedule more work into the same amount of threads.

An alternative to tokio's mpsc would be to use the async_channel crate, but I have never used it so I don't know what's the best option here.

I can open a PR with these changes if you want, but let me know if you prefer tokio's mpsc or the async_channel crate.

@seddonm1
Copy link
Contributor Author

@edrevo This is interesting and I wonder if perhaps multiple issues - although the nature of RepartitionExec makes it hard to isolate from other DataFusion components.

I ran many different scenarios yesterday to try to understand what was happening trying to understand why RepatitionExec::execute was not spawning threads. My aim from this approach was to isolate the tokio::spawn from the crossbeam channels. I stripped it back to the point where it was basically just spawning num_output_partition threads that had a single println! as their only action. I completely removed all references to channels from either the core 0..num_input_partitions loop and channels.empty() and added delays after the spawning to try to prevent a race condition. Consistently the last tokio::spawn would not be invoked on the final loop iteration (I verified this was not tokio bound by testing 0..num_input_partitions + 1 which this time correctly started num_input_partitions but not the new final iteration).

It appears to me (and I have only just started reading the tokio documentation) that these tokio::spawn calls are not yielding to the scheduler as that would normally happen via the .await call. As we are spawning 'start and forget' threads (we don't want the blocking behavior) the scheduler does not have a chance to yield for that last tokio::spawn hence why tokio::task::yield_now().await works.

There may be a second problem that you have identified with crossbeam vs tokio::mpsc which, to me at least, feels like the answer is to switch to tokio::mpsc as the dependency is already imported and is likely to be well tested with/by tokio - and the tokio runtime is unlikely to be replaced any time soon.

@alamb
Copy link
Contributor

alamb commented Feb 26, 2021

@edrevo I think removing the use of crossbeam::channel in favor of tokio::mspc::channel is the right approach.

It appears to me (and I have only just started reading the tokio documentation) that these tokio::spawn calls are not yielding to the scheduler as that would normally happen via the .await call.

@seddonm1 it is my understanding (and it may be wrong) that calls to await offer an opportunity to yield to the scheduler but the tokio scheduler will attempt to use the current thread execute whatever you are awaiting on and if that can proceed tokio will execute it -- it makes no attempt to be fair, it is trying to maximize throughput

Putting a call to tokio::task_yield_now() sticks the current task at the back of the scheduling queue (rather than really "yielding" in the normal cooperative multi-threading sense

@seddonm1
Copy link
Contributor Author

It appears to me (and I have only just started reading the tokio documentation) that these tokio::spawn calls are not yielding to the scheduler as that would normally happen via the .await call.

@seddonm1 it is my understanding (and it may be wrong) that calls to await offer an opportunity to yield to the scheduler but the tokio scheduler will attempt to use the current thread execute whatever you are awaiting on and if that can proceed tokio will execute it -- it makes no attempt to be fair, it is trying to maximize throughput

Putting a call to tokio::task_yield_now() sticks the current task at the back of the scheduling queue (rather than really "yielding" in the normal cooperative multi-threading sense

Yes, this fits with my understanding. I would like to try to create a test for this as I have only observed this whilst running from within another tokio::spawn task - which will probably impact projects like Ballista more than DataFusion.

alamb pushed a commit that referenced this pull request Feb 26, 2021
After #9523 RepartitionExec is blocking to pull all data into memory before starting the stream which crashes on large sets.

Closes #9580 from seddonm1/yield-now

Authored-by: Mike Seddon <[email protected]>
Signed-off-by: Andrew Lamb <[email protected]>
@alamb
Copy link
Contributor

alamb commented Feb 26, 2021

I filed https://issues.apache.org/jira/browse/ARROW-11802 to track the hang. @edrevo if you want to give it a go that would be great. If not, I can try and hack on it this weekend maybe.

I also merged #9580 to unblock you @edrevo (stop DataFusion from OOM'ing on you)

@seddonm1
Copy link
Contributor Author

@alamb better news. I have a very simple test to demonstrate the issue. I will push that separately.

@edrevo
Copy link
Contributor

edrevo commented Feb 26, 2021

@seddonm1, thanks for the info around the tests you have made with tokio::spawn. I would be interested in a repro case for the behavior you are seeing where the last thread isn't scheduled and there's no usage of channels. Out of curiosity, to better understand tokio's internals.

In my case, I was able to reproduce the hang but after changing to tokio's mspc the hang wasn't there anymore. Maybe I just got lucky, or maybe that change minimized the chance of a hang but doesn't eliminate it completely. You never know with these threading issues...

Adding an explicit yield call sounds fine to me, tbh, so from my point of view, leaving that call in the code is fine.

@edrevo
Copy link
Contributor

edrevo commented Feb 26, 2021

Woops, looks like we commented at the same time. Looking forward to that repro test!

@seddonm1
Copy link
Contributor Author

@edrevo @alamb @andygrove

Here is a branch with a test to reproduce the hanging behavior I was experiencing:
https://github.com/seddonm1/arrow/tree/reproduce-hang

Please run:

cargo test "many_to_many_round_robin_within_tokio_task" -- --nocapture

... and we all get the fun of debugging a nice complex problem 🍿

@andygrove
Copy link
Member

I will try this out on my 24 core thread ripper tomorrow. Thanks to all who have contributed to tracking this down.

@edrevo
Copy link
Contributor

edrevo commented Feb 27, 2021

@alamb, I would like to spend this weekend unblocking a PR I have in ballista which has been open for a week now, so I would appreciate if you could pick up https://issues.apache.org/jira/browse/ARROW-11802. I have a branch at https://github.com/edrevo/arrow/tree/remove-crossbeam that removes crossbeam from the repartition, but I haven't checked for any other usages of crossbeam. Feel free to use that as a starting point.

@seddonm1, I have checked your repro. Once you remove crossbeam from the equation, the hang doesn't reproduce in your test case either (at least on my machine).

Many thanks to everyone for the interesting discussion around this issue and for helping me with the repartition I needed to run 😄

@seddonm1
Copy link
Contributor Author

@edrevo that is great that removing crossbeam solves the test case and it sounds like a good plan anyway given Tokio explicitly talks about this potential issue.

It would be great to dig a bit deeper and really understand what's going on but in the meantime I think your change should be incorporated. 👍👍

@alamb
Copy link
Contributor

alamb commented Feb 27, 2021

@edrevo cool -- thanks. There was one other use of crossbeam in the parquet reader which I can try tomorrow morning. Thanks!

@alamb
Copy link
Contributor

alamb commented Feb 28, 2021

@edrevo / @seddonm1 I am about to run out of time this morning for removing crossbeam. However, I plan to get to it over the next few days

@seddonm1
Copy link
Contributor Author

seddonm1 commented Mar 1, 2021

@alamb I have raised a PR to merge in the test so that we can ensure whatever changes does not cause a hang.

@alamb
Copy link
Contributor

alamb commented Mar 1, 2021

Thank you @seddonm1

@alamb
Copy link
Contributor

alamb commented Mar 1, 2021

PR is #9603 for anyone else following along

@alamb
Copy link
Contributor

alamb commented Mar 1, 2021

Here is a proposed PR for removing crossbeam: #9605

alamb added a commit that referenced this pull request Mar 3, 2021
…void potential deadlocks

# Rationale

As spotted / articulated by @edrevo #9523 (comment), the intermixing of `crossbeam` channels (not designed for `async` and can block task threads) and `async` code such as DataFusion can lead to deadlock.

At least one of the crossbeam uses predates DataFusion being async (e.g. the one in the parquet reader). The use of crossbeam in the repartition operator in #8982 may have resulted from the re-use of the same pattern.

# Changes

1. Removes the use of crossbeam channels from DataFusion (in `RepartitionExec` and `ParquetExec`) and replace with tokio channels (which are designed for single threaded code).
2. Removes `crossbeam` dependency entirely
3. Removes use of `multi_thread`ed executor in tests (e.g. `#[tokio::test(flavor = "multi_thread")]`) which can mask hangs

# Kudos / Thanks

This PR incorporates the work of @seddonm1 from #9603 and @edrevo in  https://github.com/edrevo/arrow/tree/remove-crossbeam (namely 97c256c4f76b8185311f36a7b27e317588904a3a). A big thanks to both of them for their help in this endeavor.

Closes #9605 from alamb/alamb/remove_hang

Lead-authored-by: Ximo Guanter <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
Co-authored-by: Mike Seddon <[email protected]>
Signed-off-by: Andrew Lamb <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants