[GitHub] spark pull request: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/9404#discussion_r43715526
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
@@ -194,12 +194,13 @@ case class Exchange(newPartitioning: Partitioning, 
child: SparkPlan) extends Una
  */
 private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends 
Rule[SparkPlan] {
   // TODO: Determine the number of partitions.
-  private def numPartitions: Int = sqlContext.conf.numShufflePartitions
+  private def defaultPartitions: Int = sqlContext.conf.numShufflePartitions
 
   /**
* Given a required distribution, returns a partitioning that satisfies 
that distribution.
*/
-  private def canonicalPartitioning(requiredDistribution: Distribution): 
Partitioning = {
+  private def createPartitioning(requiredDistribution: Distribution,
+  numPartitions: Int): Partitioning = {
--- End diff --

Actually, no need to change this format. I will change it in my pr.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153230502
  
LGTM. I am going to merge this. @nongli Can you have a follow-up to address 
those two minor comments? 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 pull request: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153222808
  
Merged build finished. Test PASSed.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153222811
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44855/
Test PASSed.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153222410
  
**[Test build #44855 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44855/consoleFull)**
 for PR 9404 at commit 
[`8428541`](https://github.com/apache/spark/commit/84285418b7fd91a5af7693a070bcad96a38d0438).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/9404#discussion_r43709146
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
@@ -194,12 +194,13 @@ case class Exchange(newPartitioning: Partitioning, 
child: SparkPlan) extends Una
  */
 private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends 
Rule[SparkPlan] {
   // TODO: Determine the number of partitions.
-  private def numPartitions: Int = sqlContext.conf.numShufflePartitions
+  private def defaultPartitions: Int = sqlContext.conf.numShufflePartitions
 
   /**
* Given a required distribution, returns a partitioning that satisfies 
that distribution.
*/
-  private def canonicalPartitioning(requiredDistribution: Distribution): 
Partitioning = {
+  private def createPartitioning(requiredDistribution: Distribution,
+  numPartitions: Int): Partitioning = {
--- End diff --

How about the following format?
```
private def createPartitioning(
requiredDistribution: Distribution,
numPartitions: Int): Partitioning = {
```


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/9404#discussion_r43709117
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala ---
@@ -353,4 +354,62 @@ class CachedTableSuite extends QueryTest with 
SharedSQLContext {
 assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e 
}.size === 3)
 assert(sparkPlan.collect { case e: PhysicalRDD => e }.size === 0)
   }
+
+  /**
+   * Verifies that the plan for `df` contains `expected` number of 
Exchange operators.
+   */
+  private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = {
+assert(df.queryExecution.executedPlan.collect { case e: Exchange => e 
}.size == expected)
+  }
+
+  test("A cached table preserves the partitioning and ordering of its 
cached SparkPlan") {
+val table3x = testData.unionAll(testData).unionAll(testData)
+table3x.registerTempTable("testData3x")
+
+sql("SELECT key, value FROM testData3x ORDER BY 
key").registerTempTable("orderedTable")
+sqlContext.cacheTable("orderedTable")
+assertCached(sqlContext.table("orderedTable"))
+// Should not have an exchange as the query is already sorted on the 
group by key.
+verifyNumExchanges(sql("SELECT key, count(*) FROM orderedTable GROUP 
BY key"), 0)
+checkAnswer(
+  sql("SELECT key, count(*) FROM orderedTable GROUP BY key ORDER BY 
key"),
+  sql("SELECT key, count(*) FROM testData3x GROUP BY key ORDER BY 
key").collect())
+sqlContext.uncacheTable("orderedTable")
+
+// Set up two tables distributed in the same way. Try this with the 
data distributed into
+// different number of partitions.
+for (numPartitions <- 1 until 10 by 4) {
+  testData.distributeBy(Column("key") :: Nil, 
numPartitions).registerTempTable("t1")
+  testData2.distributeBy(Column("a") :: Nil, 
numPartitions).registerTempTable("t2")
+  sqlContext.cacheTable("t1")
+  sqlContext.cacheTable("t2")
+
+  // Joining them should result in no exchanges.
+  verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = 
t2.a"), 0)
+  checkAnswer(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"),
+sql("SELECT * FROM testData t1 JOIN testData2 t2 ON t1.key = 
t2.a"))
+
+  // Grouping on the partition key should result in no exchanges
+  verifyNumExchanges(sql("SELECT count(*) FROM t1 GROUP BY key"), 0)
+  checkAnswer(sql("SELECT count(*) FROM t1 GROUP BY key"),
+sql("SELECT count(*) FROM testData GROUP BY key"))
+
+  sqlContext.uncacheTable("t1")
+  sqlContext.uncacheTable("t2")
+  sqlContext.dropTempTable("t1")
+  sqlContext.dropTempTable("t2")
+}
+
+// Distribute the tables into non-matching number of partitions. Need 
to shuffle.
+testData.distributeBy(Column("key") :: Nil, 6).registerTempTable("t1")
+testData2.distributeBy(Column("a") :: Nil, 3).registerTempTable("t2")
+sqlContext.cacheTable("t1")
+sqlContext.cacheTable("t2")
+
+verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = 
t2.a"), 2)
+sqlContext.uncacheTable("t1")
+sqlContext.uncacheTable("t2")
+sqlContext.dropTempTable("t1")
+sqlContext.dropTempTable("t2")
+  }
--- End diff --

Have a case to test that we will shuffle if one join column is not the one 
used in `distribute by`?


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/9404#discussion_r43708751
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
@@ -229,12 +230,33 @@ private[sql] case class 
EnsureRequirements(sqlContext: SQLContext) extends Rule[
 if (children.length > 1
 && requiredChildDistributions.toSet != Set(UnspecifiedDistribution)
 && 
!Partitioning.allCompatible(children.map(_.outputPartitioning))) {
-  children = children.zip(requiredChildDistributions).map { case 
(child, distribution) =>
-val targetPartitioning = canonicalPartitioning(distribution)
-if (child.outputPartitioning.guarantees(targetPartitioning)) {
-  child
-} else {
-  Exchange(targetPartitioning, child)
+
+  // First check if the existing partitions of the children all match. 
This means they are
+  // partitioned by the same partitioning into the same number of 
partitions. In that case,
+  // don't try to make them match `defaultPartitions`, just use the 
existing partitioning.
+  // TODO: this should be a cost based descision. For example, a big 
relation should probably
+  // maintain its existing number of partitions and smaller partitions 
should be shuffled.
+  // defaultPartitions is arbitrary.
+  val numPartitions = children.head.outputPartitioning.numPartitions
+  val useExistingPartitioning = 
children.zip(requiredChildDistributions).forall {
+case (child, distribution) => {
+  child.outputPartitioning.guarantees(
+createPartitioning(distribution, numPartitions))
+}
+  }
--- End diff --

Just a note at here. For a join, it is possible that the left table is 
range partitioned and the right table is hash-partitioned and they both 
satisfies the required distribution. However, `createPartitioning` will only 
create `HashPartitioning` for `ClusteredDistribution` and `guarantees` at here 
will fail if the output partitioning is `RangePartitioning`.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153203733
  
**[Test build #44855 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44855/consoleFull)**
 for PR 9404 at commit 
[`8428541`](https://github.com/apache/spark/commit/84285418b7fd91a5af7693a070bcad96a38d0438).


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153202279
  
Merged build started.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153202255
  
 Merged build triggered.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153144952
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44817/
Test PASSed.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153144949
  
Merged build finished. Test PASSed.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153144771
  
**[Test build #44817 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44817/consoleFull)**
 for PR 9404 at commit 
[`e711e05`](https://github.com/apache/spark/commit/e711e05d227f6e32738cdb429cfb961189765370).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/9404#discussion_r43662693
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala ---
@@ -353,4 +354,44 @@ class CachedTableSuite extends QueryTest with 
SharedSQLContext {
 assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e 
}.size === 3)
 assert(sparkPlan.collect { case e: PhysicalRDD => e }.size === 0)
   }
+
+  /**
+   * Verifies that the plan for `df` contains `expected` number of 
Exchange operators.
+   */
+  private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = {
+assert(df.queryExecution.executedPlan.collect { case e: Exchange => e 
}.size == expected)
+  }
+
+  test("A cached table preserves the partitioning and ordering of its 
cached SparkPlan") {
+val table3x = testData.unionAll(testData).unionAll(testData)
+table3x.registerTempTable("testData3x")
+
+sql("SELECT key, value FROM testData3x ORDER BY 
key").registerTempTable("orderedTable")
+sqlContext.cacheTable("orderedTable")
+assertCached(sqlContext.table("orderedTable"))
+// Should not have an exchange as the query is already sorted on the 
group by key.
+verifyNumExchanges(sql("SELECT key, count(*) FROM orderedTable GROUP 
BY key"), 0)
+checkAnswer(
+  sql("SELECT key, count(*) FROM orderedTable GROUP BY key ORDER BY 
key"),
+  sql("SELECT key, count(*) FROM testData3x GROUP BY key ORDER BY 
key").collect())
+sqlContext.uncacheTable("orderedTable")
+
+// Set up two tables distributed in the same way.
+testData.distributeBy(Column("key") :: Nil, 5).registerTempTable("t1")
+testData2.distributeBy(Column("a") :: Nil, 5).registerTempTable("t2")
+sqlContext.cacheTable("t1")
+sqlContext.cacheTable("t2")
+
+// Joining them should result in no exchanges.
+verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = 
t2.a"), 0)
--- End diff --

ah seems partitioning the data to `5` partitions does the trick at here 
(the default parallelism is set to 5 in our tests). If you change it tom 
something like `10`, this test will fail... Unfortunately, we do not have the 
concept of equivalent class right now. So, at 
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala#L229-L240,
 `allCompatible` method does not really do what we want at here (btw, 
`allCompatible` method is trying to make sure that partitioning schemes of all 
children are compatible with each other, i.e. making sure they partition the 
data with the same partitioner and with the same number of partitions).


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153104224
  
**[Test build #44817 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44817/consoleFull)**
 for PR 9404 at commit 
[`e711e05`](https://github.com/apache/spark/commit/e711e05d227f6e32738cdb429cfb961189765370).


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/9404#discussion_r43660128
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala ---
@@ -353,4 +354,44 @@ class CachedTableSuite extends QueryTest with 
SharedSQLContext {
 assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e 
}.size === 3)
 assert(sparkPlan.collect { case e: PhysicalRDD => e }.size === 0)
   }
+
+  /**
+   * Verifies that the plan for `df` contains `expected` number of 
Exchange operators.
+   */
+  private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = {
+assert(df.queryExecution.executedPlan.collect { case e: Exchange => e 
}.size == expected)
+  }
+
+  test("A cached table preserves the partitioning and ordering of its 
cached SparkPlan") {
+val table3x = testData.unionAll(testData).unionAll(testData)
+table3x.registerTempTable("testData3x")
+
+sql("SELECT key, value FROM testData3x ORDER BY 
key").registerTempTable("orderedTable")
+sqlContext.cacheTable("orderedTable")
+assertCached(sqlContext.table("orderedTable"))
+// Should not have an exchange as the query is already sorted on the 
group by key.
+verifyNumExchanges(sql("SELECT key, count(*) FROM orderedTable GROUP 
BY key"), 0)
+checkAnswer(
+  sql("SELECT key, count(*) FROM orderedTable GROUP BY key ORDER BY 
key"),
+  sql("SELECT key, count(*) FROM testData3x GROUP BY key ORDER BY 
key").collect())
+sqlContext.uncacheTable("orderedTable")
+
+// Set up two tables distributed in the same way.
+testData.distributeBy(Column("key") :: Nil, 5).registerTempTable("t1")
+testData2.distributeBy(Column("a") :: Nil, 5).registerTempTable("t2")
+sqlContext.cacheTable("t1")
+sqlContext.cacheTable("t2")
+
+// Joining them should result in no exchanges.
+verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = 
t2.a"), 0)
+
+// Grouping on the partition key should result in no exchanges
+verifyNumExchanges(sql("SELECT count(*) FROM t1 GROUP BY key"), 0)
+
+// TODO: this is an issue with self joins. The number of exchanges 
should be 0.
+verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t1 t2 on t1.key = 
t2.key"), 1)
+
+sqlContext.uncacheTable("t1")
+sqlContext.uncacheTable("t2")
+  }
--- End diff --

How about we also check the results?


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153100484
  
Merged build started.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153100427
  
 Merged build triggered.


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153099409
  
test this please


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153099394
  
ok to test


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-02 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-153099382
  
add to whitelist


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-01 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/9404#issuecomment-152917550
  
Can one of the admins verify this patch?


---
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: [SPARK-5354] [SQL] Cached tables should preser...

2015-11-01 Thread nongli
GitHub user nongli opened a pull request:

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

[SPARK-5354] [SQL] Cached tables should preserve partitioning and ord…

…ering.

For cached tables, we can just maintain the partitioning and ordering from 
the
source relation.

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

$ git pull https://github.com/nongli/spark spark-5354

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

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


commit e711e05d227f6e32738cdb429cfb961189765370
Author: Nong Li 
Date:   2015-10-29T18:12:10Z

[SPARK-5354] [SQL] Cached tables should preserve partitioning and ordering.

For cached tables, we can just maintain the partitioning and ordering from 
the
source relation.




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