-
Notifications
You must be signed in to change notification settings - Fork 4k
ARROW-11802: [Rust][DataFusion] Remove use of crossbeam channels to avoid potential deadlocks #9605
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
Closed
Closed
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
7df0c12
add test for tokio tasks within tasks
seddonm1 3859b87
Use single threaded test harness
alamb 3b28871
remove yield_now, causes deadlock
alamb 321ec55
remove crossbeam from repartition
edrevo 70111f3
cargo fmt
alamb c3f1435
Use tokio channels in parquet reader
alamb 9b9af1f
Remove crossbeam entirely
alamb File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,21 +21,24 @@ | |
| use std::pin::Pin; | ||
| use std::sync::Arc; | ||
| use std::task::{Context, Poll}; | ||
| use std::{any::Any, vec}; | ||
| use std::{any::Any, collections::HashMap, vec}; | ||
|
|
||
| use crate::error::{DataFusionError, Result}; | ||
| use crate::physical_plan::{ExecutionPlan, Partitioning}; | ||
| use arrow::record_batch::RecordBatch; | ||
| use arrow::{array::Array, error::Result as ArrowResult}; | ||
| use arrow::{compute::take, datatypes::SchemaRef}; | ||
| use tokio_stream::wrappers::UnboundedReceiverStream; | ||
|
|
||
| use super::{hash_join::create_hashes, RecordBatchStream, SendableRecordBatchStream}; | ||
| use async_trait::async_trait; | ||
|
|
||
| use crossbeam::channel::{unbounded, Receiver, Sender}; | ||
| use futures::stream::Stream; | ||
| use futures::StreamExt; | ||
| use tokio::sync::Mutex; | ||
| use tokio::sync::{ | ||
| mpsc::{UnboundedReceiver, UnboundedSender}, | ||
| Mutex, | ||
| }; | ||
| use tokio::task::JoinHandle; | ||
|
|
||
| type MaybeBatch = Option<ArrowResult<RecordBatch>>; | ||
|
|
@@ -48,9 +51,13 @@ pub struct RepartitionExec { | |
| input: Arc<dyn ExecutionPlan>, | ||
| /// Partitioning scheme to use | ||
| partitioning: Partitioning, | ||
| /// Channels for sending batches from input partitions to output partitions | ||
| /// there is one entry in this Vec for each output partition | ||
| channels: Arc<Mutex<Vec<(Sender<MaybeBatch>, Receiver<MaybeBatch>)>>>, | ||
| /// Channels for sending batches from input partitions to output partitions. | ||
| /// Key is the partition number | ||
| channels: Arc< | ||
| Mutex< | ||
| HashMap<usize, (UnboundedSender<MaybeBatch>, UnboundedReceiver<MaybeBatch>)>, | ||
| >, | ||
| >, | ||
| } | ||
|
|
||
| impl RepartitionExec { | ||
|
|
@@ -110,23 +117,28 @@ impl ExecutionPlan for RepartitionExec { | |
| // if this is the first partition to be invoked then we need to set up initial state | ||
| if channels.is_empty() { | ||
| // create one channel per *output* partition | ||
| for _ in 0..num_output_partitions { | ||
| for partition in 0..num_output_partitions { | ||
| // Note that this operator uses unbounded channels to avoid deadlocks because | ||
| // the output partitions can be read in any order and this could cause input | ||
| // partitions to be blocked when sending data to output receivers that are not | ||
| // partitions to be blocked when sending data to output UnboundedReceivers that are not | ||
| // being read yet. This may cause high memory usage if the next operator is | ||
| // reading output partitions in order rather than concurrently. One workaround | ||
| // for this would be to add spill-to-disk capabilities. | ||
| let (sender, receiver) = unbounded::<Option<ArrowResult<RecordBatch>>>(); | ||
| channels.push((sender, receiver)); | ||
| let (sender, receiver) = tokio::sync::mpsc::unbounded_channel::< | ||
| Option<ArrowResult<RecordBatch>>, | ||
| >(); | ||
| channels.insert(partition, (sender, receiver)); | ||
| } | ||
| let random = ahash::RandomState::new(); | ||
|
|
||
| // launch one async task per *input* partition | ||
| for i in 0..num_input_partitions { | ||
| let random_state = random.clone(); | ||
| let input = self.input.clone(); | ||
| let mut channels = channels.clone(); | ||
| let mut txs: HashMap<_, _> = channels | ||
| .iter() | ||
| .map(|(partition, (tx, _rx))| (*partition, tx.clone())) | ||
| .collect(); | ||
| let partitioning = self.partitioning.clone(); | ||
| let _: JoinHandle<Result<()>> = tokio::spawn(async move { | ||
| let mut stream = input.execute(i).await?; | ||
|
|
@@ -135,7 +147,7 @@ impl ExecutionPlan for RepartitionExec { | |
| match &partitioning { | ||
| Partitioning::RoundRobinBatch(_) => { | ||
| let output_partition = counter % num_output_partitions; | ||
| let tx = &mut channels[output_partition].0; | ||
| let tx = txs.get_mut(&output_partition).unwrap(); | ||
| tx.send(Some(result)).map_err(|e| { | ||
| DataFusionError::Execution(e.to_string()) | ||
| })?; | ||
|
|
@@ -180,7 +192,7 @@ impl ExecutionPlan for RepartitionExec { | |
| input_batch.schema(), | ||
| columns, | ||
| ); | ||
| let tx = &mut channels[num_output_partition].0; | ||
| let tx = txs.get_mut(&num_output_partition).unwrap(); | ||
| tx.send(Some(output_batch)).map_err(|e| { | ||
| DataFusionError::Execution(e.to_string()) | ||
| })?; | ||
|
|
@@ -199,14 +211,12 @@ impl ExecutionPlan for RepartitionExec { | |
| } | ||
|
|
||
| // notify each output partition that this input partition has no more data | ||
| for channel in channels.iter_mut().take(num_output_partitions) { | ||
| let tx = &mut channel.0; | ||
| for (_, tx) in txs { | ||
| tx.send(None) | ||
| .map_err(|e| DataFusionError::Execution(e.to_string()))?; | ||
| } | ||
| Ok(()) | ||
| }); | ||
| tokio::task::yield_now().await; | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is the workaround added in #9580 and it is now removed in favor of what we think is the fix of the root cause |
||
| } | ||
| } | ||
|
|
||
|
|
@@ -216,7 +226,7 @@ impl ExecutionPlan for RepartitionExec { | |
| num_input_partitions, | ||
| num_input_partitions_processed: 0, | ||
| schema: self.input.schema(), | ||
| input: channels[partition].1.clone(), | ||
| input: UnboundedReceiverStream::new(channels.remove(&partition).unwrap().1), | ||
| })) | ||
| } | ||
| } | ||
|
|
@@ -230,7 +240,7 @@ impl RepartitionExec { | |
| Ok(RepartitionExec { | ||
| input, | ||
| partitioning, | ||
| channels: Arc::new(Mutex::new(vec![])), | ||
| channels: Arc::new(Mutex::new(HashMap::new())), | ||
| }) | ||
| } | ||
| } | ||
|
|
@@ -243,7 +253,7 @@ struct RepartitionStream { | |
| /// Schema | ||
| schema: SchemaRef, | ||
| /// channel containing the repartitioned batches | ||
| input: Receiver<Option<ArrowResult<RecordBatch>>>, | ||
| input: UnboundedReceiverStream<Option<ArrowResult<RecordBatch>>>, | ||
| } | ||
|
|
||
| impl Stream for RepartitionStream { | ||
|
|
@@ -253,10 +263,9 @@ impl Stream for RepartitionStream { | |
| mut self: Pin<&mut Self>, | ||
| cx: &mut Context<'_>, | ||
| ) -> Poll<Option<Self::Item>> { | ||
| match self.input.recv() { | ||
| Ok(Some(batch)) => Poll::Ready(Some(batch)), | ||
| // End of results from one input partition | ||
| Ok(None) => { | ||
| match self.input.poll_next_unpin(cx) { | ||
| Poll::Ready(Some(Some(v))) => Poll::Ready(Some(v)), | ||
| Poll::Ready(Some(None)) => { | ||
| self.num_input_partitions_processed += 1; | ||
| if self.num_input_partitions == self.num_input_partitions_processed { | ||
| // all input partitions have finished sending batches | ||
|
|
@@ -266,8 +275,8 @@ impl Stream for RepartitionStream { | |
| self.poll_next(cx) | ||
| } | ||
| } | ||
| // RecvError means receiver has exited and closed the channel | ||
| Err(_) => Poll::Ready(None), | ||
| Poll::Ready(None) => Poll::Ready(None), | ||
| Poll::Pending => Poll::Pending, | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -287,7 +296,7 @@ mod tests { | |
| use arrow::datatypes::{DataType, Field, Schema}; | ||
| use arrow::record_batch::RecordBatch; | ||
|
|
||
| #[tokio::test(flavor = "multi_thread")] | ||
| #[tokio::test] | ||
| async fn one_to_many_round_robin() -> Result<()> { | ||
| // define input partitions | ||
| let schema = test_schema(); | ||
|
|
@@ -307,7 +316,7 @@ mod tests { | |
| Ok(()) | ||
| } | ||
|
|
||
| #[tokio::test(flavor = "multi_thread")] | ||
| #[tokio::test] | ||
| async fn many_to_one_round_robin() -> Result<()> { | ||
| // define input partitions | ||
| let schema = test_schema(); | ||
|
|
@@ -324,7 +333,7 @@ mod tests { | |
| Ok(()) | ||
| } | ||
|
|
||
| #[tokio::test(flavor = "multi_thread")] | ||
| #[tokio::test] | ||
| async fn many_to_many_round_robin() -> Result<()> { | ||
| // define input partitions | ||
| let schema = test_schema(); | ||
|
|
@@ -345,7 +354,7 @@ mod tests { | |
| Ok(()) | ||
| } | ||
|
|
||
| #[tokio::test(flavor = "multi_thread")] | ||
| #[tokio::test] | ||
| async fn many_to_many_hash_partition() -> Result<()> { | ||
| // define input partitions | ||
| let schema = test_schema(); | ||
|
|
@@ -415,4 +424,32 @@ mod tests { | |
| } | ||
| Ok(output_partitions) | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn many_to_many_round_robin_within_tokio_task() -> Result<()> { | ||
| let join_handle: JoinHandle<Result<Vec<Vec<RecordBatch>>>> = | ||
| tokio::spawn(async move { | ||
| // define input partitions | ||
| let schema = test_schema(); | ||
| let partition = create_vec_batches(&schema, 50); | ||
| let partitions = | ||
| vec![partition.clone(), partition.clone(), partition.clone()]; | ||
|
|
||
| // repartition from 3 input to 5 output | ||
| repartition(&schema, partitions, Partitioning::RoundRobinBatch(5)).await | ||
| }); | ||
|
|
||
| let output_partitions = join_handle | ||
| .await | ||
| .map_err(|e| DataFusionError::Internal(e.to_string()))??; | ||
|
|
||
| assert_eq!(5, output_partitions.len()); | ||
| assert_eq!(30, output_partitions[0].len()); | ||
| assert_eq!(30, output_partitions[1].len()); | ||
| assert_eq!(30, output_partitions[2].len()); | ||
| assert_eq!(30, output_partitions[3].len()); | ||
| assert_eq!(30, output_partitions[4].len()); | ||
|
|
||
| Ok(()) | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Note @Dandandan asked about using a
HashMapvs some other structure. It is aHashMapfor reasons explained by @edrevo here: edrevo@97c256c#r47626396There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I didn't follow the answer completely; why do we need to remove it?
I think
UnboundedSenderis cheap to clone and cheap to keep in memory for the duration ofRepartitionStream?But maybe I would need to play with the code more to see why it is needed per se.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So ok if it's needed, but I just wasn't sure 👍
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @Dandandan
The use is here: https://github.com/apache/arrow/pull/9605/files#diff-b9b79e3b35bc8bfb43040ada3a4382bd0a0017ca1b1e8135be8fb310ff095674R229
Basically this code sets up all input and outputs channels for all of the partitions and then hands out one receiver at a time in some arbitrary order (depending on the
partitionargument).UnboundedReceiverhttps://docs.rs/tokio/1.2.0/tokio/sync/mpsc/struct.UnboundedReceiver.html doesn't implementClone(as it is multiple producer single consumer)I suspect with some more thought a different structure could be used, but I couldn't convince myself it was a valuable use of time.