c21 opened a new pull request #34612:
URL: https://github.com/apache/spark/pull/34612


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: 
https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: 
https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., 
'[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a 
faster review.
     7. If you want to add a new configuration, please read the guideline first 
for naming configurations in
        
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the 
guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section 
is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster 
reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class 
hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other 
DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   FULL OUTER sort merge join (non-code-gen path) [copies join keys and buffers 
input rows, even when rows from both sides do have matched 
keys](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala#L1637-L1641).
 This is unnecessary, as we can just output the row with smaller join keys, and 
only buffer when both sides have matched keys. This would save us from 
unnecessary copy and buffer, when both join sides have a lot of rows not 
matched with each other.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   Improve query performance for FULL OUTER sort merge join when code-gen is 
disabled.
   This would benefit query when both sides have a lot of rows not matched, and 
join key is big in terms of size (e.g. string type).
   
   Example micro benchmark:
   
   ```  
     def sortMergeJoin(): Unit = {
       val N = 2 << 20
       codegenBenchmark("sort merge join", N) {
         val df1 = spark.range(N).selectExpr(s"cast(id * 15485863 as string) as 
k1")
         val df2 = spark.range(N).selectExpr(s"cast(id * 15485867 as string) as 
k2")
         val df = df1.join(df2, col("k1") === col("k2"), "full_outer")
         
assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined)
         df.noop()
       }
     }
   ```
   
   Seeing run-time improvement over 60%:
   
   ```
   Running benchmark: sort merge join
     Running case: sort merge join without optimization
     Stopped after 5 iterations, 10026 ms
     Running case: sort merge join with optimization
     Stopped after 5 iterations, 5954 ms
   
   Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.16
   Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
   sort merge join:                          Best Time(ms)   Avg Time(ms)   
Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
   
------------------------------------------------------------------------------------------------------------------------
   sort merge join without optimization               1807           2005       
  157          1.2         861.4       1.0X
   sort merge join with optimization                  1135           1191       
   62          1.8         541.1       1.6X
   ```
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as 
the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes 
- provide the console output, description and/or an example to show the 
behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to 
the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   No.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some 
test cases that check the changes thoroughly including negative and positive 
cases if possible.
   If it was tested in a way different from regular unit tests, please clarify 
how you tested step by step, ideally copy and paste-able, so that other 
reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why 
it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions 
for the consistent environment, and the instructions could accord to: 
https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   Existing unit tests e.g. `OuterJoinSuite.scala`.


-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to