You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "alamb (via GitHub)" <gi...@apache.org> on 2023/05/30 21:11:22 UTC

[GitHub] [arrow-datafusion] alamb opened a new pull request, #6494: Alamb/explain analyze

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

   # Which issue does this PR close?
   
   Closes https://github.com/apache/arrow-datafusion/pull/6380
   Closes https://github.com/apache/arrow-datafusion/issues/6379
   
   # Rationale for this change
   
   See https://github.com/apache/arrow-datafusion/issues/6379
   
   # What changes are included in this PR?
   This is based off https://github.com/apache/arrow-datafusion/pull/6380 from @tustvold but it was getting a little big to just push to his branch so I decided to make a new PR
   
   1. Don't optimize the input to `AnalyzeExec`
   2. Update `AnalyzeExec` to handle multiple input streams using `futures`-fu 
   3. Add a new test
   
   # Are these changes tested?
   Yes
   
   # Are there any user-facing changes?
   Less confusing `EXPLAIN ANALYZE` results


-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

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


##########
datafusion/core/tests/sql/explain_analyze.rs:
##########
@@ -687,13 +687,31 @@ async fn csv_explain_analyze() {
     // Only test basic plumbing and try to avoid having to change too
     // many things. explain_analyze_baseline_metrics covers the values
     // in greater depth
-    let needle = "CoalescePartitionsExec, metrics=[output_rows=5, elapsed_compute=";
+    let needle = "AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))], metrics=[output_rows=5";
     assert_contains!(&formatted, needle);
 
     let verbose_needle = "Output Rows";
     assert_not_contains!(formatted, verbose_needle);
 }
 
+#[tokio::test]
+#[cfg_attr(tarpaulin, ignore)]
+async fn csv_explain_analyze_order_by() {
+    let ctx = SessionContext::new();
+    register_aggregate_csv_by_sql(&ctx).await;

Review Comment:
   here is the new test



-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

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


##########
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();

Review Comment:
   this logic was just extracted into its own function



-- 
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


[GitHub] [arrow-datafusion] alamb merged pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

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


-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

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


##########
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();

Review Comment:
   This uses the cool `JoinSet` I learned about from @nvartolomei  and @Darksonn  on https://github.com/apache/arrow-datafusion/pull/6449 ❤️ 



-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

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


##########
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:
   That is an excellent point -- I got so carried away with being clever I lost sight of that.
   
   I rewrote this logic to use a single future and also moved the output record batch creation into a function to separate the business logic from the async orchestration: 5521a7031



-- 
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


[GitHub] [arrow-datafusion] tustvold commented on a diff in pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] tustvold commented on pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

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

   Test failures do not appear to be related


-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb commented on pull request #6494: Don't optimize AnalyzeExec (#6379) (try 2)

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

   Test failures I think are due to https://github.com/apache/arrow-datafusion/issues/6495


-- 
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