[GitHub] spark pull request #18836: Update SortMergeJoinExec.scala

2017-08-03 Thread BoleynSu
Github user BoleynSu commented on a diff in the pull request:

https://github.com/apache/spark/pull/18836#discussion_r131316095
  
--- 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 --

I think we can get a SortMergeJoin paln with Cross, e.g.  `select distinct 
a.i + 1,a.* from T a cross join T t where a.i > 1 and t.i = a.i group by a.i 
having a.i > 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



[GitHub] spark issue #18836: Update SortMergeJoinExec.scala

2017-08-03 Thread BoleynSu
Github user BoleynSu commented on the issue:

https://github.com/apache/spark/pull/18836
  
@gatorsmile I am not familiar with the PR process, it is great that you can 
take it over. Thanks.


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



[GitHub] spark issue #18836: Update SortMergeJoinExec.scala

2017-08-03 Thread BoleynSu
Github user BoleynSu commented on the issue:

https://github.com/apache/spark/pull/18836
  
A test case to make the existing code fail.
@srowen I am sorry that this pull request is not well formatted but I just 
want to help.
```scala
import org.apache.spark.sql.SparkSession

object Test extends App {
  val spark = 
SparkSession.builder().master("local").appName("test").getOrCreate()
  import spark.sqlContext.implicits._
  case class T(i: Int)
  spark.sparkContext.parallelize(List(T(1), T(3), 
T(3))).toDF.createOrReplaceTempView("T")
  val in = "select distinct a.i + 1,a.* from T a cross join T t where a.i > 
1 and t.i = a.i group by a.i having a.i > 2"
  val sql = spark.sql(in)
  sql.queryExecution.executedPlan.children.map { x =>
x.children.map { x =>
  x.children.map { x =>
x.children.map { x =>
  x.children.map { x =>
x.children.map { x =>
  println(x.outputOrdering)
}
  }
}
  }
}
  }
}
```


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



[GitHub] spark pull request #18836: Update SortMergeJoinExec.scala

2017-08-03 Thread BoleynSu
GitHub user BoleynSu opened a pull request:

https://github.com/apache/spark/pull/18836

Update SortMergeJoinExec.scala

fix a bug in outputOrdering

## What changes were proposed in this pull request?

Change `case Inner` to `case _: InnerLike` so that Cross will be handled 
properly.

## How was this patch tested?

No unit tests are needed.

Please review http://spark.apache.org/contributing.html before opening a 
pull request.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/BoleynSu/spark patch-1

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/18836.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #18836


commit 54ba6bedb36e57698404dbaccb7d5639bf64770c
Author: Boleyn Su 
Date:   2017-08-03T17:55:26Z

Update SortMergeJoinExec.scala

fix a bug in outputOrdering




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