Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18836#discussion_r131557287
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
 ---
    @@ -82,7 +82,7 @@ case class SortMergeJoinExec(
     
       override def outputOrdering: Seq[SortOrder] = joinType match {
         // For inner join, orders of both sides keys should be kept.
    -    case Inner =>
    +    case _: InnerLike =>
    --- End diff --
    
    Even worse, this could cause an exception
    ```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)
    }
    ```
    It will return the following error:
    ```
    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)
        at org.apache.spark.sql.execution.ProjectExec
    ```
    
    We need to backport it to 2.2


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to