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

[GitHub] [arrow-datafusion] Dandandan opened a new pull request, #6811: Add fetch to sort preserving merge exec

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

   # 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 #6000
   
   # 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.  
   -->
   
   # 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


[GitHub] [arrow-datafusion] mustafasrepo commented on a diff in pull request #6811: Add fetch to `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/proto/src/physical_plan/mod.rs:
##########
@@ -692,7 +692,14 @@ impl AsExecutionPlan for PhysicalPlanNode {
                         }
                     })
                     .collect::<Result<Vec<_>, _>>()?;
-                Ok(Arc::new(SortPreservingMergeExec::new(exprs, input)))
+                let fetch = if sort.fetch == -1 {

Review Comment:
   ```suggestion
                   let fetch = if sort.fetch < 0 {
   ```
   Maybe this version is safer. Also `Sort` uses this paradigm



-- 
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] Dandandan commented on a diff in pull request #6811: Add fetch to sort `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_plan/sorts/merge.rs:
##########
@@ -227,11 +239,21 @@ impl<C: Cursor> SortPreservingMergeStream<C> {
             if self.advance(stream_idx) {
                 self.loser_tree_adjusted = false;
                 self.in_progress.push_row(stream_idx);
-                if self.in_progress.len() < self.batch_size {
+

Review Comment:
   Makes sense, otherwise we could maybe need two separate implementations (with/without fetch)



-- 
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] mustafasrepo commented on a diff in pull request #6811: Add fetch to `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_plan/sorts/merge.rs:
##########
@@ -140,6 +143,12 @@ struct SortPreservingMergeStream<C> {
 
     /// Vector that holds cursors for each non-exhausted input partition
     cursors: Vec<Option<C>>,
+
+    /// Optional number of rows to fetch
+    fetch: Option<usize>,
+
+    /// number of rows produces

Review Comment:
   ```suggestion
       /// number of rows produced
   ```



-- 
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] ozankabak commented on a diff in pull request #6811: Add fetch to sort `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_optimizer/sort_enforcement.rs:
##########
@@ -801,6 +803,22 @@ fn get_sort_exprs(sort_any: &Arc<dyn ExecutionPlan>) -> Result<&[PhysicalSortExp
     }
 }
 
+/// gets

Review Comment:
   I think you forgot to complete this comment 



-- 
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 #6811: Add fetch to `SortPreservingMergeExec` and `SortPreservingMergeStream`

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

   Thanks everyone -- this looks like it was a great team effort 👍 


-- 
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] Dandandan commented on a diff in pull request #6811: Add fetch to sort `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_plan/sorts/sort.rs:
##########
@@ -285,14 +286,13 @@ impl ExternalSorter {
             })
             .collect::<Result<_>>()?;
 
-        // TODO: Pushdown fetch to streaming merge (#6000)
-
         streaming_merge(
             streams,
             self.schema.clone(),
             &self.expr,
             metrics,
             self.batch_size,
+            self.fetch,

Review Comment:
   This was removed a few months ago to use `streaming_merge` in `SortExec` causing `SortExec` to spill even with a (low) `fetch`.



-- 
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] Dandandan merged pull request #6811: Add fetch to `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


-- 
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 #6811: Add fetch to sort `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_plan/sorts/merge.rs:
##########
@@ -227,11 +239,21 @@ impl<C: Cursor> SortPreservingMergeStream<C> {
             if self.advance(stream_idx) {
                 self.loser_tree_adjusted = false;
                 self.in_progress.push_row(stream_idx);
-                if self.in_progress.len() < self.batch_size {
+
+                // stop sorting if fetch has been reached
+                if self

Review Comment:
   Nit: this might be easier to read if you moved the "fetch_reached" into its own function
   
   like 
   
   ```rust
   if self.fetch_reached() {
     self.aborted = true;
   } else { ..
   ```



##########
datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs:
##########
@@ -71,6 +71,8 @@ pub struct SortPreservingMergeExec {
     expr: Vec<PhysicalSortExpr>,
     /// Execution metrics
     metrics: ExecutionPlanMetricsSet,
+    /// Optional number of rows to fetch

Review Comment:
   ```suggestion
       /// Optional number of rows to fetch. Stops producing rows after this fetch
   ```



##########
datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part:
##########
@@ -67,7 +67,7 @@ Limit: skip=0, fetch=10
 ------------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")]
 physical_plan
 GlobalLimitExec: skip=0, fetch=10
---SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST]
+--SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10

Review Comment:
   👍 



##########
datafusion/core/src/physical_plan/sorts/sort.rs:
##########
@@ -285,14 +286,13 @@ impl ExternalSorter {
             })
             .collect::<Result<_>>()?;
 
-        // TODO: Pushdown fetch to streaming merge (#6000)

Review Comment:
   🎉 



##########
datafusion/core/src/physical_plan/sorts/merge.rs:
##########
@@ -227,11 +239,21 @@ impl<C: Cursor> SortPreservingMergeStream<C> {
             if self.advance(stream_idx) {
                 self.loser_tree_adjusted = false;
                 self.in_progress.push_row(stream_idx);
-                if self.in_progress.len() < self.batch_size {
+

Review Comment:
   This is the hot path for SortPreservingMerge -- I suspect adding an extra check for fetch won't impact performance in any measurable way, but it might be worth while checking the benchmarks



-- 
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] Dandandan commented on a diff in pull request #6811: Add fetch to `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_plan/sorts/merge.rs:
##########
@@ -227,11 +239,21 @@ impl<C: Cursor> SortPreservingMergeStream<C> {
             if self.advance(stream_idx) {
                 self.loser_tree_adjusted = false;
                 self.in_progress.push_row(stream_idx);
-                if self.in_progress.len() < self.batch_size {
+

Review Comment:
   As benchmarks showed no significant difference, I kept this check in the hot path. 



-- 
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] Dandandan commented on a diff in pull request #6811: Add fetch to sort `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_optimizer/sort_enforcement.rs:
##########
@@ -801,6 +803,22 @@ fn get_sort_exprs(sort_any: &Arc<dyn ExecutionPlan>) -> Result<&[PhysicalSortExp
     }
 }
 
+/// gets

Review Comment:
   thanks



-- 
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] jychen7 commented on a diff in pull request #6811: Add fetch to sort `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_optimizer/sort_enforcement.rs:
##########


Review Comment:
   do we want to pass `sort_exec.fetch()` to the new `SortPreservingMergeExec` as well?
   
   https://github.com/apache/arrow-datafusion/blob/99170b942d91fb419d22faaadd72616bf59a0b87/datafusion/core/src/physical_optimizer/sort_enforcement.rs#L553-L558



-- 
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] Dandandan commented on a diff in pull request #6811: Add fetch to sort `SortPreservingMergeExec` and `SortPreservingMergeStream`

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


##########
datafusion/core/src/physical_plan/sorts/sort.rs:
##########
@@ -285,14 +286,13 @@ impl ExternalSorter {
             })
             .collect::<Result<_>>()?;
 
-        // TODO: Pushdown fetch to streaming merge (#6000)
-
         streaming_merge(
             streams,
             self.schema.clone(),
             &self.expr,
             metrics,
             self.batch_size,
+            self.fetch,

Review Comment:
   This was moved in a PR to use `streaming_merge` in `SortExec` causing `SortExec` to spill even with a (low) `fetch`.



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