tustvold commented on code in PR #6494: URL: https://github.com/apache/arrow-datafusion/pull/6494#discussion_r1211485548
########## datafusion/core/src/physical_plan/analyze.rs: ########## @@ -121,96 +118,102 @@ impl ExecutionPlan for AnalyzeExec { ))); } - // should be ensured by `SinglePartition` above - let input_partitions = self.input.output_partitioning().partition_count(); - if input_partitions != 1 { - return Err(DataFusionError::Internal(format!( - "AnalyzeExec invalid number of input partitions. Expected 1, got {input_partitions}" - ))); + // Gather futures that will run each input partition using a + // JoinSet to cancel outstanding futures on drop + let mut set = JoinSet::new(); + let num_input_partitions = self.input.output_partitioning().partition_count(); + + for input_partition in 0..num_input_partitions { + let input_stream = self.input.execute(input_partition, context.clone()); + + set.spawn(async move { + let mut total_rows = 0; + let mut input_stream = input_stream?; + while let Some(batch) = input_stream.next().await { + let batch = batch?; + total_rows += batch.num_rows(); + } + Ok(total_rows) + }); } - let (tx, rx) = tokio::sync::mpsc::channel(input_partitions); + // Turn the tasks in the JoinSet into a stream of + // Result<usize> representing the counts of each output + // partition. + let counts_stream = futures::stream::unfold(set, |mut set| async { + let next = set.join_next().await?; // returns Some when empty + // translate join errors (aka task panic's) into ExecutionErrors + let next = match next { + Ok(res) => res, + Err(e) => Err(DataFusionError::Execution(format!( + "Join error in AnalyzeExec: {e}" + ))), + }; + Some((next, set)) + }); + let start = Instant::now(); let captured_input = self.input.clone(); - let mut input_stream = captured_input.execute(0, context)?; let captured_schema = self.schema.clone(); let verbose = self.verbose; - // Task reads batches the input and when complete produce a - // RecordBatch with a report that is written to `tx` when done - let join_handle = tokio::task::spawn(async move { - let start = Instant::now(); - let mut total_rows = 0; - - // Note the code below ignores errors sending on tx. An - // error sending means the plan is being torn down and - // nothing is left that will handle the error (aka no one - // will hear us scream) - while let Some(b) = input_stream.next().await { - match b { - Ok(batch) => { - total_rows += batch.num_rows(); - } - b @ Err(_) => { - // try and pass on errors from input - if tx.send(b).await.is_err() { - // receiver hung up, stop executing (no - // one will look at any further results we - // send) - return; - } - } - } - } - let end = Instant::now(); + // future that gathers the input counts into an overall output + // count, and makes an output batch + let output = counts_stream Review Comment: FWIW you could just use a regular async move here, instead of needing the futures adapters ########## datafusion/core/src/physical_plan/analyze.rs: ########## @@ -121,96 +118,102 @@ impl ExecutionPlan for AnalyzeExec { ))); } - // should be ensured by `SinglePartition` above - let input_partitions = self.input.output_partitioning().partition_count(); - if input_partitions != 1 { - return Err(DataFusionError::Internal(format!( - "AnalyzeExec invalid number of input partitions. Expected 1, got {input_partitions}" - ))); + // Gather futures that will run each input partition using a + // JoinSet to cancel outstanding futures on drop + let mut set = JoinSet::new(); + let num_input_partitions = self.input.output_partitioning().partition_count(); + + for input_partition in 0..num_input_partitions { + let input_stream = self.input.execute(input_partition, context.clone()); + + set.spawn(async move { + let mut total_rows = 0; + let mut input_stream = input_stream?; + while let Some(batch) = input_stream.next().await { + let batch = batch?; + total_rows += batch.num_rows(); + } + Ok(total_rows) + }); } - let (tx, rx) = tokio::sync::mpsc::channel(input_partitions); + // Turn the tasks in the JoinSet into a stream of + // Result<usize> representing the counts of each output + // partition. + let counts_stream = futures::stream::unfold(set, |mut set| async { Review Comment: I confirmed that the tokio stream adapaters don't appear to have a JoinSet impl - https://docs.rs/tokio-stream/latest/tokio_stream/wrappers/index.html?search= -- 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-unsubscr...@arrow.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org