Skip to content

Conversation

@gatorsmile
Copy link
Member

What changes were proposed in this pull request?

author: BoleynSu
closes #18836

val df = Seq((1, 1)).toDF("i", "j")
df.createOrReplaceTempView("T")
withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
  sql("select * from (select a.i from T a cross join T t where t.i = a.i) as t1 " +
    "cross join T t2 where t2.i = t1.i").explain(true)
}

The above code could cause the following exception:

SortMergeJoinExec should not take Cross as the JoinType
java.lang.IllegalArgumentException: SortMergeJoinExec should not take Cross as the JoinType
	at org.apache.spark.sql.execution.joins.SortMergeJoinExec.outputOrdering(SortMergeJoinExec.scala:100)

Our SortMergeJoinExec supports CROSS. We should not hit such an exception. This PR is to fix the issue.

How was this patch tested?

Modified the two existing test cases.

BoleynSu and others added 2 commits August 4, 2017 01:55
@gatorsmile
Copy link
Member Author

cc @cloud-fan @BoleynSu @hvanhovell

@SparkQA
Copy link

SparkQA commented Aug 7, 2017

Test build #80313 has finished for PR 18863 at commit f351fb1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

asfgit pushed a commit that referenced this pull request Aug 7, 2017
### What changes were proposed in this pull request?
author: BoleynSu
closes #18836

```Scala
val df = Seq((1, 1)).toDF("i", "j")
df.createOrReplaceTempView("T")
withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
  sql("select * from (select a.i from T a cross join T t where t.i = a.i) as t1 " +
    "cross join T t2 where t2.i = t1.i").explain(true)
}
```
The above code could cause the following exception:
```
SortMergeJoinExec should not take Cross as the JoinType
java.lang.IllegalArgumentException: SortMergeJoinExec should not take Cross as the JoinType
	at org.apache.spark.sql.execution.joins.SortMergeJoinExec.outputOrdering(SortMergeJoinExec.scala:100)
```

Our SortMergeJoinExec supports CROSS. We should not hit such an exception. This PR is to fix the issue.

### How was this patch tested?
Modified the two existing test cases.

Author: Xiao Li <[email protected]>
Author: Boleyn Su <[email protected]>

Closes #18863 from gatorsmile/pr-18836.

(cherry picked from commit bbfd6b5)
Signed-off-by: Wenchen Fan <[email protected]>
@cloud-fan
Copy link
Contributor

LGTM, merging to master/2.2

@asfgit asfgit closed this in bbfd6b5 Aug 7, 2017
MatthewRBruce pushed a commit to Shopify/spark that referenced this pull request Jul 31, 2018
### What changes were proposed in this pull request?
author: BoleynSu
closes apache#18836

```Scala
val df = Seq((1, 1)).toDF("i", "j")
df.createOrReplaceTempView("T")
withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
  sql("select * from (select a.i from T a cross join T t where t.i = a.i) as t1 " +
    "cross join T t2 where t2.i = t1.i").explain(true)
}
```
The above code could cause the following exception:
```
SortMergeJoinExec should not take Cross as the JoinType
java.lang.IllegalArgumentException: SortMergeJoinExec should not take Cross as the JoinType
	at org.apache.spark.sql.execution.joins.SortMergeJoinExec.outputOrdering(SortMergeJoinExec.scala:100)
```

Our SortMergeJoinExec supports CROSS. We should not hit such an exception. This PR is to fix the issue.

### How was this patch tested?
Modified the two existing test cases.

Author: Xiao Li <[email protected]>
Author: Boleyn Su <[email protected]>

Closes apache#18863 from gatorsmile/pr-18836.

(cherry picked from commit bbfd6b5)
Signed-off-by: Wenchen Fan <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants