You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "tustvold (via GitHub)" <gi...@apache.org> on 2023/10/13 14:45:16 UTC

[PR] Extract ReceiverStreamBuilder [arrow-datafusion]

tustvold opened a new pull request, #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817

   ## Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Closes #.
   
   ## Rationale for this change
   
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   An alternative to https://github.com/apache/arrow-datafusion/pull/7800 that instead of trying to make `RecordBatchReceiverStreamBuilder` handle non-RecordBatch inputs, instead extracts the lower-level stream management logic.
   
   ## What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   ## Are these changes tested?
   
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   
   ## Are there any user-facing changes?
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please add the `api change` label.
   -->


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#issuecomment-1762748989

   Given panic propagation and task cancellation are the entire purpose of this construction, I think we're therefore not duplicating something upstream? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#discussion_r1358365586


##########
datafusion/physical-plan/src/stream.rs:
##########
@@ -38,6 +38,113 @@ use tokio::task::JoinSet;
 use super::metrics::BaselineMetrics;
 use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream};
 
+/// Creates a stream from a collection of producing tasks, routing panics to the stream
+pub(crate) struct ReceiverStreamBuilder<O> {
+    tx: Sender<Result<O>>,
+    rx: Receiver<Result<O>>,
+    join_set: JoinSet<Result<()>>,
+}
+
+impl<O: Send + 'static> ReceiverStreamBuilder<O> {
+    /// create new channels with the specified buffer size
+    pub fn new(capacity: usize) -> Self {
+        let (tx, rx) = tokio::sync::mpsc::channel(capacity);
+
+        Self {
+            tx,
+            rx,
+            join_set: JoinSet::new(),
+        }
+    }
+
+    /// Get a handle for sending [`O`] to the output
+    pub fn tx(&self) -> Sender<Result<O>> {
+        self.tx.clone()
+    }
+
+    /// Spawn task that will be aborted if this builder (or the stream
+    /// built from it) are dropped
+    pub fn spawn<F>(&mut self, task: F)
+    where
+        F: Future<Output = Result<()>>,
+        F: Send + 'static,
+    {
+        self.join_set.spawn(task);
+    }
+
+    /// Spawn a blocking task that will be aborted if this builder (or the stream
+    /// built from it) are dropped
+    ///
+    /// this is often used to spawn tasks that write to the sender
+    /// retrieved from `Self::tx`
+    pub fn spawn_blocking<F>(&mut self, f: F)
+    where
+        F: FnOnce() -> Result<()>,
+        F: Send + 'static,
+    {
+        self.join_set.spawn_blocking(f);
+    }
+
+    /// Create a stream of all [`O`] written to `tx`
+    pub fn build(self) -> BoxStream<'static, Result<O>> {
+        let Self {
+            tx,
+            rx,
+            mut join_set,
+        } = self;
+
+        // don't need tx
+        drop(tx);
+
+        // future that checks the result of the join set, and propagates panic if seen

Review Comment:
   This is the interesting logic that needs DRYing



##########
datafusion/physical-plan/src/stream.rs:
##########
@@ -47,28 +154,22 @@ use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream};
 ///
 /// This also handles propagating panic`s and canceling the tasks.
 pub struct RecordBatchReceiverStreamBuilder {
-    tx: Sender<Result<RecordBatch>>,
-    rx: Receiver<Result<RecordBatch>>,
     schema: SchemaRef,
-    join_set: JoinSet<Result<()>>,
+    inner: ReceiverStreamBuilder<RecordBatch>,
 }
 
 impl RecordBatchReceiverStreamBuilder {
     /// create new channels with the specified buffer size
     pub fn new(schema: SchemaRef, capacity: usize) -> Self {
-        let (tx, rx) = tokio::sync::mpsc::channel(capacity);
-
         Self {
-            tx,
-            rx,
             schema,
-            join_set: JoinSet::new(),
+            inner: ReceiverStreamBuilder::new(capacity),

Review Comment:
   It seemed unnecessary / undesirable to burden ReceiverStreamBuilder with a notion of Schema



##########
datafusion/physical-plan/src/stream.rs:
##########
@@ -110,7 +211,7 @@ impl RecordBatchReceiverStreamBuilder {
     ) {
         let output = self.tx();
 
-        self.spawn(async move {
+        self.inner.spawn(async move {

Review Comment:
   This logic by comparison is rather ExecutionPlan specifi, and I don't think valuable to DRY



##########
datafusion/physical-plan/src/stream.rs:
##########
@@ -38,6 +38,113 @@ use tokio::task::JoinSet;
 use super::metrics::BaselineMetrics;
 use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream};
 
+/// Creates a stream from a collection of producing tasks, routing panics to the stream
+pub(crate) struct ReceiverStreamBuilder<O> {

Review Comment:
   I made this pub(crate) but it could easily be made public should we wish to do so



##########
datafusion/physical-plan/src/stream.rs:
##########
@@ -155,80 +256,14 @@ impl RecordBatchReceiverStreamBuilder {
         });
     }
 
-    /// Create a stream of all `RecordBatch`es written to `tx`
+    /// Create a stream of all [`O`] written to `tx`
     pub fn build(self) -> SendableRecordBatchStream {
-        let Self {
-            tx,
-            rx,
-            schema,
-            mut join_set,
-        } = self;
-
-        // don't need tx
-        drop(tx);
-
-        // future that checks the result of the join set, and propagates panic if seen
-        let check = async move {
-            while let Some(result) = join_set.join_next().await {
-                match result {
-                    Ok(task_result) => {
-                        match task_result {
-                            // nothing to report
-                            Ok(_) => continue,
-                            // This means a blocking task error
-                            Err(e) => {
-                                return Some(exec_err!("Spawned Task error: {e}"));
-                            }
-                        }
-                    }
-                    // This means a tokio task error, likely a panic
-                    Err(e) => {
-                        if e.is_panic() {
-                            // resume on the main thread
-                            std::panic::resume_unwind(e.into_panic());
-                        } else {
-                            // This should only occur if the task is
-                            // cancelled, which would only occur if
-                            // the JoinSet were aborted, which in turn
-                            // would imply that the receiver has been
-                            // dropped and this code is not running
-                            return Some(internal_err!("Non Panic Task error: {e}"));
-                        }
-                    }
-                }
-            }
-            None
-        };
-
-        let check_stream = futures::stream::once(check)
-            // unwrap Option / only return the error
-            .filter_map(|item| async move { item });
-
-        // Convert the receiver into a stream
-        let rx_stream = futures::stream::unfold(rx, |mut rx| async move {
-            let next_item = rx.recv().await;
-            next_item.map(|next_item| (next_item, rx))
-        });
-
-        // Merge the streams together so whichever is ready first
-        // produces the batch
-        let inner = futures::stream::select(rx_stream, check_stream).boxed();
-
-        Box::pin(RecordBatchReceiverStream { schema, inner })
+        Box::pin(RecordBatchStreamAdapter::new(self.schema, self.inner.build()))
     }
 }
 
-/// A [`SendableRecordBatchStream`] that combines [`RecordBatch`]es from multiple inputs,
-/// on new tokio Tasks,  increasing the potential parallelism.
-///
-/// This structure also handles propagating panics and cancelling the
-/// underlying tasks correctly.
-///
-/// Use [`Self::builder`] to construct one.
-pub struct RecordBatchReceiverStream {

Review Comment:
   This type doesn't really make a lot of sense, given it isn't actually what the builder returns



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#issuecomment-1763344062

   I plan to merge this PR when the CI passes


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#discussion_r1358707688


##########
datafusion/physical-plan/src/stream.rs:
##########
@@ -155,80 +256,17 @@ impl RecordBatchReceiverStreamBuilder {
         });
     }
 
-    /// Create a stream of all `RecordBatch`es written to `tx`
+    /// Create a stream of all [`RecordBatch`] written to `tx`
     pub fn build(self) -> SendableRecordBatchStream {
-        let Self {
-            tx,
-            rx,
-            schema,
-            mut join_set,
-        } = self;
-
-        // don't need tx
-        drop(tx);
-
-        // future that checks the result of the join set, and propagates panic if seen
-        let check = async move {
-            while let Some(result) = join_set.join_next().await {
-                match result {
-                    Ok(task_result) => {
-                        match task_result {
-                            // nothing to report
-                            Ok(_) => continue,
-                            // This means a blocking task error
-                            Err(e) => {
-                                return Some(exec_err!("Spawned Task error: {e}"));
-                            }
-                        }
-                    }
-                    // This means a tokio task error, likely a panic
-                    Err(e) => {
-                        if e.is_panic() {
-                            // resume on the main thread
-                            std::panic::resume_unwind(e.into_panic());
-                        } else {
-                            // This should only occur if the task is
-                            // cancelled, which would only occur if
-                            // the JoinSet were aborted, which in turn
-                            // would imply that the receiver has been
-                            // dropped and this code is not running
-                            return Some(internal_err!("Non Panic Task error: {e}"));
-                        }
-                    }
-                }
-            }
-            None
-        };
-
-        let check_stream = futures::stream::once(check)
-            // unwrap Option / only return the error
-            .filter_map(|item| async move { item });
-
-        // Convert the receiver into a stream
-        let rx_stream = futures::stream::unfold(rx, |mut rx| async move {
-            let next_item = rx.recv().await;
-            next_item.map(|next_item| (next_item, rx))
-        });
-
-        // Merge the streams together so whichever is ready first
-        // produces the batch
-        let inner = futures::stream::select(rx_stream, check_stream).boxed();
-
-        Box::pin(RecordBatchReceiverStream { schema, inner })
+        Box::pin(RecordBatchStreamAdapter::new(
+            self.schema,
+            self.inner.build(),
+        ))
     }
 }
 
-/// A [`SendableRecordBatchStream`] that combines [`RecordBatch`]es from multiple inputs,
-/// on new tokio Tasks,  increasing the potential parallelism.
-///
-/// This structure also handles propagating panics and cancelling the
-/// underlying tasks correctly.
-///
-/// Use [`Self::builder`] to construct one.
-pub struct RecordBatchReceiverStream {
-    schema: SchemaRef,
-    inner: BoxStream<'static, Result<RecordBatch>>,
-}
+#[doc(hidden)]
+pub struct RecordBatchReceiverStream {}
 
 impl RecordBatchReceiverStream {
     /// Create a builder with an internal buffer of capacity batches.

Review Comment:
   Yeah, this method documentation also seems incorrect at this time. 



##########
datafusion/physical-plan/src/stream.rs:
##########
@@ -47,28 +154,22 @@ use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream};
 ///
 /// This also handles propagating panic`s and canceling the tasks.
 pub struct RecordBatchReceiverStreamBuilder {
-    tx: Sender<Result<RecordBatch>>,
-    rx: Receiver<Result<RecordBatch>>,
     schema: SchemaRef,
-    join_set: JoinSet<Result<()>>,
+    inner: ReceiverStreamBuilder<RecordBatch>,
 }
 
 impl RecordBatchReceiverStreamBuilder {
     /// create new channels with the specified buffer size
     pub fn new(schema: SchemaRef, capacity: usize) -> Self {
-        let (tx, rx) = tokio::sync::mpsc::channel(capacity);
-
         Self {
-            tx,
-            rx,
             schema,
-            join_set: JoinSet::new(),
+            inner: ReceiverStreamBuilder::new(capacity),

Review Comment:
   I agree



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "waynexia (via GitHub)" <gi...@apache.org>.
waynexia commented on PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#issuecomment-1762749893

   Thanks for explaining, LGTM 👍 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold merged PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#discussion_r1359317870


##########
datafusion/physical-plan/src/stream.rs:
##########
@@ -38,6 +38,113 @@ use tokio::task::JoinSet;
 use super::metrics::BaselineMetrics;
 use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream};
 
+/// Creates a stream from a collection of producing tasks, routing panics to the stream

Review Comment:
   ```suggestion
   /// Creates a stream from a collection of producing tasks, routing panics to the stream.
   ///
   /// Note that this is similar to  [`ReceiverStream` from tokio-stream], with the differences being:
   ///
   /// 1. Methods to bound and "detach"  tasks (`spawn()` and `spawn_blocking()`).
   ///
   /// 2. Propagates panics, whereas the `tokio` version doesn't propagate panics to the receiver.
   ///
   /// 3. Automatically cancels any outstanding tasks when the receiver stream is dropped. 
   ///
   /// [`ReceiverStream` from tokio-stream]: https://docs.rs/tokio-stream/latest/tokio_stream/wrappers/struct.ReceiverStream.html
   
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "wiedld (via GitHub)" <gi...@apache.org>.
wiedld commented on PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#issuecomment-1763191096

   > What do you think @wiedld ?
   
   Looks great. TY.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "waynexia (via GitHub)" <gi...@apache.org>.
waynexia commented on PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#issuecomment-1762745713

   Tokio version doesn't propagate panics. A panicked task will only panic the env it runs. It provides cancellation from the mpsc, dropped consumer (or the stream itself) will result in errors in senders.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#issuecomment-1762744246

   AFAICT the tokio version doesn't provide panic propagation nor cancellation?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Extract ReceiverStreamBuilder [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #7817:
URL: https://github.com/apache/arrow-datafusion/pull/7817#issuecomment-1762815092

   I tried to encode this conversation into comments


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org