alamb commented on code in PR #6494: URL: https://github.com/apache/arrow-datafusion/pull/6494#discussion_r1210832519
########## datafusion/core/src/physical_plan/analyze.rs: ########## @@ -121,96 +123,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 think looking at https://github.com/apache/arrow-datafusion/pull/6494/files?w=1 makes it clearer what I did -- which was to change the plumbing to use `future`s and `stream` fu rather than channels ########## datafusion/core/src/physical_optimizer/pipeline_checker.rs: ########## @@ -387,7 +387,7 @@ mod sql_tests { }; let test2 = UnaryTestCase { source_type: SourceType::Unbounded, - expect_fail: true, + expect_fail: false, Review Comment: it works now! -- 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