spark git commit: [SPARK-9141] [SQL] Remove project collapsing from DataFrame API

2015-08-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 34dcf1010 - 23d982204


[SPARK-9141] [SQL] Remove project collapsing from DataFrame API

Currently we collapse successive projections that are added by `withColumn`.  
However, this optimization violates the constraint that adding nodes to a plan 
will never change its analyzed form and thus breaks caching.  Instead of doing 
early optimization, in this PR I just fix some low-hanging slowness in the 
analyzer.  In particular, I add a mechanism for skipping already analyzed 
subplans, `resolveOperators` and `resolveExpression`.  Since trees are 
generally immutable after construction, it's safe to annotate a plan as already 
analyzed as any transformation will create a new tree with this bit no longer 
set.

Together these result in a faster analyzer than before, even with added timing 
instrumentation.

```
Original Code
[info] 3430ms
[info] 2205ms
[info] 1973ms
[info] 1982ms
[info] 1916ms

Without Project Collapsing in DataFrame
[info] 44610ms
[info] 45977ms
[info] 46423ms
[info] 46306ms
[info] 54723ms

With analyzer optimizations
[info] 6394ms
[info] 4630ms
[info] 4388ms
[info] 4093ms
[info] 4113ms

With resolveOperators
[info] 2495ms
[info] 1380ms
[info] 1685ms
[info] 1414ms
[info] 1240ms
```

Author: Michael Armbrust mich...@databricks.com

Closes #7920 from marmbrus/withColumnCache and squashes the following commits:

2145031 [Michael Armbrust] fix hive udfs tests
5a5a525 [Michael Armbrust] remove wrong comment
7a507d5 [Michael Armbrust] style
b59d710 [Michael Armbrust] revert small change
1fa5949 [Michael Armbrust] move logic into LogicalPlan, add tests
0e2cb43 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into 
withColumnCache
c926e24 [Michael Armbrust] naming
e593a2d [Michael Armbrust] style
f5a929e [Michael Armbrust] [SPARK-9141][SQL] Remove project collapsing from 
DataFrame API
38b1c83 [Michael Armbrust] WIP


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/23d98220
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/23d98220
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/23d98220

Branch: refs/heads/master
Commit: 23d982204bb9ef74d3b788a32ce6608116968719
Parents: 34dcf10
Author: Michael Armbrust mich...@databricks.com
Authored: Wed Aug 5 09:01:45 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Aug 5 09:01:45 2015 -0700

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 28 
 .../sql/catalyst/analysis/CheckAnalysis.scala   |  3 +
 .../catalyst/analysis/HiveTypeCoercion.scala| 30 
 .../spark/sql/catalyst/plans/QueryPlan.scala|  5 +-
 .../catalyst/plans/logical/LogicalPlan.scala| 51 +-
 .../spark/sql/catalyst/rules/RuleExecutor.scala | 22 ++
 .../spark/sql/catalyst/trees/TreeNode.scala | 64 -
 .../sql/catalyst/plans/LogicalPlanSuite.scala   | 72 
 .../scala/org/apache/spark/sql/DataFrame.scala  |  6 +-
 .../apache/spark/sql/execution/SparkPlan.scala  |  2 +-
 .../apache/spark/sql/execution/pythonUDFs.scala |  2 +-
 .../org/apache/spark/sql/CachedTableSuite.scala | 20 ++
 .../org/apache/spark/sql/DataFrameSuite.scala   | 12 
 .../hive/execution/HiveCompatibilitySuite.scala |  5 ++
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  5 +-
 .../org/apache/spark/sql/hive/hiveUDFs.scala|  3 +-
 .../spark/sql/hive/execution/HiveUDFSuite.scala |  8 +--
 .../sql/hive/execution/SQLQuerySuite.scala  |  2 +
 18 files changed, 234 insertions(+), 106 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/23d98220/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index ca17f3e..6de31f4 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -90,7 +90,7 @@ class Analyzer(
*/
   object CTESubstitution extends Rule[LogicalPlan] {
 // TODO allow subquery to define CTE
-def apply(plan: LogicalPlan): LogicalPlan = plan transform  {
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators  {
   case With(child, relations) = substituteCTE(child, relations)
   case other = other
 }
@@ -116,7 +116,7 @@ class Analyzer(
* Substitute child plan with WindowSpecDefinitions.
*/
   object WindowsSubstitution extends Rule[LogicalPlan] {
-def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+def apply(plan: LogicalPlan): LogicalPlan = plan 

spark git commit: [SPARK-9141] [SQL] Remove project collapsing from DataFrame API

2015-08-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 eedb996dd - 125827a4f


[SPARK-9141] [SQL] Remove project collapsing from DataFrame API

Currently we collapse successive projections that are added by `withColumn`.  
However, this optimization violates the constraint that adding nodes to a plan 
will never change its analyzed form and thus breaks caching.  Instead of doing 
early optimization, in this PR I just fix some low-hanging slowness in the 
analyzer.  In particular, I add a mechanism for skipping already analyzed 
subplans, `resolveOperators` and `resolveExpression`.  Since trees are 
generally immutable after construction, it's safe to annotate a plan as already 
analyzed as any transformation will create a new tree with this bit no longer 
set.

Together these result in a faster analyzer than before, even with added timing 
instrumentation.

```
Original Code
[info] 3430ms
[info] 2205ms
[info] 1973ms
[info] 1982ms
[info] 1916ms

Without Project Collapsing in DataFrame
[info] 44610ms
[info] 45977ms
[info] 46423ms
[info] 46306ms
[info] 54723ms

With analyzer optimizations
[info] 6394ms
[info] 4630ms
[info] 4388ms
[info] 4093ms
[info] 4113ms

With resolveOperators
[info] 2495ms
[info] 1380ms
[info] 1685ms
[info] 1414ms
[info] 1240ms
```

Author: Michael Armbrust mich...@databricks.com

Closes #7920 from marmbrus/withColumnCache and squashes the following commits:

2145031 [Michael Armbrust] fix hive udfs tests
5a5a525 [Michael Armbrust] remove wrong comment
7a507d5 [Michael Armbrust] style
b59d710 [Michael Armbrust] revert small change
1fa5949 [Michael Armbrust] move logic into LogicalPlan, add tests
0e2cb43 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into 
withColumnCache
c926e24 [Michael Armbrust] naming
e593a2d [Michael Armbrust] style
f5a929e [Michael Armbrust] [SPARK-9141][SQL] Remove project collapsing from 
DataFrame API
38b1c83 [Michael Armbrust] WIP

(cherry picked from commit 23d982204bb9ef74d3b788a32ce6608116968719)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/125827a4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/125827a4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/125827a4

Branch: refs/heads/branch-1.5
Commit: 125827a4fa00bf904d2f6db6db9f86bee82c4829
Parents: eedb996
Author: Michael Armbrust mich...@databricks.com
Authored: Wed Aug 5 09:01:45 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Aug 5 09:01:56 2015 -0700

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 28 
 .../sql/catalyst/analysis/CheckAnalysis.scala   |  3 +
 .../catalyst/analysis/HiveTypeCoercion.scala| 30 
 .../spark/sql/catalyst/plans/QueryPlan.scala|  5 +-
 .../catalyst/plans/logical/LogicalPlan.scala| 51 +-
 .../spark/sql/catalyst/rules/RuleExecutor.scala | 22 ++
 .../spark/sql/catalyst/trees/TreeNode.scala | 64 -
 .../sql/catalyst/plans/LogicalPlanSuite.scala   | 72 
 .../scala/org/apache/spark/sql/DataFrame.scala  |  6 +-
 .../apache/spark/sql/execution/SparkPlan.scala  |  2 +-
 .../apache/spark/sql/execution/pythonUDFs.scala |  2 +-
 .../org/apache/spark/sql/CachedTableSuite.scala | 20 ++
 .../org/apache/spark/sql/DataFrameSuite.scala   | 12 
 .../hive/execution/HiveCompatibilitySuite.scala |  5 ++
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  5 +-
 .../org/apache/spark/sql/hive/hiveUDFs.scala|  3 +-
 .../spark/sql/hive/execution/HiveUDFSuite.scala |  8 +--
 .../sql/hive/execution/SQLQuerySuite.scala  |  2 +
 18 files changed, 234 insertions(+), 106 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/125827a4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index ca17f3e..6de31f4 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -90,7 +90,7 @@ class Analyzer(
*/
   object CTESubstitution extends Rule[LogicalPlan] {
 // TODO allow subquery to define CTE
-def apply(plan: LogicalPlan): LogicalPlan = plan transform  {
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators  {
   case With(child, relations) = substituteCTE(child, relations)
   case other = other
 }
@@ -116,7 +116,7 @@ class Analyzer(
* Substitute child plan with WindowSpecDefinitions.
*/
   object WindowsSubstitution extends 

spark git commit: [SPARK-9141] [SQL] [MINOR] Fix comments of PR #7920

2015-08-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 7a969a696 - 1f8c364b9


[SPARK-9141] [SQL] [MINOR] Fix comments of PR #7920

This is a follow-up of https://github.com/apache/spark/pull/7920 to fix 
comments.

Author: Yin Huai yh...@databricks.com

Closes #7964 from yhuai/SPARK-9141-follow-up and squashes the following commits:

4d0ee80 [Yin Huai] Fix comments.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1f8c364b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1f8c364b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1f8c364b

Branch: refs/heads/master
Commit: 1f8c364b9c6636f06986f5f80d5a49b7a7772ac3
Parents: 7a969a6
Author: Yin Huai yh...@databricks.com
Authored: Wed Aug 5 11:03:02 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Aug 5 11:03:02 2015 -0700

--
 .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala   | 3 +--
 .../org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala| 3 ++-
 2 files changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1f8c364b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 6de31f4..82158e6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -140,7 +140,7 @@ class Analyzer(
   object ResolveAliases extends Rule[LogicalPlan] {
 private def assignAliases(exprs: Seq[NamedExpression]) = {
   // The `UnresolvedAlias`s will appear only at root of a expression tree, 
we don't need
-  // to resolveOperator down the whole tree.
+  // to traverse the whole tree.
   exprs.zipWithIndex.map {
 case (u @ UnresolvedAlias(child), i) =
   child match {
@@ -873,7 +873,6 @@ class Analyzer(
 // Aggregate with Having clause will be triggered.
 def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
 
-
   // Aggregate with Having clause. This rule works with an unresolved 
Aggregate because
   // a resolved Aggregate will not have Window Functions.
   case f @ Filter(condition, a @ Aggregate(groupingExprs, aggregateExprs, 
child))

http://git-wip-us.apache.org/repos/asf/spark/blob/1f8c364b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
index 797b29f..455a381 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
@@ -23,7 +23,8 @@ import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.util._
 
 /**
- * Provides helper methods for comparing plans.
+ * This suite is used to test [[LogicalPlan]]'s `resolveOperators` and make 
sure it can correctly
+ * skips sub-trees that have already been marked as analyzed.
  */
 class LogicalPlanSuite extends SparkFunSuite {
   private var invocationCount = 0


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



spark git commit: [SPARK-9141] [SQL] [MINOR] Fix comments of PR #7920

2015-08-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 03bcf627d - 19018d542


[SPARK-9141] [SQL] [MINOR] Fix comments of PR #7920

This is a follow-up of https://github.com/apache/spark/pull/7920 to fix 
comments.

Author: Yin Huai yh...@databricks.com

Closes #7964 from yhuai/SPARK-9141-follow-up and squashes the following commits:

4d0ee80 [Yin Huai] Fix comments.

(cherry picked from commit 1f8c364b9c6636f06986f5f80d5a49b7a7772ac3)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/19018d54
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/19018d54
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/19018d54

Branch: refs/heads/branch-1.5
Commit: 19018d542589af58db8fc16b05d1e77e7cb71585
Parents: 03bcf62
Author: Yin Huai yh...@databricks.com
Authored: Wed Aug 5 11:03:02 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Aug 5 11:03:12 2015 -0700

--
 .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala   | 3 +--
 .../org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala| 3 ++-
 2 files changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/19018d54/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 6de31f4..82158e6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -140,7 +140,7 @@ class Analyzer(
   object ResolveAliases extends Rule[LogicalPlan] {
 private def assignAliases(exprs: Seq[NamedExpression]) = {
   // The `UnresolvedAlias`s will appear only at root of a expression tree, 
we don't need
-  // to resolveOperator down the whole tree.
+  // to traverse the whole tree.
   exprs.zipWithIndex.map {
 case (u @ UnresolvedAlias(child), i) =
   child match {
@@ -873,7 +873,6 @@ class Analyzer(
 // Aggregate with Having clause will be triggered.
 def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
 
-
   // Aggregate with Having clause. This rule works with an unresolved 
Aggregate because
   // a resolved Aggregate will not have Window Functions.
   case f @ Filter(condition, a @ Aggregate(groupingExprs, aggregateExprs, 
child))

http://git-wip-us.apache.org/repos/asf/spark/blob/19018d54/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
index 797b29f..455a381 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
@@ -23,7 +23,8 @@ import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.util._
 
 /**
- * Provides helper methods for comparing plans.
+ * This suite is used to test [[LogicalPlan]]'s `resolveOperators` and make 
sure it can correctly
+ * skips sub-trees that have already been marked as analyzed.
  */
 class LogicalPlanSuite extends SparkFunSuite {
   private var invocationCount = 0


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



spark git commit: [SPARK-9649] Fix flaky test MasterSuite - randomize ports

2015-08-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master eb5b8f4a6 - 5f0fb6466


[SPARK-9649] Fix flaky test MasterSuite - randomize ports

```
Error Message

Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 
retries!
Stacktrace

  java.net.BindException: Failed to bind to: /127.0.0.1:7093: Service 
'sparkMaster' failed after 16 retries!
  at 
org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:272)
  at 
akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:393)
  at 
akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:389)
  at scala.util.Success$$anonfun$map$1.apply(Try.scala:206)
  at scala.util.Try$.apply(Try.scala:161)
```

Author: Andrew Or and...@databricks.com

Closes #7968 from andrewor14/fix-master-flaky-test and squashes the following 
commits:

fcc42ef [Andrew Or] Randomize port


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5f0fb646
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5f0fb646
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5f0fb646

Branch: refs/heads/master
Commit: 5f0fb6466f5e3607f7fca9b2371a73b3deef3fdf
Parents: eb5b8f4
Author: Andrew Or and...@databricks.com
Authored: Wed Aug 5 14:12:22 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Aug 5 14:12:22 2015 -0700

--
 .../scala/org/apache/spark/deploy/master/MasterSuite.scala   | 8 
 1 file changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5f0fb646/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala 
b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
index 30780a0..ae0e037 100644
--- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
@@ -93,8 +93,8 @@ class MasterSuite extends SparkFunSuite with Matchers with 
Eventually with Priva
   publicAddress = 
 )
 
-val (rpcEnv, uiPort, restPort) =
-  Master.startRpcEnvAndEndpoint(127.0.0.1, 7077, 8080, conf)
+val (rpcEnv, _, _) =
+  Master.startRpcEnvAndEndpoint(127.0.0.1, 0, 0, conf)
 
 try {
   rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, rpcEnv.address, 
Master.ENDPOINT_NAME)
@@ -343,8 +343,8 @@ class MasterSuite extends SparkFunSuite with Matchers with 
Eventually with Priva
 
   private def makeMaster(conf: SparkConf = new SparkConf): Master = {
 val securityMgr = new SecurityManager(conf)
-val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, localhost, 7077, conf, 
securityMgr)
-val master = new Master(rpcEnv, rpcEnv.address, 8080, securityMgr, conf)
+val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, localhost, 0, conf, 
securityMgr)
+val master = new Master(rpcEnv, rpcEnv.address, 0, securityMgr, conf)
 master
   }
 


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



spark git commit: [SPARK-9649] Fix flaky test MasterSuite - randomize ports

2015-08-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 b8136d7e0 - 05cbf133d


[SPARK-9649] Fix flaky test MasterSuite - randomize ports

```
Error Message

Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 
retries!
Stacktrace

  java.net.BindException: Failed to bind to: /127.0.0.1:7093: Service 
'sparkMaster' failed after 16 retries!
  at 
org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:272)
  at 
akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:393)
  at 
akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:389)
  at scala.util.Success$$anonfun$map$1.apply(Try.scala:206)
  at scala.util.Try$.apply(Try.scala:161)
```

Author: Andrew Or and...@databricks.com

Closes #7968 from andrewor14/fix-master-flaky-test and squashes the following 
commits:

fcc42ef [Andrew Or] Randomize port

(cherry picked from commit 5f0fb6466f5e3607f7fca9b2371a73b3deef3fdf)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/05cbf133
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/05cbf133
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/05cbf133

Branch: refs/heads/branch-1.5
Commit: 05cbf133d8d12904532353118a7de42fd0a7f155
Parents: b8136d7
Author: Andrew Or and...@databricks.com
Authored: Wed Aug 5 14:12:22 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Aug 5 14:12:31 2015 -0700

--
 .../scala/org/apache/spark/deploy/master/MasterSuite.scala   | 8 
 1 file changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/05cbf133/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala 
b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
index 30780a0..ae0e037 100644
--- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
@@ -93,8 +93,8 @@ class MasterSuite extends SparkFunSuite with Matchers with 
Eventually with Priva
   publicAddress = 
 )
 
-val (rpcEnv, uiPort, restPort) =
-  Master.startRpcEnvAndEndpoint(127.0.0.1, 7077, 8080, conf)
+val (rpcEnv, _, _) =
+  Master.startRpcEnvAndEndpoint(127.0.0.1, 0, 0, conf)
 
 try {
   rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, rpcEnv.address, 
Master.ENDPOINT_NAME)
@@ -343,8 +343,8 @@ class MasterSuite extends SparkFunSuite with Matchers with 
Eventually with Priva
 
   private def makeMaster(conf: SparkConf = new SparkConf): Master = {
 val securityMgr = new SecurityManager(conf)
-val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, localhost, 7077, conf, 
securityMgr)
-val master = new Master(rpcEnv, rpcEnv.address, 8080, securityMgr, conf)
+val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, localhost, 0, conf, 
securityMgr)
+val master = new Master(rpcEnv, rpcEnv.address, 0, securityMgr, conf)
 master
   }
 


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



spark git commit: [SPARK-9361] [SQL] Refactor new aggregation code to reduce the times of checking compatibility

2015-07-30 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 7bbf02f0b - 5363ed715


[SPARK-9361] [SQL] Refactor new aggregation code to reduce the times of 
checking compatibility

JIRA: https://issues.apache.org/jira/browse/SPARK-9361

Currently, we call `aggregate.Utils.tryConvert` in many places to check it the 
logical.Aggregate can be run with new aggregation. But looks like 
`aggregate.Utils.tryConvert` will cost considerable time to run. We should only 
call `tryConvert` once and keep it value in `logical.Aggregate` and reuse it.

In `org.apache.spark.sql.execution.aggregate.Utils`, the codes involving with 
`tryConvert` should be moved to catalyst because it actually doesn't deal with 
execution details.

Author: Liang-Chi Hsieh vii...@appier.com

Closes #7677 from viirya/refactor_aggregate and squashes the following commits:

babea30 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into 
refactor_aggregate
9a589d7 [Liang-Chi Hsieh] Fix scala style.
0a91329 [Liang-Chi Hsieh] Refactor new aggregation code to reduce the times to 
call tryConvert.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5363ed71
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5363ed71
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5363ed71

Branch: refs/heads/master
Commit: 5363ed71568c3e7c082146d654a9c669d692d894
Parents: 7bbf02f
Author: Liang-Chi Hsieh vii...@appier.com
Authored: Thu Jul 30 10:30:37 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Thu Jul 30 10:32:12 2015 -0700

--
 .../expressions/aggregate/interfaces.scala  |   4 +-
 .../catalyst/expressions/aggregate/utils.scala  | 167 +++
 .../catalyst/plans/logical/basicOperators.scala |   3 +
 .../spark/sql/execution/SparkStrategies.scala   |  34 ++--
 .../spark/sql/execution/aggregate/utils.scala   | 144 
 5 files changed, 188 insertions(+), 164 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5363ed71/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
index 9fb7623..d08f553 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
@@ -42,7 +42,7 @@ private[sql] case object Partial extends AggregateMode
 private[sql] case object PartialMerge extends AggregateMode
 
 /**
- * An [[AggregateFunction2]] with [[PartialMerge]] mode is used to merge 
aggregation buffers
+ * An [[AggregateFunction2]] with [[Final]] mode is used to merge aggregation 
buffers
  * containing intermediate results for this function and then generate final 
result.
  * This function updates the given aggregation buffer by merging multiple 
aggregation buffers.
  * When it has processed all input rows, the final result of this function is 
returned.
@@ -50,7 +50,7 @@ private[sql] case object PartialMerge extends AggregateMode
 private[sql] case object Final extends AggregateMode
 
 /**
- * An [[AggregateFunction2]] with [[Partial]] mode is used to evaluate this 
function directly
+ * An [[AggregateFunction2]] with [[Complete]] mode is used to evaluate this 
function directly
  * from original input rows without any partial aggregation.
  * This function updates the given aggregation buffer with the original input 
of this
  * function. When it has processed all input rows, the final result of this 
function is returned.

http://git-wip-us.apache.org/repos/asf/spark/blob/5363ed71/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/utils.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/utils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/utils.scala
new file mode 100644
index 000..4a43318
--- /dev/null
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/utils.scala
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the 

spark git commit: [SPARK-9785] [SQL] HashPartitioning compatibility should consider expression ordering

2015-08-11 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master d378396f8 - dfe347d2c


[SPARK-9785] [SQL] HashPartitioning compatibility should consider expression 
ordering

HashPartitioning compatibility is currently defined w.r.t the _set_ of 
expressions, but the ordering of those expressions matters when computing hash 
codes; this could lead to incorrect answers if we mistakenly avoided a shuffle 
based on the assumption that HashPartitionings with the same expressions in 
different orders will produce equivalent row hashcodes. The first commit adds a 
regression test which illustrates this problem.

The fix for this is simple: make `HashPartitioning.compatibleWith` and 
`HashPartitioning.guarantees` sensitive to the expression ordering (i.e. do not 
perform set comparison).

Author: Josh Rosen joshro...@databricks.com

Closes #8074 from JoshRosen/hashpartitioning-compatiblewith-fixes and squashes 
the following commits:

b61412f [Josh Rosen] Demonstrate that I haven't cheated in my fix
0b4d7d9 [Josh Rosen] Update so that clusteringSet is only used in satisfies().
dc9c9d7 [Josh Rosen] Add failing regression test for SPARK-9785


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dfe347d2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dfe347d2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dfe347d2

Branch: refs/heads/master
Commit: dfe347d2cae3eb05d7539aaf72db3d309e711213
Parents: d378396
Author: Josh Rosen joshro...@databricks.com
Authored: Tue Aug 11 08:52:15 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Tue Aug 11 08:52:15 2015 -0700

--
 .../catalyst/plans/physical/partitioning.scala  | 15 ++
 .../spark/sql/catalyst/PartitioningSuite.scala  | 55 
 2 files changed, 60 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/dfe347d2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index 5a89a90..5ac3f1f 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -216,26 +216,23 @@ case class HashPartitioning(expressions: Seq[Expression], 
numPartitions: Int)
   override def nullable: Boolean = false
   override def dataType: DataType = IntegerType
 
-  lazy val clusteringSet = expressions.toSet
-
   override def satisfies(required: Distribution): Boolean = required match {
 case UnspecifiedDistribution = true
 case ClusteredDistribution(requiredClustering) =
-  clusteringSet.subsetOf(requiredClustering.toSet)
+  expressions.toSet.subsetOf(requiredClustering.toSet)
 case _ = false
   }
 
   override def compatibleWith(other: Partitioning): Boolean = other match {
-case o: HashPartitioning =
-  this.clusteringSet == o.clusteringSet  this.numPartitions == 
o.numPartitions
+case o: HashPartitioning = this == o
 case _ = false
   }
 
   override def guarantees(other: Partitioning): Boolean = other match {
-case o: HashPartitioning =
-  this.clusteringSet == o.clusteringSet  this.numPartitions == 
o.numPartitions
+case o: HashPartitioning = this == o
 case _ = false
   }
+
 }
 
 /**
@@ -257,15 +254,13 @@ case class RangePartitioning(ordering: Seq[SortOrder], 
numPartitions: Int)
   override def nullable: Boolean = false
   override def dataType: DataType = IntegerType
 
-  private[this] lazy val clusteringSet = ordering.map(_.child).toSet
-
   override def satisfies(required: Distribution): Boolean = required match {
 case UnspecifiedDistribution = true
 case OrderedDistribution(requiredOrdering) =
   val minSize = Seq(requiredOrdering.size, ordering.size).min
   requiredOrdering.take(minSize) == ordering.take(minSize)
 case ClusteredDistribution(requiredClustering) =
-  clusteringSet.subsetOf(requiredClustering.toSet)
+  ordering.map(_.child).toSet.subsetOf(requiredClustering.toSet)
 case _ = false
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/dfe347d2/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/PartitioningSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/PartitioningSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/PartitioningSuite.scala
new file mode 100644
index 000..5b802cc
--- /dev/null
+++ 

[2/2] spark git commit: [SPARK-9646] [SQL] Add metrics for all join and aggregate operators

2015-08-11 Thread yhuai
[SPARK-9646] [SQL] Add metrics for all join and aggregate operators

This PR added metrics for all join and aggregate operators. However, I found 
the metrics may be confusing in the following two case:
1. The iterator is not totally consumed and the metric values will be less.
2. Recreating the iterators will make metric values look bigger than the size 
of the input source, such as `CartesianProduct`.

Author: zsxwing zsxw...@gmail.com

Closes #8060 from zsxwing/sql-metrics and squashes the following commits:

40f3fc1 [zsxwing] Mark LongSQLMetric private[metric] to avoid using incorrectly 
and leak memory
b1b9071 [zsxwing] Merge branch 'master' into sql-metrics
4bef25a [zsxwing] Add metrics for SortMergeOuterJoin
95ccfc6 [zsxwing] Merge branch 'master' into sql-metrics
67cb4dd [zsxwing] Add metrics for Project and TungstenProject; remove metrics 
from PhysicalRDD and LocalTableScan
0eb47d4 [zsxwing] Merge branch 'master' into sql-metrics
dd9d932 [zsxwing] Avoid creating new Iterators
589ea26 [zsxwing] Add metrics for all join and aggregate operators

(cherry picked from commit 5831294a7a8fa2524133c5d718cbc8187d2b0620)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/767ee188
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/767ee188
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/767ee188

Branch: refs/heads/branch-1.5
Commit: 767ee1884b8ecba3afa8ed19a562626361d54f50
Parents: 71460b8
Author: zsxwing zsxw...@gmail.com
Authored: Tue Aug 11 12:39:13 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Tue Aug 11 12:39:39 2015 -0700

--
 .../apache/spark/sql/execution/Aggregate.scala  |  11 +
 .../spark/sql/execution/ExistingRDD.scala   |   2 -
 .../spark/sql/execution/LocalTableScan.scala|   2 -
 .../apache/spark/sql/execution/SparkPlan.scala  |  25 +-
 .../aggregate/SortBasedAggregate.scala  |  12 +-
 .../SortBasedAggregationIterator.scala  |  18 +-
 .../execution/aggregate/TungstenAggregate.scala |  12 +-
 .../aggregate/TungstenAggregationIterator.scala |  11 +-
 .../spark/sql/execution/basicOperators.scala|  36 +-
 .../sql/execution/joins/BroadcastHashJoin.scala |  30 +-
 .../joins/BroadcastHashOuterJoin.scala  |  40 +-
 .../joins/BroadcastLeftSemiJoinHash.scala   |  24 +-
 .../joins/BroadcastNestedLoopJoin.scala |  27 +-
 .../sql/execution/joins/CartesianProduct.scala  |  25 +-
 .../spark/sql/execution/joins/HashJoin.scala|   7 +-
 .../sql/execution/joins/HashOuterJoin.scala |  30 +-
 .../sql/execution/joins/HashSemiJoin.scala  |  23 +-
 .../sql/execution/joins/HashedRelation.scala|   8 +-
 .../sql/execution/joins/LeftSemiJoinBNL.scala   |  19 +-
 .../sql/execution/joins/LeftSemiJoinHash.scala  |  18 +-
 .../sql/execution/joins/ShuffledHashJoin.scala  |  16 +-
 .../execution/joins/ShuffledHashOuterJoin.scala |  29 +-
 .../sql/execution/joins/SortMergeJoin.scala |  21 +-
 .../execution/joins/SortMergeOuterJoin.scala|  38 +-
 .../spark/sql/execution/metric/SQLMetrics.scala |   6 +
 .../execution/joins/HashedRelationSuite.scala   |  14 +-
 .../sql/execution/metric/SQLMetricsSuite.scala  | 450 ++-
 27 files changed, 847 insertions(+), 107 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/767ee188/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
index e8c6a0f..f3b6a3a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
@@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.errors._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.execution.metric.SQLMetrics
 
 /**
  * :: DeveloperApi ::
@@ -45,6 +46,10 @@ case class Aggregate(
 child: SparkPlan)
   extends UnaryNode {
 
+  override private[sql] lazy val metrics = Map(
+numInputRows - SQLMetrics.createLongMetric(sparkContext, number of 
input rows),
+numOutputRows - SQLMetrics.createLongMetric(sparkContext, number of 
output rows))
+
   override def requiredChildDistribution: List[Distribution] = {
 if (partial) {
   UnspecifiedDistribution :: Nil
@@ -121,12 +126,15 @@ case class Aggregate(
   }
 
   protected override def doExecute(): RDD[InternalRow] = attachTree(this, 
execute) {
+val numInputRows = longMetric(numInputRows)
+val numOutputRows = 

[2/2] spark git commit: [SPARK-9646] [SQL] Add metrics for all join and aggregate operators

2015-08-11 Thread yhuai
[SPARK-9646] [SQL] Add metrics for all join and aggregate operators

This PR added metrics for all join and aggregate operators. However, I found 
the metrics may be confusing in the following two case:
1. The iterator is not totally consumed and the metric values will be less.
2. Recreating the iterators will make metric values look bigger than the size 
of the input source, such as `CartesianProduct`.

Author: zsxwing zsxw...@gmail.com

Closes #8060 from zsxwing/sql-metrics and squashes the following commits:

40f3fc1 [zsxwing] Mark LongSQLMetric private[metric] to avoid using incorrectly 
and leak memory
b1b9071 [zsxwing] Merge branch 'master' into sql-metrics
4bef25a [zsxwing] Add metrics for SortMergeOuterJoin
95ccfc6 [zsxwing] Merge branch 'master' into sql-metrics
67cb4dd [zsxwing] Add metrics for Project and TungstenProject; remove metrics 
from PhysicalRDD and LocalTableScan
0eb47d4 [zsxwing] Merge branch 'master' into sql-metrics
dd9d932 [zsxwing] Avoid creating new Iterators
589ea26 [zsxwing] Add metrics for all join and aggregate operators


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5831294a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5831294a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5831294a

Branch: refs/heads/master
Commit: 5831294a7a8fa2524133c5d718cbc8187d2b0620
Parents: 5b8bb1b
Author: zsxwing zsxw...@gmail.com
Authored: Tue Aug 11 12:39:13 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Tue Aug 11 12:39:13 2015 -0700

--
 .../apache/spark/sql/execution/Aggregate.scala  |  11 +
 .../spark/sql/execution/ExistingRDD.scala   |   2 -
 .../spark/sql/execution/LocalTableScan.scala|   2 -
 .../apache/spark/sql/execution/SparkPlan.scala  |  25 +-
 .../aggregate/SortBasedAggregate.scala  |  12 +-
 .../SortBasedAggregationIterator.scala  |  18 +-
 .../execution/aggregate/TungstenAggregate.scala |  12 +-
 .../aggregate/TungstenAggregationIterator.scala |  11 +-
 .../spark/sql/execution/basicOperators.scala|  36 +-
 .../sql/execution/joins/BroadcastHashJoin.scala |  30 +-
 .../joins/BroadcastHashOuterJoin.scala  |  40 +-
 .../joins/BroadcastLeftSemiJoinHash.scala   |  24 +-
 .../joins/BroadcastNestedLoopJoin.scala |  27 +-
 .../sql/execution/joins/CartesianProduct.scala  |  25 +-
 .../spark/sql/execution/joins/HashJoin.scala|   7 +-
 .../sql/execution/joins/HashOuterJoin.scala |  30 +-
 .../sql/execution/joins/HashSemiJoin.scala  |  23 +-
 .../sql/execution/joins/HashedRelation.scala|   8 +-
 .../sql/execution/joins/LeftSemiJoinBNL.scala   |  19 +-
 .../sql/execution/joins/LeftSemiJoinHash.scala  |  18 +-
 .../sql/execution/joins/ShuffledHashJoin.scala  |  16 +-
 .../execution/joins/ShuffledHashOuterJoin.scala |  29 +-
 .../sql/execution/joins/SortMergeJoin.scala |  21 +-
 .../execution/joins/SortMergeOuterJoin.scala|  38 +-
 .../spark/sql/execution/metric/SQLMetrics.scala |   6 +
 .../execution/joins/HashedRelationSuite.scala   |  14 +-
 .../sql/execution/metric/SQLMetricsSuite.scala  | 450 ++-
 27 files changed, 847 insertions(+), 107 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5831294a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
index e8c6a0f..f3b6a3a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
@@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.errors._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.execution.metric.SQLMetrics
 
 /**
  * :: DeveloperApi ::
@@ -45,6 +46,10 @@ case class Aggregate(
 child: SparkPlan)
   extends UnaryNode {
 
+  override private[sql] lazy val metrics = Map(
+numInputRows - SQLMetrics.createLongMetric(sparkContext, number of 
input rows),
+numOutputRows - SQLMetrics.createLongMetric(sparkContext, number of 
output rows))
+
   override def requiredChildDistribution: List[Distribution] = {
 if (partial) {
   UnspecifiedDistribution :: Nil
@@ -121,12 +126,15 @@ case class Aggregate(
   }
 
   protected override def doExecute(): RDD[InternalRow] = attachTree(this, 
execute) {
+val numInputRows = longMetric(numInputRows)
+val numOutputRows = longMetric(numOutputRows)
 if (groupingExpressions.isEmpty) {
   child.execute().mapPartitions { iter =
 val buffer = 

[1/2] spark git commit: [SPARK-9646] [SQL] Add metrics for all join and aggregate operators

2015-08-11 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 71460b889 - 767ee1884


http://git-wip-us.apache.org/repos/asf/spark/blob/767ee188/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
index 953284c..7383d3f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
@@ -25,15 +25,24 @@ import 
com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._
 import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._
 
 import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.test.TestSQLContext
+import org.apache.spark.sql._
+import org.apache.spark.sql.execution.ui.SparkPlanGraph
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.test.{SQLTestUtils, TestSQLContext}
 import org.apache.spark.util.Utils
 
+class SQLMetricsSuite extends SparkFunSuite with SQLTestUtils {
 
-class SQLMetricsSuite extends SparkFunSuite {
+  override val sqlContext = TestSQLContext
+
+  import sqlContext.implicits._
 
   test(LongSQLMetric should not box Long) {
 val l = SQLMetrics.createLongMetric(TestSQLContext.sparkContext, long)
-val f = () = { l += 1L }
+val f = () = {
+  l += 1L
+  l.add(1L)
+}
 BoxingFinder.getClassReader(f.getClass).foreach { cl =
   val boxingFinder = new BoxingFinder()
   cl.accept(boxingFinder, 0)
@@ -51,6 +60,441 @@ class SQLMetricsSuite extends SparkFunSuite {
   assert(boxingFinder.boxingInvokes.nonEmpty, Found find boxing in this 
test)
 }
   }
+
+  /**
+   * Call `df.collect()` and verify if the collected metrics are same as 
expectedMetrics.
+   *
+   * @param df `DataFrame` to run
+   * @param expectedNumOfJobs number of jobs that will run
+   * @param expectedMetrics the expected metrics. The format is
+   *`nodeId - (operatorName, metric name - metric 
value)`.
+   */
+  private def testSparkPlanMetrics(
+  df: DataFrame,
+  expectedNumOfJobs: Int,
+  expectedMetrics: Map[Long, (String, Map[String, Any])]): Unit = {
+val previousExecutionIds = TestSQLContext.listener.executionIdToData.keySet
+df.collect()
+TestSQLContext.sparkContext.listenerBus.waitUntilEmpty(1)
+val executionIds = 
TestSQLContext.listener.executionIdToData.keySet.diff(previousExecutionIds)
+assert(executionIds.size === 1)
+val executionId = executionIds.head
+val jobs = TestSQLContext.listener.getExecution(executionId).get.jobs
+// Use = because there is a race condition that we may miss some jobs
+// TODO Change it to = once we fix the race condition that missing the 
JobStarted event.
+assert(jobs.size = expectedNumOfJobs)
+if (jobs.size == expectedNumOfJobs) {
+  // If we can track all jobs, check the metric values
+  val metricValues = 
TestSQLContext.listener.getExecutionMetrics(executionId)
+  val actualMetrics = 
SparkPlanGraph(df.queryExecution.executedPlan).nodes.filter { node =
+expectedMetrics.contains(node.id)
+  }.map { node =
+val nodeMetrics = node.metrics.map { metric =
+  val metricValue = metricValues(metric.accumulatorId)
+  (metric.name, metricValue)
+}.toMap
+(node.id, node.name - nodeMetrics)
+  }.toMap
+  assert(expectedMetrics === actualMetrics)
+} else {
+  // TODO Remove this else once we fix the race condition that missing 
the JobStarted event.
+  // Since we cannot track all jobs, the metric values could be wrong and 
we should not check
+  // them.
+  logWarning(Due to a race condition, we miss some jobs and cannot verify 
the metric values)
+}
+  }
+
+  test(Project metrics) {
+withSQLConf(
+  SQLConf.UNSAFE_ENABLED.key - false,
+  SQLConf.CODEGEN_ENABLED.key - false,
+  SQLConf.TUNGSTEN_ENABLED.key - false) {
+  // Assume the execution plan is
+  // PhysicalRDD(nodeId = 1) - Project(nodeId = 0)
+  val df = TestData.person.select('name)
+  testSparkPlanMetrics(df, 1, Map(
+0L -(Project, Map(
+  number of rows - 2L)))
+  )
+}
+  }
+
+  test(TungstenProject metrics) {
+withSQLConf(
+  SQLConf.UNSAFE_ENABLED.key - true,
+  SQLConf.CODEGEN_ENABLED.key - true,
+  SQLConf.TUNGSTEN_ENABLED.key - true) {
+  // Assume the execution plan is
+  // PhysicalRDD(nodeId = 1) - TungstenProject(nodeId = 0)
+  val df = TestData.person.select('name)
+  testSparkPlanMetrics(df, 1, Map(
+0L -(TungstenProject, Map(
+  number of rows - 2L)))
+  )
+}
+  }
+
+  test(Filter metrics) {
+// Assume the execution plan is
+// 

spark git commit: [SPARK-9611] [SQL] Fixes a few corner cases when we spill a UnsafeFixedWidthAggregationMap

2015-08-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 eb2229ac0 - f24cd8cb9


[SPARK-9611] [SQL] Fixes a few corner cases when we spill a 
UnsafeFixedWidthAggregationMap

This PR has the following three small fixes.

1. UnsafeKVExternalSorter does not use 0 as the initialSize to create an 
UnsafeInMemorySorter if its BytesToBytesMap is empty.
2. We will not not spill a InMemorySorter if it is empty.
3. We will not add a SpillReader to a SpillMerger if this SpillReader is empty.

JIRA: https://issues.apache.org/jira/browse/SPARK-9611

Author: Yin Huai yh...@databricks.com

Closes #7948 from yhuai/unsafeEmptyMap and squashes the following commits:

9727abe [Yin Huai] Address Josh's comments.
34b6f76 [Yin Huai] 1. UnsafeKVExternalSorter does not use 0 as the initialSize 
to create an UnsafeInMemorySorter if its BytesToBytesMap is empty. 2. Do not 
spill a InMemorySorter if it is empty. 3. Do not add spill to SpillMerger if 
this spill is empty.

(cherry picked from commit 4581badbc8aa7e5a37ba7f7f83cc3860240f5dd3)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f24cd8cb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f24cd8cb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f24cd8cb

Branch: refs/heads/branch-1.5
Commit: f24cd8cb9998de7932086acc28c8f3f1a18edb35
Parents: eb2229a
Author: Yin Huai yh...@databricks.com
Authored: Wed Aug 5 19:19:09 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Aug 5 19:19:21 2015 -0700

--
 .../unsafe/sort/UnsafeExternalSorter.java   |  36 ---
 .../unsafe/sort/UnsafeSorterSpillMerger.java|  12 ++-
 .../sql/execution/UnsafeKVExternalSorter.java   |   6 +-
 .../UnsafeFixedWidthAggregationMapSuite.scala   | 108 ++-
 4 files changed, 141 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f24cd8cb/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
--
diff --git 
a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
 
b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index e6ddd08..8f78fc5 100644
--- 
a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ 
b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -191,24 +191,29 @@ public final class UnsafeExternalSorter {
   spillWriters.size(),
   spillWriters.size()  1 ?  times :  time);
 
-final UnsafeSorterSpillWriter spillWriter =
-  new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, 
writeMetrics,
-inMemSorter.numRecords());
-spillWriters.add(spillWriter);
-final UnsafeSorterIterator sortedRecords = inMemSorter.getSortedIterator();
-while (sortedRecords.hasNext()) {
-  sortedRecords.loadNext();
-  final Object baseObject = sortedRecords.getBaseObject();
-  final long baseOffset = sortedRecords.getBaseOffset();
-  final int recordLength = sortedRecords.getRecordLength();
-  spillWriter.write(baseObject, baseOffset, recordLength, 
sortedRecords.getKeyPrefix());
+// We only write out contents of the inMemSorter if it is not empty.
+if (inMemSorter.numRecords()  0) {
+  final UnsafeSorterSpillWriter spillWriter =
+new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, 
writeMetrics,
+  inMemSorter.numRecords());
+  spillWriters.add(spillWriter);
+  final UnsafeSorterIterator sortedRecords = 
inMemSorter.getSortedIterator();
+  while (sortedRecords.hasNext()) {
+sortedRecords.loadNext();
+final Object baseObject = sortedRecords.getBaseObject();
+final long baseOffset = sortedRecords.getBaseOffset();
+final int recordLength = sortedRecords.getRecordLength();
+spillWriter.write(baseObject, baseOffset, recordLength, 
sortedRecords.getKeyPrefix());
+  }
+  spillWriter.close();
 }
-spillWriter.close();
+
 final long spillSize = freeMemory();
 // Note that this is more-or-less going to be a multiple of the page size, 
so wasted space in
 // pages will currently be counted as memory spilled even though that 
space isn't actually
 // written to disk. This also counts the space needed to store the 
sorter's pointer array.
 taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
+
 initializeForWriting();
   }
 
@@ -505,12 +510,11 @@ public final class UnsafeExternalSorter {
   final UnsafeSorterSpillMerger spillMerger =
 new UnsafeSorterSpillMerger(recordComparator, prefixComparator, 
numIteratorsToMerge

spark git commit: [SPARK-9611] [SQL] Fixes a few corner cases when we spill a UnsafeFixedWidthAggregationMap

2015-08-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 4399b7b09 - 4581badbc


[SPARK-9611] [SQL] Fixes a few corner cases when we spill a 
UnsafeFixedWidthAggregationMap

This PR has the following three small fixes.

1. UnsafeKVExternalSorter does not use 0 as the initialSize to create an 
UnsafeInMemorySorter if its BytesToBytesMap is empty.
2. We will not not spill a InMemorySorter if it is empty.
3. We will not add a SpillReader to a SpillMerger if this SpillReader is empty.

JIRA: https://issues.apache.org/jira/browse/SPARK-9611

Author: Yin Huai yh...@databricks.com

Closes #7948 from yhuai/unsafeEmptyMap and squashes the following commits:

9727abe [Yin Huai] Address Josh's comments.
34b6f76 [Yin Huai] 1. UnsafeKVExternalSorter does not use 0 as the initialSize 
to create an UnsafeInMemorySorter if its BytesToBytesMap is empty. 2. Do not 
spill a InMemorySorter if it is empty. 3. Do not add spill to SpillMerger if 
this spill is empty.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4581badb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4581badb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4581badb

Branch: refs/heads/master
Commit: 4581badbc8aa7e5a37ba7f7f83cc3860240f5dd3
Parents: 4399b7b
Author: Yin Huai yh...@databricks.com
Authored: Wed Aug 5 19:19:09 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Aug 5 19:19:09 2015 -0700

--
 .../unsafe/sort/UnsafeExternalSorter.java   |  36 ---
 .../unsafe/sort/UnsafeSorterSpillMerger.java|  12 ++-
 .../sql/execution/UnsafeKVExternalSorter.java   |   6 +-
 .../UnsafeFixedWidthAggregationMapSuite.scala   | 108 ++-
 4 files changed, 141 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4581badb/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
--
diff --git 
a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
 
b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index e6ddd08..8f78fc5 100644
--- 
a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ 
b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -191,24 +191,29 @@ public final class UnsafeExternalSorter {
   spillWriters.size(),
   spillWriters.size()  1 ?  times :  time);
 
-final UnsafeSorterSpillWriter spillWriter =
-  new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, 
writeMetrics,
-inMemSorter.numRecords());
-spillWriters.add(spillWriter);
-final UnsafeSorterIterator sortedRecords = inMemSorter.getSortedIterator();
-while (sortedRecords.hasNext()) {
-  sortedRecords.loadNext();
-  final Object baseObject = sortedRecords.getBaseObject();
-  final long baseOffset = sortedRecords.getBaseOffset();
-  final int recordLength = sortedRecords.getRecordLength();
-  spillWriter.write(baseObject, baseOffset, recordLength, 
sortedRecords.getKeyPrefix());
+// We only write out contents of the inMemSorter if it is not empty.
+if (inMemSorter.numRecords()  0) {
+  final UnsafeSorterSpillWriter spillWriter =
+new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, 
writeMetrics,
+  inMemSorter.numRecords());
+  spillWriters.add(spillWriter);
+  final UnsafeSorterIterator sortedRecords = 
inMemSorter.getSortedIterator();
+  while (sortedRecords.hasNext()) {
+sortedRecords.loadNext();
+final Object baseObject = sortedRecords.getBaseObject();
+final long baseOffset = sortedRecords.getBaseOffset();
+final int recordLength = sortedRecords.getRecordLength();
+spillWriter.write(baseObject, baseOffset, recordLength, 
sortedRecords.getKeyPrefix());
+  }
+  spillWriter.close();
 }
-spillWriter.close();
+
 final long spillSize = freeMemory();
 // Note that this is more-or-less going to be a multiple of the page size, 
so wasted space in
 // pages will currently be counted as memory spilled even though that 
space isn't actually
 // written to disk. This also counts the space needed to store the 
sorter's pointer array.
 taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
+
 initializeForWriting();
   }
 
@@ -505,12 +510,11 @@ public final class UnsafeExternalSorter {
   final UnsafeSorterSpillMerger spillMerger =
 new UnsafeSorterSpillMerger(recordComparator, prefixComparator, 
numIteratorsToMerge);
   for (UnsafeSorterSpillWriter spillWriter : spillWriters) {
-spillMerger.addSpill(spillWriter.getReader(blockManager

spark git commit: [SPARK-9593] [SQL] [HOTFIX] Makes the Hadoop shims loading fix more robust

2015-08-06 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 93085c992 - 9f94c85ff


[SPARK-9593] [SQL] [HOTFIX] Makes the Hadoop shims loading fix more robust

This is a follow-up of #7929.

We found that Jenkins SBT master build still fails because of the Hadoop shims 
loading issue. But the failure doesn't appear to be deterministic. My suspect 
is that Hadoop `VersionInfo` class may fail to inspect Hadoop version, and the 
shims loading branch is skipped.

This PR tries to make the fix more robust:

1. When Hadoop version is available, we load `Hadoop20SShims` for versions = 
2.0.x as srowen suggested in PR #7929.
2. Otherwise, we use `Path.getPathWithoutSchemeAndAuthority` as a probe method, 
which doesn't exist in Hadoop 1.x or 2.0.x. If this method is not found, 
`Hadoop20SShims` is also loaded.

Author: Cheng Lian l...@databricks.com

Closes #7994 from liancheng/spark-9593/fix-hadoop-shims and squashes the 
following commits:

e1d3d70 [Cheng Lian] Fixes typo in comments
8d971da [Cheng Lian] Makes the Hadoop shims loading fix more robust


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9f94c85f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9f94c85f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9f94c85f

Branch: refs/heads/master
Commit: 9f94c85ff35df6289371f80edde51c2aa6c4bcdc
Parents: 93085c9
Author: Cheng Lian l...@databricks.com
Authored: Thu Aug 6 09:53:53 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Thu Aug 6 09:53:53 2015 -0700

--
 .../spark/sql/hive/client/ClientWrapper.scala   | 88 
 1 file changed, 55 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9f94c85f/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index 211a3b8..3d05b58 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -68,45 +68,67 @@ private[hive] class ClientWrapper(
 
   // !! HACK ALERT !!
   //
-  // This method is a surgical fix for Hadoop version 2.0.0-mr1-cdh4.1.1, 
which is used by Spark EC2
-  // scripts.  We should remove this after upgrading Spark EC2 scripts to some 
more recent Hadoop
-  // version in the future.
-  //
   // Internally, Hive `ShimLoader` tries to load different versions of Hadoop 
shims by checking
-  // version information gathered from Hadoop jar files.  If the major version 
number is 1,
-  // `Hadoop20SShims` will be loaded.  Otherwise, if the major version number 
is 2, `Hadoop23Shims`
-  // will be chosen.
+  // major version number gathered from Hadoop jar files:
+  //
+  // - For major version number 1, load `Hadoop20SShims`, where 20S stands 
for Hadoop 0.20 with
+  //   security.
+  // - For major version number 2, load `Hadoop23Shims`, where 23 stands for 
Hadoop 0.23.
   //
-  // However, part of APIs in Hadoop 2.0.x and 2.1.x versions were in flux due 
to historical
-  // reasons. So 2.0.0-mr1-cdh4.1.1 is actually more Hadoop-1-like and should 
be used together with
-  // `Hadoop20SShims`, but `Hadoop20SShims` is chose because the major version 
number here is 2.
+  // However, APIs in Hadoop 2.0.x and 2.1.x versions were in flux due to 
historical reasons. It
+  // turns out that Hadoop 2.0.x versions should also be used together with 
`Hadoop20SShims`, but
+  // `Hadoop23Shims` is chosen because the major version number here is 2.
   //
-  // Here we check for this specific version and loads `Hadoop20SShims` via 
reflection.  Note that
-  // we can't check for string literal 2.0.0-mr1-cdh4.1.1 because the 
obtained version string
-  // comes from Maven artifact org.apache.hadoop:hadoop-common:2.0.0-cdh4.1.1, 
which doesn't have
-  // the mr1 tag in its version string.
+  // To fix this issue, we try to inspect Hadoop version via 
`org.apache.hadoop.utils.VersionInfo`
+  // and load `Hadoop20SShims` for Hadoop 1.x and 2.0.x versions.  If Hadoop 
version information is
+  // not available, we decide whether to override the shims or not by checking 
for existence of a
+  // probe method which doesn't exist in Hadoop 1.x or 2.0.x versions.
   private def overrideHadoopShims(): Unit = {
-val VersionPattern = 2\.0\.0.*cdh4.*.r
-
-VersionInfo.getVersion match {
-  case VersionPattern() =
-val shimClassName = org.apache.hadoop.hive.shims.Hadoop20SShims
-logInfo(sLoading Hadoop shims $shimClassName)
-
-try {
-  val shimsField = 

spark git commit: [SPARK-9593] [SQL] Fixes Hadoop shims loading

2015-08-06 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 c39d5d144 - 11c28a568


[SPARK-9593] [SQL] Fixes Hadoop shims loading

This PR is used to workaround CDH Hadoop versions like 2.0.0-mr1-cdh4.1.1.

Internally, Hive `ShimLoader` tries to load different versions of Hadoop shims 
by checking version information gathered from Hadoop jar files.  If the major 
version number is 1, `Hadoop20SShims` will be loaded.  Otherwise, if the major 
version number is 2, `Hadoop23Shims` will be chosen.  However, CDH Hadoop 
versions like 2.0.0-mr1-cdh4.1.1 have 2 as major version number, but contain 
Hadoop 1 code.  This confuses Hive `ShimLoader` and loads wrong version of 
shims.

In this PR we check for existence of the 
`Path.getPathWithoutSchemeAndAuthority` method, which doesn't exist in Hadoop 1 
(it's also the method that reveals this shims loading issue), and load 
`Hadoop20SShims` when it doesn't exist.

Author: Cheng Lian l...@databricks.com

Closes #7929 from liancheng/spark-9593/fix-hadoop-shims-loading and squashes 
the following commits:

c99b497 [Cheng Lian] Narrows down the fix to handle 2.0.0-*cdh4* Hadoop 
versions only
b17e955 [Cheng Lian] Updates comments
490d8f2 [Cheng Lian] Fixes Scala style issue
9c6c12d [Cheng Lian] Fixes Hadoop shims loading

(cherry picked from commit 70112ff22bd1aee7689c5d3af9b66c9b8ceb3ec3)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/11c28a56
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/11c28a56
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/11c28a56

Branch: refs/heads/branch-1.5
Commit: 11c28a568ea55fcde54048d357e0709e08be3072
Parents: c39d5d1
Author: Cheng Lian l...@databricks.com
Authored: Wed Aug 5 20:03:54 2015 +0800
Committer: Yin Huai yh...@databricks.com
Committed: Thu Aug 6 09:59:42 2015 -0700

--
 .../spark/sql/hive/client/ClientWrapper.scala   | 48 
 1 file changed, 48 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/11c28a56/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index dc372be..211a3b8 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -32,6 +32,8 @@ import org.apache.hadoop.hive.ql.metadata.Hive
 import org.apache.hadoop.hive.ql.processors._
 import org.apache.hadoop.hive.ql.session.SessionState
 import org.apache.hadoop.hive.ql.{Driver, metadata}
+import org.apache.hadoop.hive.shims.{HadoopShims, ShimLoader}
+import org.apache.hadoop.util.VersionInfo
 
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.expressions.Expression
@@ -62,6 +64,52 @@ private[hive] class ClientWrapper(
   extends ClientInterface
   with Logging {
 
+  overrideHadoopShims()
+
+  // !! HACK ALERT !!
+  //
+  // This method is a surgical fix for Hadoop version 2.0.0-mr1-cdh4.1.1, 
which is used by Spark EC2
+  // scripts.  We should remove this after upgrading Spark EC2 scripts to some 
more recent Hadoop
+  // version in the future.
+  //
+  // Internally, Hive `ShimLoader` tries to load different versions of Hadoop 
shims by checking
+  // version information gathered from Hadoop jar files.  If the major version 
number is 1,
+  // `Hadoop20SShims` will be loaded.  Otherwise, if the major version number 
is 2, `Hadoop23Shims`
+  // will be chosen.
+  //
+  // However, part of APIs in Hadoop 2.0.x and 2.1.x versions were in flux due 
to historical
+  // reasons. So 2.0.0-mr1-cdh4.1.1 is actually more Hadoop-1-like and should 
be used together with
+  // `Hadoop20SShims`, but `Hadoop20SShims` is chose because the major version 
number here is 2.
+  //
+  // Here we check for this specific version and loads `Hadoop20SShims` via 
reflection.  Note that
+  // we can't check for string literal 2.0.0-mr1-cdh4.1.1 because the 
obtained version string
+  // comes from Maven artifact org.apache.hadoop:hadoop-common:2.0.0-cdh4.1.1, 
which doesn't have
+  // the mr1 tag in its version string.
+  private def overrideHadoopShims(): Unit = {
+val VersionPattern = 2\.0\.0.*cdh4.*.r
+
+VersionInfo.getVersion match {
+  case VersionPattern() =
+val shimClassName = org.apache.hadoop.hive.shims.Hadoop20SShims
+logInfo(sLoading Hadoop shims $shimClassName)
+
+try {
+  val shimsField = classOf[ShimLoader].getDeclaredField(hadoopShims)
+  // scalastyle:off classforname
+  val shimsClass = 

spark git commit: [SPARK-9632] [SQL] [HOT-FIX] Fix build.

2015-08-06 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 2382b483a - b51159def


[SPARK-9632] [SQL] [HOT-FIX] Fix build.

seems https://github.com/apache/spark/pull/7955 breaks the build.

Author: Yin Huai yh...@databricks.com

Closes #8001 from yhuai/SPARK-9632-fixBuild and squashes the following commits:

6c257dd [Yin Huai] Fix build.

(cherry picked from commit cdd53b762bf358616b313e3334b5f6945caf9ab1)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b51159de
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b51159de
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b51159de

Branch: refs/heads/branch-1.5
Commit: b51159defd9351c237b5b79834aa58f9d109b389
Parents: 2382b48
Author: Yin Huai yh...@databricks.com
Authored: Thu Aug 6 11:15:54 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Thu Aug 6 11:16:11 2015 -0700

--
 .../main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b51159de/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
index 207e667..11d10b2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 
 /**
  * An extended version of [[InternalRow]] that implements all special getters, 
toString


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



spark git commit: [SPARK-9632] [SQL] [HOT-FIX] Fix build.

2015-08-06 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 2eca46a17 - cdd53b762


[SPARK-9632] [SQL] [HOT-FIX] Fix build.

seems https://github.com/apache/spark/pull/7955 breaks the build.

Author: Yin Huai yh...@databricks.com

Closes #8001 from yhuai/SPARK-9632-fixBuild and squashes the following commits:

6c257dd [Yin Huai] Fix build.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cdd53b76
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cdd53b76
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cdd53b76

Branch: refs/heads/master
Commit: cdd53b762bf358616b313e3334b5f6945caf9ab1
Parents: 2eca46a
Author: Yin Huai yh...@databricks.com
Authored: Thu Aug 6 11:15:54 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Thu Aug 6 11:15:54 2015 -0700

--
 .../main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cdd53b76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
index 7657fb5..fd42fac 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 
 /**
  * An extended interface to [[InternalRow]] that allows the values for each 
column to be updated.


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



spark git commit: [SPARK-9593] [SQL] [HOTFIX] Makes the Hadoop shims loading fix more robust

2015-08-06 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 11c28a568 - cc4c569a8


[SPARK-9593] [SQL] [HOTFIX] Makes the Hadoop shims loading fix more robust

This is a follow-up of #7929.

We found that Jenkins SBT master build still fails because of the Hadoop shims 
loading issue. But the failure doesn't appear to be deterministic. My suspect 
is that Hadoop `VersionInfo` class may fail to inspect Hadoop version, and the 
shims loading branch is skipped.

This PR tries to make the fix more robust:

1. When Hadoop version is available, we load `Hadoop20SShims` for versions = 
2.0.x as srowen suggested in PR #7929.
2. Otherwise, we use `Path.getPathWithoutSchemeAndAuthority` as a probe method, 
which doesn't exist in Hadoop 1.x or 2.0.x. If this method is not found, 
`Hadoop20SShims` is also loaded.

Author: Cheng Lian l...@databricks.com

Closes #7994 from liancheng/spark-9593/fix-hadoop-shims and squashes the 
following commits:

e1d3d70 [Cheng Lian] Fixes typo in comments
8d971da [Cheng Lian] Makes the Hadoop shims loading fix more robust

(cherry picked from commit 9f94c85ff35df6289371f80edde51c2aa6c4bcdc)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cc4c569a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cc4c569a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cc4c569a

Branch: refs/heads/branch-1.5
Commit: cc4c569a8d7eb4696439568c2ba246b77223647f
Parents: 11c28a5
Author: Cheng Lian l...@databricks.com
Authored: Thu Aug 6 09:53:53 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Thu Aug 6 10:00:08 2015 -0700

--
 .../spark/sql/hive/client/ClientWrapper.scala   | 88 
 1 file changed, 55 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cc4c569a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index 211a3b8..3d05b58 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -68,45 +68,67 @@ private[hive] class ClientWrapper(
 
   // !! HACK ALERT !!
   //
-  // This method is a surgical fix for Hadoop version 2.0.0-mr1-cdh4.1.1, 
which is used by Spark EC2
-  // scripts.  We should remove this after upgrading Spark EC2 scripts to some 
more recent Hadoop
-  // version in the future.
-  //
   // Internally, Hive `ShimLoader` tries to load different versions of Hadoop 
shims by checking
-  // version information gathered from Hadoop jar files.  If the major version 
number is 1,
-  // `Hadoop20SShims` will be loaded.  Otherwise, if the major version number 
is 2, `Hadoop23Shims`
-  // will be chosen.
+  // major version number gathered from Hadoop jar files:
+  //
+  // - For major version number 1, load `Hadoop20SShims`, where 20S stands 
for Hadoop 0.20 with
+  //   security.
+  // - For major version number 2, load `Hadoop23Shims`, where 23 stands for 
Hadoop 0.23.
   //
-  // However, part of APIs in Hadoop 2.0.x and 2.1.x versions were in flux due 
to historical
-  // reasons. So 2.0.0-mr1-cdh4.1.1 is actually more Hadoop-1-like and should 
be used together with
-  // `Hadoop20SShims`, but `Hadoop20SShims` is chose because the major version 
number here is 2.
+  // However, APIs in Hadoop 2.0.x and 2.1.x versions were in flux due to 
historical reasons. It
+  // turns out that Hadoop 2.0.x versions should also be used together with 
`Hadoop20SShims`, but
+  // `Hadoop23Shims` is chosen because the major version number here is 2.
   //
-  // Here we check for this specific version and loads `Hadoop20SShims` via 
reflection.  Note that
-  // we can't check for string literal 2.0.0-mr1-cdh4.1.1 because the 
obtained version string
-  // comes from Maven artifact org.apache.hadoop:hadoop-common:2.0.0-cdh4.1.1, 
which doesn't have
-  // the mr1 tag in its version string.
+  // To fix this issue, we try to inspect Hadoop version via 
`org.apache.hadoop.utils.VersionInfo`
+  // and load `Hadoop20SShims` for Hadoop 1.x and 2.0.x versions.  If Hadoop 
version information is
+  // not available, we decide whether to override the shims or not by checking 
for existence of a
+  // probe method which doesn't exist in Hadoop 1.x or 2.0.x versions.
   private def overrideHadoopShims(): Unit = {
-val VersionPattern = 2\.0\.0.*cdh4.*.r
-
-VersionInfo.getVersion match {
-  case VersionPattern() =
-val shimClassName = org.apache.hadoop.hive.shims.Hadoop20SShims
-

spark git commit: [SPARK-9674] Re-enable ignored test in SQLQuerySuite

2015-08-07 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 05d04e10a - 881548ab2


[SPARK-9674] Re-enable ignored test in SQLQuerySuite

The original code that this test tests is removed in 
https://github.com/apache/spark/commit/9270bd06fd0b16892e3f37213b5bc7813ea11fdd.
 It was ignored shortly before that so we never caught it. This patch 
re-enables the test and adds the code necessary to make it pass.

JoshRosen yhuai

Author: Andrew Or and...@databricks.com

Closes #8015 from andrewor14/SPARK-9674 and squashes the following commits:

225eac2 [Andrew Or] Merge branch 'master' of github.com:apache/spark into 
SPARK-9674
8c24209 [Andrew Or] Fix NPE
e541d64 [Andrew Or] Track aggregation memory for both sort and hash
0be3a42 [Andrew Or] Fix test


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/881548ab
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/881548ab
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/881548ab

Branch: refs/heads/master
Commit: 881548ab20fa4c4b635c51d956b14bd13981e2f4
Parents: 05d04e1
Author: Andrew Or and...@databricks.com
Authored: Fri Aug 7 14:20:13 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Fri Aug 7 14:20:13 2015 -0700

--
 .../spark/unsafe/map/BytesToBytesMap.java   | 37 +---
 .../map/AbstractBytesToBytesMapSuite.java   | 20 +++
 .../UnsafeFixedWidthAggregationMap.java |  7 ++--
 .../sql/execution/UnsafeKVExternalSorter.java   |  7 
 .../aggregate/TungstenAggregationIterator.scala | 32 +
 .../org/apache/spark/sql/SQLQuerySuite.scala|  8 ++---
 6 files changed, 85 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/881548ab/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
--
diff --git 
a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java 
b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
index 0636ae7..7f79cd1 100644
--- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
+++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
@@ -109,7 +109,7 @@ public final class BytesToBytesMap {
* Position {@code 2 * i} in the array is used to track a pointer to the key 
at index {@code i},
* while position {@code 2 * i + 1} in the array holds key's full 32-bit 
hashcode.
*/
-  private LongArray longArray;
+  @Nullable private LongArray longArray;
   // TODO: we're wasting 32 bits of space here; we can probably store fewer 
bits of the hashcode
   // and exploit word-alignment to use fewer bits to hold the address.  This 
might let us store
   // only one long per map entry, increasing the chance that this array will 
fit in cache at the
@@ -124,7 +124,7 @@ public final class BytesToBytesMap {
* A {@link BitSet} used to track location of the map where the key is set.
* Size of the bitset should be half of the size of the long array.
*/
-  private BitSet bitset;
+  @Nullable private BitSet bitset;
 
   private final double loadFactor;
 
@@ -166,6 +166,8 @@ public final class BytesToBytesMap {
 
   private long numHashCollisions = 0;
 
+  private long peakMemoryUsedBytes = 0L;
+
   public BytesToBytesMap(
   TaskMemoryManager taskMemoryManager,
   ShuffleMemoryManager shuffleMemoryManager,
@@ -321,6 +323,9 @@ public final class BytesToBytesMap {
   Object keyBaseObject,
   long keyBaseOffset,
   int keyRowLengthBytes) {
+assert(bitset != null);
+assert(longArray != null);
+
 if (enablePerfMetrics) {
   numKeyLookups++;
 }
@@ -410,6 +415,7 @@ public final class BytesToBytesMap {
 }
 
 private Location with(int pos, int keyHashcode, boolean isDefined) {
+  assert(longArray != null);
   this.pos = pos;
   this.isDefined = isDefined;
   this.keyHashcode = keyHashcode;
@@ -525,6 +531,9 @@ public final class BytesToBytesMap {
   assert (!isDefined) : Can only set value once for a key;
   assert (keyLengthBytes % 8 == 0);
   assert (valueLengthBytes % 8 == 0);
+  assert(bitset != null);
+  assert(longArray != null);
+
   if (numElements == MAX_CAPACITY) {
 throw new IllegalStateException(BytesToBytesMap has reached maximum 
capacity);
   }
@@ -658,6 +667,7 @@ public final class BytesToBytesMap {
* This method is idempotent and can be called multiple times.
*/
   public void free() {
+updatePeakMemoryUsed();
 longArray = null;
 bitset = null;
 IteratorMemoryBlock dataPagesIterator = dataPages.iterator();
@@ -684,14 +694,30 @@ public final class BytesToBytesMap {
 
   /**
* Returns the total amount of memory, in bytes, consumed by this map's 
managed structures

spark git commit: [SPARK-9674] Re-enable ignored test in SQLQuerySuite

2015-08-07 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 d13b5c895 - 547120287


[SPARK-9674] Re-enable ignored test in SQLQuerySuite

The original code that this test tests is removed in 
https://github.com/apache/spark/commit/9270bd06fd0b16892e3f37213b5bc7813ea11fdd.
 It was ignored shortly before that so we never caught it. This patch 
re-enables the test and adds the code necessary to make it pass.

JoshRosen yhuai

Author: Andrew Or and...@databricks.com

Closes #8015 from andrewor14/SPARK-9674 and squashes the following commits:

225eac2 [Andrew Or] Merge branch 'master' of github.com:apache/spark into 
SPARK-9674
8c24209 [Andrew Or] Fix NPE
e541d64 [Andrew Or] Track aggregation memory for both sort and hash
0be3a42 [Andrew Or] Fix test

(cherry picked from commit 881548ab20fa4c4b635c51d956b14bd13981e2f4)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/54712028
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/54712028
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/54712028

Branch: refs/heads/branch-1.5
Commit: 5471202871b717cec7bb16fc42cd7658b8ddbe01
Parents: d13b5c8
Author: Andrew Or and...@databricks.com
Authored: Fri Aug 7 14:20:13 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Fri Aug 7 14:20:46 2015 -0700

--
 .../spark/unsafe/map/BytesToBytesMap.java   | 37 +---
 .../map/AbstractBytesToBytesMapSuite.java   | 20 +++
 .../UnsafeFixedWidthAggregationMap.java |  7 ++--
 .../sql/execution/UnsafeKVExternalSorter.java   |  7 
 .../aggregate/TungstenAggregationIterator.scala | 32 +
 .../org/apache/spark/sql/SQLQuerySuite.scala|  8 ++---
 6 files changed, 85 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/54712028/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
--
diff --git 
a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java 
b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
index 0636ae7..7f79cd1 100644
--- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
+++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
@@ -109,7 +109,7 @@ public final class BytesToBytesMap {
* Position {@code 2 * i} in the array is used to track a pointer to the key 
at index {@code i},
* while position {@code 2 * i + 1} in the array holds key's full 32-bit 
hashcode.
*/
-  private LongArray longArray;
+  @Nullable private LongArray longArray;
   // TODO: we're wasting 32 bits of space here; we can probably store fewer 
bits of the hashcode
   // and exploit word-alignment to use fewer bits to hold the address.  This 
might let us store
   // only one long per map entry, increasing the chance that this array will 
fit in cache at the
@@ -124,7 +124,7 @@ public final class BytesToBytesMap {
* A {@link BitSet} used to track location of the map where the key is set.
* Size of the bitset should be half of the size of the long array.
*/
-  private BitSet bitset;
+  @Nullable private BitSet bitset;
 
   private final double loadFactor;
 
@@ -166,6 +166,8 @@ public final class BytesToBytesMap {
 
   private long numHashCollisions = 0;
 
+  private long peakMemoryUsedBytes = 0L;
+
   public BytesToBytesMap(
   TaskMemoryManager taskMemoryManager,
   ShuffleMemoryManager shuffleMemoryManager,
@@ -321,6 +323,9 @@ public final class BytesToBytesMap {
   Object keyBaseObject,
   long keyBaseOffset,
   int keyRowLengthBytes) {
+assert(bitset != null);
+assert(longArray != null);
+
 if (enablePerfMetrics) {
   numKeyLookups++;
 }
@@ -410,6 +415,7 @@ public final class BytesToBytesMap {
 }
 
 private Location with(int pos, int keyHashcode, boolean isDefined) {
+  assert(longArray != null);
   this.pos = pos;
   this.isDefined = isDefined;
   this.keyHashcode = keyHashcode;
@@ -525,6 +531,9 @@ public final class BytesToBytesMap {
   assert (!isDefined) : Can only set value once for a key;
   assert (keyLengthBytes % 8 == 0);
   assert (valueLengthBytes % 8 == 0);
+  assert(bitset != null);
+  assert(longArray != null);
+
   if (numElements == MAX_CAPACITY) {
 throw new IllegalStateException(BytesToBytesMap has reached maximum 
capacity);
   }
@@ -658,6 +667,7 @@ public final class BytesToBytesMap {
* This method is idempotent and can be called multiple times.
*/
   public void free() {
+updatePeakMemoryUsed();
 longArray = null;
 bitset = null;
 IteratorMemoryBlock dataPagesIterator = dataPages.iterator();
@@ -684,14 +694,30 @@ public final class

spark git commit: [SPARK-6212] [SQL] The EXPLAIN output of CTAS only shows the analyzed plan

2015-08-08 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 874b9d855 - 251d1eef4


[SPARK-6212] [SQL] The EXPLAIN output of CTAS only shows the analyzed plan

JIRA: https://issues.apache.org/jira/browse/SPARK-6212

Author: Yijie Shen henry.yijies...@gmail.com

Closes #7986 from yjshen/ctas_explain and squashes the following commits:

bb6fee5 [Yijie Shen] refine test
f731041 [Yijie Shen] address comment
b2cf8ab [Yijie Shen] bug fix
bd7eb20 [Yijie Shen] ctas explain

(cherry picked from commit 3ca995b78f373251081f6877623649bfba3040b2)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/251d1eef
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/251d1eef
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/251d1eef

Branch: refs/heads/branch-1.5
Commit: 251d1eef4848e7c1bd0fff826dfbde48c6de1145
Parents: 874b9d8
Author: Yijie Shen henry.yijies...@gmail.com
Authored: Sat Aug 8 21:05:50 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Sat Aug 8 21:06:26 2015 -0700

--
 .../apache/spark/sql/execution/commands.scala   |  2 ++
 .../hive/execution/CreateTableAsSelect.scala|  4 ++-
 .../sql/hive/execution/HiveExplainSuite.scala   | 35 ++--
 3 files changed, 38 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/251d1eef/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index 6b83025..95209e6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -69,6 +69,8 @@ private[sql] case class ExecutedCommand(cmd: RunnableCommand) 
extends SparkPlan
 val converted = sideEffectResult.map(convert(_).asInstanceOf[InternalRow])
 sqlContext.sparkContext.parallelize(converted, 1)
   }
+
+  override def argString: String = cmd.toString
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/251d1eef/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
index 84358cb..8422287 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
@@ -40,6 +40,8 @@ case class CreateTableAsSelect(
   def database: String = tableDesc.database
   def tableName: String = tableDesc.name
 
+  override def children: Seq[LogicalPlan] = Seq(query)
+
   override def run(sqlContext: SQLContext): Seq[Row] = {
 val hiveContext = sqlContext.asInstanceOf[HiveContext]
 lazy val metastoreRelation: MetastoreRelation = {
@@ -91,6 +93,6 @@ case class CreateTableAsSelect(
   }
 
   override def argString: String = {
-s[Database:$database, TableName: $tableName, InsertIntoHiveTable]\n + 
query.toString
+s[Database:$database, TableName: $tableName, InsertIntoHiveTable]
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/251d1eef/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
index 8215dd6..44c5b80 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
@@ -17,13 +17,18 @@
 
 package org.apache.spark.sql.hive.execution
 
-import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.{SQLContext, QueryTest}
+import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
+import org.apache.spark.sql.test.SQLTestUtils
 
 /**
  * A set of tests that validates support for Hive Explain command.
  */
-class HiveExplainSuite extends QueryTest {
+class HiveExplainSuite extends QueryTest with SQLTestUtils {
+
+  def sqlContext: SQLContext = TestHive
+
   test(explain extended command) {
 checkExistence(sql( explain   select * from src where key=123 ), true,
== Physical Plan ==)
@@ -74,4 +79,30 @@ class HiveExplainSuite extends QueryTest {
   Limit,
   src)
   }
+
+  

spark git commit: [SPARK-6212] [SQL] The EXPLAIN output of CTAS only shows the analyzed plan

2015-08-08 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 25c363e93 - 3ca995b78


[SPARK-6212] [SQL] The EXPLAIN output of CTAS only shows the analyzed plan

JIRA: https://issues.apache.org/jira/browse/SPARK-6212

Author: Yijie Shen henry.yijies...@gmail.com

Closes #7986 from yjshen/ctas_explain and squashes the following commits:

bb6fee5 [Yijie Shen] refine test
f731041 [Yijie Shen] address comment
b2cf8ab [Yijie Shen] bug fix
bd7eb20 [Yijie Shen] ctas explain


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3ca995b7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3ca995b7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3ca995b7

Branch: refs/heads/master
Commit: 3ca995b78f373251081f6877623649bfba3040b2
Parents: 25c363e
Author: Yijie Shen henry.yijies...@gmail.com
Authored: Sat Aug 8 21:05:50 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Sat Aug 8 21:05:50 2015 -0700

--
 .../apache/spark/sql/execution/commands.scala   |  2 ++
 .../hive/execution/CreateTableAsSelect.scala|  4 ++-
 .../sql/hive/execution/HiveExplainSuite.scala   | 35 ++--
 3 files changed, 38 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3ca995b7/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index 6b83025..95209e6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -69,6 +69,8 @@ private[sql] case class ExecutedCommand(cmd: RunnableCommand) 
extends SparkPlan
 val converted = sideEffectResult.map(convert(_).asInstanceOf[InternalRow])
 sqlContext.sparkContext.parallelize(converted, 1)
   }
+
+  override def argString: String = cmd.toString
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/3ca995b7/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
index 84358cb..8422287 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
@@ -40,6 +40,8 @@ case class CreateTableAsSelect(
   def database: String = tableDesc.database
   def tableName: String = tableDesc.name
 
+  override def children: Seq[LogicalPlan] = Seq(query)
+
   override def run(sqlContext: SQLContext): Seq[Row] = {
 val hiveContext = sqlContext.asInstanceOf[HiveContext]
 lazy val metastoreRelation: MetastoreRelation = {
@@ -91,6 +93,6 @@ case class CreateTableAsSelect(
   }
 
   override def argString: String = {
-s[Database:$database, TableName: $tableName, InsertIntoHiveTable]\n + 
query.toString
+s[Database:$database, TableName: $tableName, InsertIntoHiveTable]
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/3ca995b7/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
index 8215dd6..44c5b80 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
@@ -17,13 +17,18 @@
 
 package org.apache.spark.sql.hive.execution
 
-import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.{SQLContext, QueryTest}
+import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
+import org.apache.spark.sql.test.SQLTestUtils
 
 /**
  * A set of tests that validates support for Hive Explain command.
  */
-class HiveExplainSuite extends QueryTest {
+class HiveExplainSuite extends QueryTest with SQLTestUtils {
+
+  def sqlContext: SQLContext = TestHive
+
   test(explain extended command) {
 checkExistence(sql( explain   select * from src where key=123 ), true,
== Physical Plan ==)
@@ -74,4 +79,30 @@ class HiveExplainSuite extends QueryTest {
   Limit,
   src)
   }
+
+  test(SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan) {
+withTempTable(jt) {
+  val rdd = 

spark git commit: [SPARK-8930] [SQL] Throw a AnalysisException with meaningful messages if DataFrame#explode takes a star in expressions

2015-08-09 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 b12f0737f - 1ce5061bb


[SPARK-8930] [SQL] Throw a AnalysisException with meaningful messages if 
DataFrame#explode takes a star in expressions

Author: Yijie Shen henry.yijies...@gmail.com

Closes #8057 from yjshen/explode_star and squashes the following commits:

eae181d [Yijie Shen] change explaination message
54c9d11 [Yijie Shen] meaning message for * in explode

(cherry picked from commit 68ccc6e184598822b19a880fdd4597b66a1c2d92)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1ce5061b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1ce5061b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1ce5061b

Branch: refs/heads/branch-1.5
Commit: 1ce5061bbca7e9905abf9980ddbf30afe599e26e
Parents: b12f073
Author: Yijie Shen henry.yijies...@gmail.com
Authored: Sun Aug 9 11:44:51 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Sun Aug 9 11:45:08 2015 -0700

--
 .../spark/sql/catalyst/analysis/Analyzer.scala   |  4 +++-
 .../spark/sql/catalyst/analysis/AnalysisTest.scala   |  4 +++-
 .../scala/org/apache/spark/sql/DataFrameSuite.scala  | 15 +++
 3 files changed, 21 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1ce5061b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 82158e6..a684dbc 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -408,7 +408,7 @@ class Analyzer(
 /**
  * Returns true if `exprs` contains a [[Star]].
  */
-protected def containsStar(exprs: Seq[Expression]): Boolean =
+def containsStar(exprs: Seq[Expression]): Boolean =
   exprs.exists(_.collect { case _: Star = true }.nonEmpty)
   }
 
@@ -602,6 +602,8 @@ class Analyzer(
*/
   object ResolveGenerate extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case g: Generate if ResolveReferences.containsStar(g.generator.children) 
=
+failAnalysis(Cannot explode *, explode can only be applied on a 
specific column.)
   case p: Generate if !p.child.resolved || !p.generator.resolved = p
   case g: Generate if !g.resolved =
 g.copy(generatorOutput = makeGeneratorOutput(g.generator, 
g.generatorOutput.map(_.name)))

http://git-wip-us.apache.org/repos/asf/spark/blob/1ce5061b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
index ee1f8f5..53b3695 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
@@ -71,6 +71,8 @@ trait AnalysisTest extends PlanTest {
 val e = intercept[Exception] {
   analyzer.checkAnalysis(analyzer.execute(inputPlan))
 }
-expectedErrors.forall(e.getMessage.contains)
+
assert(expectedErrors.map(_.toLowerCase).forall(e.getMessage.toLowerCase.contains),
+  sExpected to throw Exception contains: ${expectedErrors.mkString(, 
)},  +
+sactually we get ${e.getMessage})
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/1ce5061b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 0212637..c49f256 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -134,6 +134,21 @@ class DataFrameSuite extends QueryTest with SQLTestUtils {
 )
   }
 
+  test(SPARK-8930: explode should fail with a meaningful message if it takes 
a star) {
+val df = Seq((1, 1,2), (2, 4), (3, 7,8,9)).toDF(prefix, 
csv)
+val e = intercept[AnalysisException] {
+  df.explode($*) { case Row(prefix: String, csv: String) =
+csv.split(,).map(v = Tuple1(prefix + : + v)).toSeq
+  }.queryExecution.assertAnalyzed()
+}
+

spark git commit: [SPARK-8930] [SQL] Throw a AnalysisException with meaningful messages if DataFrame#explode takes a star in expressions

2015-08-09 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master e9c36938b - 68ccc6e18


[SPARK-8930] [SQL] Throw a AnalysisException with meaningful messages if 
DataFrame#explode takes a star in expressions

Author: Yijie Shen henry.yijies...@gmail.com

Closes #8057 from yjshen/explode_star and squashes the following commits:

eae181d [Yijie Shen] change explaination message
54c9d11 [Yijie Shen] meaning message for * in explode


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/68ccc6e1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/68ccc6e1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/68ccc6e1

Branch: refs/heads/master
Commit: 68ccc6e184598822b19a880fdd4597b66a1c2d92
Parents: e9c3693
Author: Yijie Shen henry.yijies...@gmail.com
Authored: Sun Aug 9 11:44:51 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Sun Aug 9 11:44:51 2015 -0700

--
 .../spark/sql/catalyst/analysis/Analyzer.scala   |  4 +++-
 .../spark/sql/catalyst/analysis/AnalysisTest.scala   |  4 +++-
 .../scala/org/apache/spark/sql/DataFrameSuite.scala  | 15 +++
 3 files changed, 21 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/68ccc6e1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 82158e6..a684dbc 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -408,7 +408,7 @@ class Analyzer(
 /**
  * Returns true if `exprs` contains a [[Star]].
  */
-protected def containsStar(exprs: Seq[Expression]): Boolean =
+def containsStar(exprs: Seq[Expression]): Boolean =
   exprs.exists(_.collect { case _: Star = true }.nonEmpty)
   }
 
@@ -602,6 +602,8 @@ class Analyzer(
*/
   object ResolveGenerate extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case g: Generate if ResolveReferences.containsStar(g.generator.children) 
=
+failAnalysis(Cannot explode *, explode can only be applied on a 
specific column.)
   case p: Generate if !p.child.resolved || !p.generator.resolved = p
   case g: Generate if !g.resolved =
 g.copy(generatorOutput = makeGeneratorOutput(g.generator, 
g.generatorOutput.map(_.name)))

http://git-wip-us.apache.org/repos/asf/spark/blob/68ccc6e1/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
index ee1f8f5..53b3695 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
@@ -71,6 +71,8 @@ trait AnalysisTest extends PlanTest {
 val e = intercept[Exception] {
   analyzer.checkAnalysis(analyzer.execute(inputPlan))
 }
-expectedErrors.forall(e.getMessage.contains)
+
assert(expectedErrors.map(_.toLowerCase).forall(e.getMessage.toLowerCase.contains),
+  sExpected to throw Exception contains: ${expectedErrors.mkString(, 
)},  +
+sactually we get ${e.getMessage})
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/68ccc6e1/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 0212637..c49f256 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -134,6 +134,21 @@ class DataFrameSuite extends QueryTest with SQLTestUtils {
 )
   }
 
+  test(SPARK-8930: explode should fail with a meaningful message if it takes 
a star) {
+val df = Seq((1, 1,2), (2, 4), (3, 7,8,9)).toDF(prefix, 
csv)
+val e = intercept[AnalysisException] {
+  df.explode($*) { case Row(prefix: String, csv: String) =
+csv.split(,).map(v = Tuple1(prefix + : + v)).toSeq
+  }.queryExecution.assertAnalyzed()
+}
+assert(e.getMessage.contains(
+  Cannot explode *, explode can only be applied on a specific column.))
+
+df.explode('prefix, 'csv) { case 

spark git commit: [SPARK-9703] [SQL] Refactor EnsureRequirements to avoid certain unnecessary shuffles

2015-08-09 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master a863348fd - 23cf5af08


[SPARK-9703] [SQL] Refactor EnsureRequirements to avoid certain unnecessary 
shuffles

This pull request refactors the `EnsureRequirements` planning rule in order to 
avoid the addition of certain unnecessary shuffles.

As an example of how unnecessary shuffles can occur, consider SortMergeJoin, 
which requires clustered distribution and sorted ordering of its children's 
input rows. Say that both of SMJ's children produce unsorted output but are 
both SinglePartition. In this case, we will need to inject sort operators but 
should not need to inject Exchanges. Unfortunately, it looks like the 
EnsureRequirements unnecessarily repartitions using a hash partitioning.

This patch solves this problem by refactoring `EnsureRequirements` to properly 
implement the `compatibleWith` checks that were broken in earlier 
implementations. See the significant inline comments for a better description 
of how this works. The majority of this PR is new comments and test cases, with 
few actual changes to the code.

Author: Josh Rosen joshro...@databricks.com

Closes #7988 from JoshRosen/exchange-fixes and squashes the following commits:

38006e7 [Josh Rosen] Rewrite EnsureRequirements _yet again_ to make things even 
simpler
0983f75 [Josh Rosen] More guarantees vs. compatibleWith cleanup; delete 
BroadcastPartitioning.
8784bd9 [Josh Rosen] Giant comment explaining compatibleWith vs. guarantees
1307c50 [Josh Rosen] Update conditions for requiring child compatibility.
18cddeb [Josh Rosen] Rename DummyPlan to DummySparkPlan.
2c7e126 [Josh Rosen] Merge remote-tracking branch 'origin/master' into 
exchange-fixes
fee65c4 [Josh Rosen] Further refinement to comments / reasoning
642b0bb [Josh Rosen] Further expand comment / reasoning
06aba0c [Josh Rosen] Add more comments
8dbc845 [Josh Rosen] Add even more tests.
4f08278 [Josh Rosen] Fix the test by adding the compatibility check to 
EnsureRequirements
a1c12b9 [Josh Rosen] Add failing test to demonstrate allCompatible bug
0725a34 [Josh Rosen] Small assertion cleanup.
5172ac5 [Josh Rosen] Add test for 
requiresChildrenToProduceSameNumberOfPartitions.
2e0f33a [Josh Rosen] Write a more generic test for EnsureRequirements.
752b8de [Josh Rosen] style fix
c628daf [Josh Rosen] Revert accidental ExchangeSuite change.
c9fb231 [Josh Rosen] Rewrite exchange to fix better handle this case.
adcc742 [Josh Rosen] Move test to PlannerSuite.
0675956 [Josh Rosen] Preserving ordering and partitioning in row format 
converters also does not help.
cc5669c [Josh Rosen] Adding outputPartitioning to Repartition does not fix the 
test.
2dfc648 [Josh Rosen] Add failing test illustrating bad exchange planning.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/23cf5af0
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/23cf5af0
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/23cf5af0

Branch: refs/heads/master
Commit: 23cf5af08d98da771c41571c00a2f5cafedfebdd
Parents: a863348
Author: Josh Rosen joshro...@databricks.com
Authored: Sun Aug 9 14:26:01 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Sun Aug 9 14:26:01 2015 -0700

--
 .../catalyst/plans/physical/partitioning.scala  | 128 ++--
 .../apache/spark/sql/execution/Exchange.scala   | 104 +++--
 .../spark/sql/execution/basicOperators.scala|   5 +
 .../sql/execution/rowFormatConverters.scala |   5 +
 .../spark/sql/execution/PlannerSuite.scala  | 151 +++
 5 files changed, 328 insertions(+), 65 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/23cf5af0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index ec659ce..5a89a90 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -75,6 +75,37 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) 
extends Distribution {
   def clustering: Set[Expression] = ordering.map(_.child).toSet
 }
 
+/**
+ * Describes how an operator's output is split across partitions. The 
`compatibleWith`,
+ * `guarantees`, and `satisfies` methods describe relationships between child 
partitionings,
+ * target partitionings, and [[Distribution]]s. These relations are described 
more precisely in
+ * their individual method docs, but at a high level:
+ *
+ *  - `satisfies` is a relationship between 

spark git commit: [SPARK-9703] [SQL] Refactor EnsureRequirements to avoid certain unnecessary shuffles

2015-08-09 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 1ce5061bb - 323d68606


[SPARK-9703] [SQL] Refactor EnsureRequirements to avoid certain unnecessary 
shuffles

This pull request refactors the `EnsureRequirements` planning rule in order to 
avoid the addition of certain unnecessary shuffles.

As an example of how unnecessary shuffles can occur, consider SortMergeJoin, 
which requires clustered distribution and sorted ordering of its children's 
input rows. Say that both of SMJ's children produce unsorted output but are 
both SinglePartition. In this case, we will need to inject sort operators but 
should not need to inject Exchanges. Unfortunately, it looks like the 
EnsureRequirements unnecessarily repartitions using a hash partitioning.

This patch solves this problem by refactoring `EnsureRequirements` to properly 
implement the `compatibleWith` checks that were broken in earlier 
implementations. See the significant inline comments for a better description 
of how this works. The majority of this PR is new comments and test cases, with 
few actual changes to the code.

Author: Josh Rosen joshro...@databricks.com

Closes #7988 from JoshRosen/exchange-fixes and squashes the following commits:

38006e7 [Josh Rosen] Rewrite EnsureRequirements _yet again_ to make things even 
simpler
0983f75 [Josh Rosen] More guarantees vs. compatibleWith cleanup; delete 
BroadcastPartitioning.
8784bd9 [Josh Rosen] Giant comment explaining compatibleWith vs. guarantees
1307c50 [Josh Rosen] Update conditions for requiring child compatibility.
18cddeb [Josh Rosen] Rename DummyPlan to DummySparkPlan.
2c7e126 [Josh Rosen] Merge remote-tracking branch 'origin/master' into 
exchange-fixes
fee65c4 [Josh Rosen] Further refinement to comments / reasoning
642b0bb [Josh Rosen] Further expand comment / reasoning
06aba0c [Josh Rosen] Add more comments
8dbc845 [Josh Rosen] Add even more tests.
4f08278 [Josh Rosen] Fix the test by adding the compatibility check to 
EnsureRequirements
a1c12b9 [Josh Rosen] Add failing test to demonstrate allCompatible bug
0725a34 [Josh Rosen] Small assertion cleanup.
5172ac5 [Josh Rosen] Add test for 
requiresChildrenToProduceSameNumberOfPartitions.
2e0f33a [Josh Rosen] Write a more generic test for EnsureRequirements.
752b8de [Josh Rosen] style fix
c628daf [Josh Rosen] Revert accidental ExchangeSuite change.
c9fb231 [Josh Rosen] Rewrite exchange to fix better handle this case.
adcc742 [Josh Rosen] Move test to PlannerSuite.
0675956 [Josh Rosen] Preserving ordering and partitioning in row format 
converters also does not help.
cc5669c [Josh Rosen] Adding outputPartitioning to Repartition does not fix the 
test.
2dfc648 [Josh Rosen] Add failing test illustrating bad exchange planning.

(cherry picked from commit 23cf5af08d98da771c41571c00a2f5cafedfebdd)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/323d6860
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/323d6860
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/323d6860

Branch: refs/heads/branch-1.5
Commit: 323d686063937adb34729de16ea181bd56a97705
Parents: 1ce5061
Author: Josh Rosen joshro...@databricks.com
Authored: Sun Aug 9 14:26:01 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Sun Aug 9 14:26:16 2015 -0700

--
 .../catalyst/plans/physical/partitioning.scala  | 128 ++--
 .../apache/spark/sql/execution/Exchange.scala   | 104 +++--
 .../spark/sql/execution/basicOperators.scala|   5 +
 .../sql/execution/rowFormatConverters.scala |   5 +
 .../spark/sql/execution/PlannerSuite.scala  | 151 +++
 5 files changed, 328 insertions(+), 65 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/323d6860/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index ec659ce..5a89a90 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -75,6 +75,37 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) 
extends Distribution {
   def clustering: Set[Expression] = ordering.map(_.child).toSet
 }
 
+/**
+ * Describes how an operator's output is split across partitions. The 
`compatibleWith`,
+ * `guarantees`, and `satisfies` methods describe relationships between child 
partitionings,
+ * target partitionings, and [[Distribution]]s. These relations are described 

spark git commit: [SPARK-9743] [SQL] Fixes JSONRelation refreshing

2015-08-10 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 f75c64b0c - 94b2f5b32


[SPARK-9743] [SQL] Fixes JSONRelation refreshing

PR #7696 added two `HadoopFsRelation.refresh()` calls ([this] [1], and [this] 
[2]) in `DataSourceStrategy` to make test case `InsertSuite.save directly to 
the path of a JSON table` pass. However, this forces every `HadoopFsRelation` 
table scan to do a refresh, which can be super expensive for tables with large 
number of partitions.

The reason why the original test case fails without the `refresh()` calls is 
that, the old JSON relation builds the base RDD with the input paths, while 
`HadoopFsRelation` provides `FileStatus`es of leaf files. With the old JSON 
relation, we can create a temporary table based on a path, writing data to 
that, and then read newly written data without refreshing the table. This is no 
long true for `HadoopFsRelation`.

This PR removes those two expensive refresh calls, and moves the refresh into 
`JSONRelation` to fix this issue. We might want to update `HadoopFsRelation` 
interface to provide better support for this use case.

[1]: 
https://github.com/apache/spark/blob/ebfd91c542aaead343cb154277fcf9114382fee7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala#L63
[2]: 
https://github.com/apache/spark/blob/ebfd91c542aaead343cb154277fcf9114382fee7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala#L91

Author: Cheng Lian l...@databricks.com

Closes #8035 from liancheng/spark-9743/fix-json-relation-refreshing and 
squashes the following commits:

ec1957d [Cheng Lian] Fixes JSONRelation refreshing

(cherry picked from commit e3fef0f9e17b1766a3869cb80ce7e4cd521cb7b6)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/94b2f5b3
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/94b2f5b3
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/94b2f5b3

Branch: refs/heads/branch-1.5
Commit: 94b2f5b3213553278ead376c24e63f019a18e793
Parents: f75c64b
Author: Cheng Lian l...@databricks.com
Authored: Mon Aug 10 09:07:08 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Mon Aug 10 09:07:20 2015 -0700

--
 .../datasources/DataSourceStrategy.scala |  2 --
 .../org/apache/spark/sql/json/JSONRelation.scala | 19 +++
 .../apache/spark/sql/sources/interfaces.scala|  2 +-
 .../apache/spark/sql/sources/InsertSuite.scala   | 10 +-
 4 files changed, 21 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/94b2f5b3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
index 5b5fa8c..78a4acd 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
@@ -60,7 +60,6 @@ private[sql] object DataSourceStrategy extends Strategy with 
Logging {
 // Scanning partitioned HadoopFsRelation
 case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
HadoopFsRelation))
 if t.partitionSpec.partitionColumns.nonEmpty =
-  t.refresh()
   val selectedPartitions = prunePartitions(filters, 
t.partitionSpec).toArray
 
   logInfo {
@@ -88,7 +87,6 @@ private[sql] object DataSourceStrategy extends Strategy with 
Logging {
 
 // Scanning non-partitioned HadoopFsRelation
 case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
HadoopFsRelation)) =
-  t.refresh()
   // See buildPartitionedTableScan for the reason that we need to create a 
shard
   // broadcast HadoopConf.
   val sharedHadoopConf = SparkHadoopUtil.get.conf

http://git-wip-us.apache.org/repos/asf/spark/blob/94b2f5b3/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
index b34a272..5bb9e62 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
@@ -22,20 +22,22 @@ import java.io.CharArrayWriter
 import com.fasterxml.jackson.core.JsonFactory
 import com.google.common.base.Objects
 import org.apache.hadoop.fs.{FileStatus, Path}
-import org.apache.hadoop.io.{Text, 

spark git commit: [SPARK-9849] [SQL] DirectParquetOutputCommitter qualified name should be backward compatible

2015-08-11 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 5a5bbc299 - afa757c98


[SPARK-9849] [SQL] DirectParquetOutputCommitter qualified name should be 
backward compatible

DirectParquetOutputCommitter was moved in SPARK-9763. However, users can 
explicitly set the class as a config option, so we must be able to resolve the 
old committer qualified name.

Author: Reynold Xin r...@databricks.com

Closes #8114 from rxin/SPARK-9849.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/afa757c9
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/afa757c9
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/afa757c9

Branch: refs/heads/master
Commit: afa757c98c537965007cad4c61c436887f3ac6a6
Parents: 5a5bbc2
Author: Reynold Xin r...@databricks.com
Authored: Tue Aug 11 18:08:49 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Tue Aug 11 18:08:49 2015 -0700

--
 .../datasources/parquet/ParquetRelation.scala   |  7 +
 .../datasources/parquet/ParquetIOSuite.scala| 27 +++-
 2 files changed, 33 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/afa757c9/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
index 4086a13..c71c69b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
@@ -209,6 +209,13 @@ private[sql] class ParquetRelation(
   override def prepareJobForWrite(job: Job): OutputWriterFactory = {
 val conf = ContextUtil.getConfiguration(job)
 
+// SPARK-9849 DirectParquetOutputCommitter qualified name should be 
backward compatible
+val committerClassname = 
conf.get(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key)
+if (committerClassname == 
org.apache.spark.sql.parquet.DirectParquetOutputCommitter) {
+  conf.set(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key,
+classOf[DirectParquetOutputCommitter].getCanonicalName)
+}
+
 val committerClass =
   conf.getClass(
 SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key,

http://git-wip-us.apache.org/repos/asf/spark/blob/afa757c9/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
index ee925af..cb16634 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
@@ -390,7 +390,32 @@ class ParquetIOSuite extends QueryTest with ParquetTest {
 }
   }
 
-  test(SPARK-8121: spark.sql.parquet.output.committer.class shouldn't be 
overriden) {
+  test(SPARK-9849 DirectParquetOutputCommitter qualified name should be 
backward compatible) {
+val clonedConf = new Configuration(configuration)
+
+// Write to a parquet file and let it fail.
+// _temporary should be missing if direct output committer works.
+try {
+  configuration.set(spark.sql.parquet.output.committer.class,
+org.apache.spark.sql.parquet.DirectParquetOutputCommitter)
+  sqlContext.udf.register(div0, (x: Int) = x / 0)
+  withTempPath { dir =
+intercept[org.apache.spark.SparkException] {
+  sqlContext.sql(select div0(1)).write.parquet(dir.getCanonicalPath)
+}
+val path = new Path(dir.getCanonicalPath, _temporary)
+val fs = path.getFileSystem(configuration)
+assert(!fs.exists(path))
+  }
+} finally {
+  // Hadoop 1 doesn't have `Configuration.unset`
+  configuration.clear()
+  clonedConf.foreach(entry = configuration.set(entry.getKey, 
entry.getValue))
+}
+  }
+
+
+  test(SPARK-8121: spark.sql.parquet.output.committer.class shouldn't be 
overridden) {
 withTempPath { dir =
   val clonedConf = new Configuration(configuration)
 


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



spark git commit: [SPARK-9849] [SQL] DirectParquetOutputCommitter qualified name should be backward compatible

2015-08-11 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 b7497e3a2 - ec7a4b9b0


[SPARK-9849] [SQL] DirectParquetOutputCommitter qualified name should be 
backward compatible

DirectParquetOutputCommitter was moved in SPARK-9763. However, users can 
explicitly set the class as a config option, so we must be able to resolve the 
old committer qualified name.

Author: Reynold Xin r...@databricks.com

Closes #8114 from rxin/SPARK-9849.

(cherry picked from commit afa757c98c537965007cad4c61c436887f3ac6a6)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ec7a4b9b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ec7a4b9b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ec7a4b9b

Branch: refs/heads/branch-1.5
Commit: ec7a4b9b0b1183965e086f724877d69bccbdbcbe
Parents: b7497e3
Author: Reynold Xin r...@databricks.com
Authored: Tue Aug 11 18:08:49 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Tue Aug 11 18:09:05 2015 -0700

--
 .../datasources/parquet/ParquetRelation.scala   |  7 +
 .../datasources/parquet/ParquetIOSuite.scala| 27 +++-
 2 files changed, 33 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ec7a4b9b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
index 4086a13..c71c69b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala
@@ -209,6 +209,13 @@ private[sql] class ParquetRelation(
   override def prepareJobForWrite(job: Job): OutputWriterFactory = {
 val conf = ContextUtil.getConfiguration(job)
 
+// SPARK-9849 DirectParquetOutputCommitter qualified name should be 
backward compatible
+val committerClassname = 
conf.get(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key)
+if (committerClassname == 
org.apache.spark.sql.parquet.DirectParquetOutputCommitter) {
+  conf.set(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key,
+classOf[DirectParquetOutputCommitter].getCanonicalName)
+}
+
 val committerClass =
   conf.getClass(
 SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key,

http://git-wip-us.apache.org/repos/asf/spark/blob/ec7a4b9b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
index ee925af..cb16634 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
@@ -390,7 +390,32 @@ class ParquetIOSuite extends QueryTest with ParquetTest {
 }
   }
 
-  test(SPARK-8121: spark.sql.parquet.output.committer.class shouldn't be 
overriden) {
+  test(SPARK-9849 DirectParquetOutputCommitter qualified name should be 
backward compatible) {
+val clonedConf = new Configuration(configuration)
+
+// Write to a parquet file and let it fail.
+// _temporary should be missing if direct output committer works.
+try {
+  configuration.set(spark.sql.parquet.output.committer.class,
+org.apache.spark.sql.parquet.DirectParquetOutputCommitter)
+  sqlContext.udf.register(div0, (x: Int) = x / 0)
+  withTempPath { dir =
+intercept[org.apache.spark.SparkException] {
+  sqlContext.sql(select div0(1)).write.parquet(dir.getCanonicalPath)
+}
+val path = new Path(dir.getCanonicalPath, _temporary)
+val fs = path.getFileSystem(configuration)
+assert(!fs.exists(path))
+  }
+} finally {
+  // Hadoop 1 doesn't have `Configuration.unset`
+  configuration.clear()
+  clonedConf.foreach(entry = configuration.set(entry.getKey, 
entry.getValue))
+}
+  }
+
+
+  test(SPARK-8121: spark.sql.parquet.output.committer.class shouldn't be 
overridden) {
 withTempPath { dir =
   val clonedConf = new Configuration(configuration)
 


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



spark git commit: [SPARK-9385] [HOT-FIX] [PYSPARK] Comment out Python style check

2015-07-27 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master c1be9f309 - 2104931d7


[SPARK-9385] [HOT-FIX] [PYSPARK] Comment out Python style check

https://issues.apache.org/jira/browse/SPARK-9385

Comment out Python style check because of error shown in 
https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/3088/AMPLAB_JENKINS_BUILD_PROFILE=hadoop1.0,label=centos/console

Author: Yin Huai yh...@databricks.com

Closes #7702 from yhuai/SPARK-9385 and squashes the following commits:

146e6ef [Yin Huai] Comment out Python style check because of error shown in 
https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/3088/AMPLAB_JENKINS_BUILD_PROFILE=hadoop1.0,label=centos/console


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2104931d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2104931d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2104931d

Branch: refs/heads/master
Commit: 2104931d7d726eda2c098e0f403c7f1533df8746
Parents: c1be9f3
Author: Yin Huai yh...@databricks.com
Authored: Mon Jul 27 15:18:48 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Mon Jul 27 15:18:48 2015 -0700

--
 dev/run-tests.py | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2104931d/dev/run-tests.py
--
diff --git a/dev/run-tests.py b/dev/run-tests.py
index 1f0d218..d1cb668 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -198,8 +198,9 @@ def run_scala_style_checks():
 
 
 def run_python_style_checks():
-set_title_and_block(Running Python style checks, BLOCK_PYTHON_STYLE)
-run_cmd([os.path.join(SPARK_HOME, dev, lint-python)])
+# set_title_and_block(Running Python style checks, BLOCK_PYTHON_STYLE)
+# run_cmd([os.path.join(SPARK_HOME, dev, lint-python)])
+pass
 
 
 def build_spark_documentation():


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



spark git commit: [SPARK-9385] [PYSPARK] Enable PEP8 but disable installing pylint.

2015-07-27 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master ab6259566 - dafe8d857


[SPARK-9385] [PYSPARK] Enable PEP8 but disable installing pylint.

Instead of disabling all python style check, we should enable PEP8. So, this PR 
just comments out the part installing pylint.

Author: Yin Huai yh...@databricks.com

Closes #7704 from yhuai/SPARK-9385 and squashes the following commits:

0056359 [Yin Huai] Enable PEP8 but disable installing pylint.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dafe8d85
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dafe8d85
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dafe8d85

Branch: refs/heads/master
Commit: dafe8d857dff4c61981476282cbfe11f5c008078
Parents: ab62595
Author: Yin Huai yh...@databricks.com
Authored: Mon Jul 27 15:49:42 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Mon Jul 27 15:49:42 2015 -0700

--
 dev/lint-python  | 30 +++---
 dev/run-tests.py |  5 ++---
 2 files changed, 17 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/dafe8d85/dev/lint-python
--
diff --git a/dev/lint-python b/dev/lint-python
index 53bccc1..575dbb0 100755
--- a/dev/lint-python
+++ b/dev/lint-python
@@ -58,21 +58,21 @@ export PYTHONPATH=$SPARK_ROOT_DIR/dev/pylint
 export PYLINT_HOME=$PYTHONPATH
 export PATH=$PYTHONPATH:$PATH
 
-if [ ! -d $PYLINT_HOME ]; then
-mkdir $PYLINT_HOME
-# Redirect the annoying pylint installation output.
-easy_install -d $PYLINT_HOME pylint==1.4.4  $PYLINT_INSTALL_INFO
-easy_install_status=$?
-
-if [ $easy_install_status -ne 0 ]; then
-echo Unable to install pylint locally in \$PYTHONPATH\.
-cat $PYLINT_INSTALL_INFO
-exit $easy_install_status
-fi
-
-rm $PYLINT_INSTALL_INFO
-
-fi
+# if [ ! -d $PYLINT_HOME ]; then
+# mkdir $PYLINT_HOME
+# # Redirect the annoying pylint installation output.
+# easy_install -d $PYLINT_HOME pylint==1.4.4  $PYLINT_INSTALL_INFO
+# easy_install_status=$?
+#
+# if [ $easy_install_status -ne 0 ]; then
+# echo Unable to install pylint locally in \$PYTHONPATH\.
+# cat $PYLINT_INSTALL_INFO
+# exit $easy_install_status
+# fi
+#
+# rm $PYLINT_INSTALL_INFO
+#
+# fi
 
 # There is no need to write this output to a file
 #+ first, but we do so so that the check status can

http://git-wip-us.apache.org/repos/asf/spark/blob/dafe8d85/dev/run-tests.py
--
diff --git a/dev/run-tests.py b/dev/run-tests.py
index d1cb668..1f0d218 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -198,9 +198,8 @@ def run_scala_style_checks():
 
 
 def run_python_style_checks():
-# set_title_and_block(Running Python style checks, BLOCK_PYTHON_STYLE)
-# run_cmd([os.path.join(SPARK_HOME, dev, lint-python)])
-pass
+set_title_and_block(Running Python style checks, BLOCK_PYTHON_STYLE)
+run_cmd([os.path.join(SPARK_HOME, dev, lint-python)])
 
 
 def build_spark_documentation():


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



spark git commit: [SPARK-9386] [SQL] Feature flag for metastore partition pruning

2015-07-27 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 8ddfa52c2 - ce89ff477


[SPARK-9386] [SQL] Feature flag for metastore partition pruning

Since we have been seeing a lot of failures related to this new feature, lets 
put it behind a flag and turn it off by default.

Author: Michael Armbrust mich...@databricks.com

Closes #7703 from marmbrus/optionalMetastorePruning and squashes the following 
commits:

6ad128c [Michael Armbrust] style
8447835 [Michael Armbrust] [SPARK-9386][SQL] Feature flag for metastore 
partition pruning
fd37b87 [Michael Armbrust] add config flag


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ce89ff47
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ce89ff47
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ce89ff47

Branch: refs/heads/master
Commit: ce89ff477aea6def68265ed218f6105680755c9a
Parents: 8ddfa52
Author: Michael Armbrust mich...@databricks.com
Authored: Mon Jul 27 17:32:34 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Mon Jul 27 17:32:34 2015 -0700

--
 .../src/main/scala/org/apache/spark/sql/SQLConf.scala   |  7 +++
 .../apache/spark/sql/hive/HiveMetastoreCatalog.scala| 12 +++-
 .../apache/spark/sql/hive/client/ClientInterface.scala  | 10 --
 3 files changed, 22 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ce89ff47/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
index 9b2dbd7..40eba33 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
@@ -301,6 +301,11 @@ private[spark] object SQLConf {
 defaultValue = Some(true),
 doc = TODO)
 
+  val HIVE_METASTORE_PARTITION_PRUNING = 
booleanConf(spark.sql.hive.metastorePartitionPruning,
+defaultValue = Some(false),
+doc = When true, some predicates will be pushed down into the Hive 
metastore so that  +
+  unmatching partitions can be eliminated earlier.)
+
   val COLUMN_NAME_OF_CORRUPT_RECORD = 
stringConf(spark.sql.columnNameOfCorruptRecord,
 defaultValue = Some(_corrupt_record),
 doc = TODO)
@@ -456,6 +461,8 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf {
 
   private[spark] def verifyPartitionPath: Boolean = 
getConf(HIVE_VERIFY_PARTITION_PATH)
 
+  private[spark] def metastorePartitionPruning: Boolean = 
getConf(HIVE_METASTORE_PARTITION_PRUNING)
+
   private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT)
 
   private[spark] def sortMergeJoinEnabled: Boolean = getConf(SORTMERGE_JOIN)

http://git-wip-us.apache.org/repos/asf/spark/blob/ce89ff47/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 9c707a7..3180c05 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -678,8 +678,18 @@ private[hive] case class MetastoreRelation
 }
   )
 
+  // When metastore partition pruning is turned off, we cache the list of all 
partitions to
+  // mimic the behavior of Spark  1.5
+  lazy val allPartitions = table.getAllPartitions
+
   def getHiveQlPartitions(predicates: Seq[Expression] = Nil): Seq[Partition] = 
{
-table.getPartitions(predicates).map { p =
+val rawPartitions = if (sqlContext.conf.metastorePartitionPruning) {
+  table.getPartitions(predicates)
+} else {
+  allPartitions
+}
+
+rawPartitions.map { p =
   val tPartition = new org.apache.hadoop.hive.metastore.api.Partition
   tPartition.setDbName(databaseName)
   tPartition.setTableName(tableName)

http://git-wip-us.apache.org/repos/asf/spark/blob/ce89ff47/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala
index 1656587..d834b4e 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala
@@ -72,12 +72,10 @@ private[hive] case class HiveTable(
 
   def isPartitioned: Boolean = 

spark git commit: [SPARK-9082] [SQL] Filter using non-deterministic expressions should not be pushed down

2015-07-22 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master b55a36bc3 - 76520955f


[SPARK-9082] [SQL] Filter using non-deterministic expressions should not be 
pushed down

Author: Wenchen Fan cloud0...@outlook.com

Closes #7446 from cloud-fan/filter and squashes the following commits:

330021e [Wenchen Fan] add exists to tree node
2cab68c [Wenchen Fan] more enhance
949be07 [Wenchen Fan] push down part of predicate if possible
3912f84 [Wenchen Fan] address comments
8ce15ca [Wenchen Fan] fix bug
557158e [Wenchen Fan] Filter using non-deterministic expressions should not be 
pushed down


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/76520955
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/76520955
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/76520955

Branch: refs/heads/master
Commit: 76520955fddbda87a5c53d0a394dedc91dce67e8
Parents: b55a36b
Author: Wenchen Fan cloud0...@outlook.com
Authored: Wed Jul 22 11:45:51 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Jul 22 11:45:51 2015 -0700

--
 .../sql/catalyst/optimizer/Optimizer.scala  | 50 
 .../optimizer/FilterPushdownSuite.scala | 45 +-
 2 files changed, 84 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/76520955/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index e42f0b9..d2db3dd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -541,20 +541,50 @@ object SimplifyFilters extends Rule[LogicalPlan] {
  *
  * This heuristic is valid assuming the expression evaluation cost is minimal.
  */
-object PushPredicateThroughProject extends Rule[LogicalPlan] {
+object PushPredicateThroughProject extends Rule[LogicalPlan] with 
PredicateHelper {
   def apply(plan: LogicalPlan): LogicalPlan = plan transform {
 case filter @ Filter(condition, project @ Project(fields, grandChild)) =
-  val sourceAliases = fields.collect { case a @ Alias(c, _) =
-(a.toAttribute: Attribute) - c
-  }.toMap
-  project.copy(child = filter.copy(
-replaceAlias(condition, sourceAliases),
-grandChild))
+  // Create a map of Aliases to their values from the child projection.
+  // e.g., 'SELECT a + b AS c, d ...' produces Map(c - a + b).
+  val aliasMap = AttributeMap(fields.collect {
+case a: Alias = (a.toAttribute, a.child)
+  })
+
+  // Split the condition into small conditions by `And`, so that we can 
push down part of this
+  // condition without nondeterministic expressions.
+  val andConditions = splitConjunctivePredicates(condition)
+  val nondeterministicConditions = 
andConditions.filter(hasNondeterministic(_, aliasMap))
+
+  // If there is no nondeterministic conditions, push down the whole 
condition.
+  if (nondeterministicConditions.isEmpty) {
+project.copy(child = Filter(replaceAlias(condition, aliasMap), 
grandChild))
+  } else {
+// If they are all nondeterministic conditions, leave it un-changed.
+if (nondeterministicConditions.length == andConditions.length) {
+  filter
+} else {
+  val deterministicConditions = 
andConditions.filterNot(hasNondeterministic(_, aliasMap))
+  // Push down the small conditions without nondeterministic 
expressions.
+  val pushedCondition = deterministicConditions.map(replaceAlias(_, 
aliasMap)).reduce(And)
+  Filter(nondeterministicConditions.reduce(And),
+project.copy(child = Filter(pushedCondition, grandChild)))
+}
+  }
+  }
+
+  private def hasNondeterministic(
+  condition: Expression,
+  sourceAliases: AttributeMap[Expression]) = {
+condition.collect {
+  case a: Attribute if sourceAliases.contains(a) = sourceAliases(a)
+}.exists(!_.deterministic)
   }
 
-  private def replaceAlias(condition: Expression, sourceAliases: 
Map[Attribute, Expression]) = {
-condition transform {
-  case a: AttributeReference = sourceAliases.getOrElse(a, a)
+  // Substitute any attributes that are produced by the child projection, so 
that we safely
+  // eliminate it.
+  private def replaceAlias(condition: Expression, sourceAliases: 
AttributeMap[Expression]) = {
+condition.transform {
+  case a: Attribute = sourceAliases.getOrElse(a, a)
 }
   }
 }


spark git commit: [SPARK-9254] [BUILD] [HOTFIX] sbt-launch-lib.bash should support HTTP/HTTPS redirection

2015-07-25 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.4 ff5e5f228 - 712e13bba


[SPARK-9254] [BUILD] [HOTFIX] sbt-launch-lib.bash should support HTTP/HTTPS 
redirection

Target file(s) can be hosted on CDN nodes. HTTP/HTTPS redirection must be 
supported to download these files.

Author: Cheng Lian l...@databricks.com

Closes #7597 from liancheng/spark-9254 and squashes the following commits:

fd266ca [Cheng Lian] Uses `--fail' to make curl return non-zero value and 
remove garbage output when the download fails
a7cbfb3 [Cheng Lian] Supports HTTP/HTTPS redirection

(cherry picked from commit b55a36bc30a628d76baa721d38789fc219eccc27)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/712e13bb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/712e13bb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/712e13bb

Branch: refs/heads/branch-1.4
Commit: 712e13bba4fe9aaafe9e7a4c3b891501628e6bb7
Parents: ff5e5f2
Author: Cheng Lian l...@databricks.com
Authored: Wed Jul 22 09:32:42 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Fri Jul 24 23:04:38 2015 -0700

--
 build/sbt-launch-lib.bash | 8 ++--
 1 file changed, 6 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/712e13bb/build/sbt-launch-lib.bash
--
diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash
index 504be48..7930a38 100755
--- a/build/sbt-launch-lib.bash
+++ b/build/sbt-launch-lib.bash
@@ -51,9 +51,13 @@ acquire_sbt_jar () {
 printf Attempting to fetch sbt\n
 JAR_DL=${JAR}.part
 if [ $(command -v curl) ]; then
-  (curl --silent ${URL1}  ${JAR_DL} || curl --silent ${URL2}  
${JAR_DL})  mv ${JAR_DL} ${JAR}
+  (curl --fail --location --silent ${URL1}  ${JAR_DL} ||\
+(rm -f ${JAR_DL}  curl --fail --location --silent ${URL2}  
${JAR_DL})) \
+mv ${JAR_DL} ${JAR}
 elif [ $(command -v wget) ]; then
-  (wget --quiet ${URL1} -O ${JAR_DL} || wget --quiet ${URL2} -O 
${JAR_DL})  mv ${JAR_DL} ${JAR}
+  (wget --quiet ${URL1} -O ${JAR_DL} ||\
+(rm -f ${JAR_DL}  wget --quiet ${URL2} -O ${JAR_DL})) \
+mv ${JAR_DL} ${JAR}
 else
   printf You do not have curl or wget installed, please install sbt 
manually from http://www.scala-sbt.org/\n;
   exit -1


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



spark git commit: [SPARK-9082] [SQL] [FOLLOW-UP] use `partition` in `PushPredicateThroughProject`

2015-07-23 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 26ed22aec - 52ef76de2


[SPARK-9082] [SQL] [FOLLOW-UP] use `partition` in `PushPredicateThroughProject`

a follow up of https://github.com/apache/spark/pull/7446

Author: Wenchen Fan cloud0...@outlook.com

Closes #7607 from cloud-fan/tmp and squashes the following commits:

7106989 [Wenchen Fan] use `partition` in `PushPredicateThroughProject`


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/52ef76de
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/52ef76de
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/52ef76de

Branch: refs/heads/master
Commit: 52ef76de219c4bf19c54c99414b89a67d0bf457b
Parents: 26ed22a
Author: Wenchen Fan cloud0...@outlook.com
Authored: Thu Jul 23 09:37:53 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Thu Jul 23 09:38:02 2015 -0700

--
 .../sql/catalyst/optimizer/Optimizer.scala  | 22 +++-
 1 file changed, 8 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/52ef76de/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index d2db3dd..b59f800 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -553,33 +553,27 @@ object PushPredicateThroughProject extends 
Rule[LogicalPlan] with PredicateHelpe
   // Split the condition into small conditions by `And`, so that we can 
push down part of this
   // condition without nondeterministic expressions.
   val andConditions = splitConjunctivePredicates(condition)
-  val nondeterministicConditions = 
andConditions.filter(hasNondeterministic(_, aliasMap))
+
+  val (deterministic, nondeterministic) = 
andConditions.partition(_.collect {
+case a: Attribute if aliasMap.contains(a) = aliasMap(a)
+  }.forall(_.deterministic))
 
   // If there is no nondeterministic conditions, push down the whole 
condition.
-  if (nondeterministicConditions.isEmpty) {
+  if (nondeterministic.isEmpty) {
 project.copy(child = Filter(replaceAlias(condition, aliasMap), 
grandChild))
   } else {
 // If they are all nondeterministic conditions, leave it un-changed.
-if (nondeterministicConditions.length == andConditions.length) {
+if (deterministic.isEmpty) {
   filter
 } else {
-  val deterministicConditions = 
andConditions.filterNot(hasNondeterministic(_, aliasMap))
   // Push down the small conditions without nondeterministic 
expressions.
-  val pushedCondition = deterministicConditions.map(replaceAlias(_, 
aliasMap)).reduce(And)
-  Filter(nondeterministicConditions.reduce(And),
+  val pushedCondition = deterministic.map(replaceAlias(_, 
aliasMap)).reduce(And)
+  Filter(nondeterministic.reduce(And),
 project.copy(child = Filter(pushedCondition, grandChild)))
 }
   }
   }
 
-  private def hasNondeterministic(
-  condition: Expression,
-  sourceAliases: AttributeMap[Expression]) = {
-condition.collect {
-  case a: Attribute if sourceAliases.contains(a) = sourceAliases(a)
-}.exists(!_.deterministic)
-  }
-
   // Substitute any attributes that are produced by the child projection, so 
that we safely
   // eliminate it.
   private def replaceAlias(condition: Expression, sourceAliases: 
AttributeMap[Expression]) = {


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



spark git commit: [SPARK-6941] [SQL] Provide a better error message to when inserting into RDD based table

2015-07-16 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master b536d5dc6 - 43dac2c88


[SPARK-6941] [SQL] Provide a better error message to when inserting into RDD 
based table

JIRA: https://issues.apache.org/jira/browse/SPARK-6941

Author: Yijie Shen henry.yijies...@gmail.com

Closes #7342 from yijieshen/SPARK-6941 and squashes the following commits:

f82cbe7 [Yijie Shen] reorder import
dd67e40 [Yijie Shen] resolve comments
09518af [Yijie Shen] fix import order in DataframeSuite
0c635d4 [Yijie Shen] make match more specific
9df388d [Yijie Shen] move check into PreWriteCheck
847ab20 [Yijie Shen] Detect insertion error in DataSourceStrategy


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/43dac2c8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/43dac2c8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/43dac2c8

Branch: refs/heads/master
Commit: 43dac2c880d6f310a958531aee0bb4ac1d9b7025
Parents: b536d5d
Author: Yijie Shen henry.yijies...@gmail.com
Authored: Thu Jul 16 10:52:09 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Thu Jul 16 10:52:09 2015 -0700

--
 .../org/apache/spark/sql/sources/rules.scala|  9 +++-
 .../org/apache/spark/sql/DataFrameSuite.scala   | 55 ++--
 2 files changed, 60 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/43dac2c8/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala
index a3fd7f1..40ee048 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.{SaveMode, AnalysisException}
 import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, Catalog}
 import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Alias}
 import org.apache.spark.sql.catalyst.plans.logical
-import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.types.DataType
 
@@ -119,6 +119,13 @@ private[sql] case class PreWriteCheck(catalog: Catalog) 
extends (LogicalPlan =
 // The relation in l is not an InsertableRelation.
 failAnalysis(s$l does not allow insertion.)
 
+  case logical.InsertIntoTable(t, _, _, _, _) =
+if (!t.isInstanceOf[LeafNode] || t == OneRowRelation || 
t.isInstanceOf[LocalRelation]) {
+  failAnalysis(sInserting into an RDD-based table is not allowed.)
+} else {
+  // OK
+}
+
   case CreateTableUsingAsSelect(tableName, _, _, _, SaveMode.Overwrite, _, 
query) =
 // When the SaveMode is Overwrite, we need to check if the table is an 
input table of
 // the query. If so, we will throw an AnalysisException to let users 
know it is not allowed.

http://git-wip-us.apache.org/repos/asf/spark/blob/43dac2c8/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index f592a99..23244fd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -17,19 +17,23 @@
 
 package org.apache.spark.sql
 
+import java.io.File
+
 import scala.language.postfixOps
 
+import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.types._
-import org.apache.spark.sql.test.{ExamplePointUDT, ExamplePoint}
-
+import org.apache.spark.sql.test.{ExamplePointUDT, ExamplePoint, SQLTestUtils}
 
-class DataFrameSuite extends QueryTest {
+class DataFrameSuite extends QueryTest with SQLTestUtils {
   import org.apache.spark.sql.TestData._
 
   lazy val ctx = org.apache.spark.sql.test.TestSQLContext
   import ctx.implicits._
 
+  def sqlContext: SQLContext = ctx
+
   test(analysis error should be eagerly reported) {
 val oldSetting = ctx.conf.dataFrameEagerAnalysis
 // Eager analysis.
@@ -761,4 +765,49 @@ class DataFrameSuite extends QueryTest {
 assert(f.getMessage.contains(column3))
 assert(!f.getMessage.contains(column2))
   }
+
+  test(SPARK-6941: Better error message for inserting into RDD-based Table) {
+withTempDir { dir =
+
+  val tempParquetFile = new File(dir, tmp_parquet)
+  val tempJsonFile = new 

spark git commit: [SPARK-8800] [SQL] Fix inaccurate precision/scale of Decimal division operation

2015-07-14 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master fb1d06fc2 - 4b5cfc988


[SPARK-8800] [SQL] Fix inaccurate precision/scale of Decimal division operation

JIRA: https://issues.apache.org/jira/browse/SPARK-8800

Previously, we turn to Java BigDecimal's divide with specified ROUNDING_MODE to 
avoid non-terminating decimal expansion problem. However, as JihongMA reported, 
for the division operation on some specific values, we get inaccurate results.

Author: Liang-Chi Hsieh vii...@gmail.com

Closes #7212 from viirya/fix_decimal4 and squashes the following commits:

4205a0a [Liang-Chi Hsieh] Fix inaccuracy precision/scale of Decimal division 
operation.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4b5cfc98
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4b5cfc98
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4b5cfc98

Branch: refs/heads/master
Commit: 4b5cfc988f23988c2334882a255d494fc93d252e
Parents: fb1d06f
Author: Liang-Chi Hsieh vii...@gmail.com
Authored: Tue Jul 14 14:19:27 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Tue Jul 14 14:19:27 2015 -0700

--
 .../scala/org/apache/spark/sql/types/Decimal.scala| 14 +++---
 .../apache/spark/sql/types/decimal/DecimalSuite.scala | 10 +-
 2 files changed, 20 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4b5cfc98/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
index 5a16948..f5bd068 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
@@ -145,6 +145,14 @@ final class Decimal extends Ordered[Decimal] with 
Serializable {
 }
   }
 
+  def toLimitedBigDecimal: BigDecimal = {
+if (decimalVal.ne(null)) {
+  decimalVal
+} else {
+  BigDecimal(longVal, _scale)
+}
+  }
+
   def toJavaBigDecimal: java.math.BigDecimal = toBigDecimal.underlying()
 
   def toUnscaledLong: Long = {
@@ -269,9 +277,9 @@ final class Decimal extends Ordered[Decimal] with 
Serializable {
 if (that.isZero) {
   null
 } else {
-  // To avoid non-terminating decimal expansion problem, we turn to Java 
BigDecimal's divide
-  // with specified ROUNDING_MODE.
-  Decimal(toJavaBigDecimal.divide(that.toJavaBigDecimal, ROUNDING_MODE.id))
+  // To avoid non-terminating decimal expansion problem, we get scala's 
BigDecimal with limited
+  // precision and scala.
+  Decimal(toLimitedBigDecimal / that.toLimitedBigDecimal)
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4b5cfc98/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
index 5f31296..030bb6d 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
@@ -170,6 +170,14 @@ class DecimalSuite extends SparkFunSuite with 
PrivateMethodTester {
 
   test(fix non-terminating decimal expansion problem) {
 val decimal = Decimal(1.0, 10, 3) / Decimal(3.0, 10, 3)
-assert(decimal.toString === 0.333)
+// The difference between decimal should not be more than 0.001.
+assert(decimal.toDouble - 0.333  0.001)
+  }
+
+  test(fix loss of precision/scale when doing division operation) {
+val a = Decimal(2) / Decimal(3)
+assert(a.toDouble  1.0  a.toDouble  0.6)
+val b = Decimal(1) / Decimal(8)
+assert(b.toDouble === 0.125)
   }
 }


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



spark git commit: [SPARK-9060] [SQL] Revert SPARK-8359, SPARK-8800, and SPARK-8677

2015-07-15 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 73d92b00b - 9c64a75bf


[SPARK-9060] [SQL] Revert SPARK-8359, SPARK-8800, and SPARK-8677

JIRA: https://issues.apache.org/jira/browse/SPARK-9060

This PR reverts:
* 
https://github.com/apache/spark/commit/31bd30687bc29c0e457c37308d489ae2b6e5b72a 
(SPARK-8359)
* 
https://github.com/apache/spark/commit/24fda7381171738cbbbacb5965393b660763e562 
(SPARK-8677)
* 
https://github.com/apache/spark/commit/4b5cfc988f23988c2334882a255d494fc93d252e 
(SPARK-8800)

Author: Yin Huai yh...@databricks.com

Closes #7426 from yhuai/SPARK-9060 and squashes the following commits:

651264d [Yin Huai] Revert [SPARK-8359] [SQL] Fix incorrect decimal precision 
after multiplication
cfda7e4 [Yin Huai] Revert [SPARK-8677] [SQL] Fix non-terminating decimal 
expansion for decimal divide operation
2de9afe [Yin Huai] Revert [SPARK-8800] [SQL] Fix inaccurate precision/scale of 
Decimal division operation


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9c64a75b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9c64a75b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9c64a75b

Branch: refs/heads/master
Commit: 9c64a75bfc5e2566d1b4cd0d9b4585a818086ca6
Parents: 73d92b0
Author: Yin Huai yh...@databricks.com
Authored: Wed Jul 15 21:08:30 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Jul 15 21:08:30 2015 -0700

--
 .../org/apache/spark/sql/types/Decimal.scala| 21 ++--
 .../spark/sql/types/decimal/DecimalSuite.scala  | 18 -
 2 files changed, 2 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9c64a75b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
index f5bd068..a85af9e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
@@ -17,8 +17,6 @@
 
 package org.apache.spark.sql.types
 
-import java.math.{MathContext, RoundingMode}
-
 import org.apache.spark.annotation.DeveloperApi
 
 /**
@@ -139,14 +137,6 @@ final class Decimal extends Ordered[Decimal] with 
Serializable {
 
   def toBigDecimal: BigDecimal = {
 if (decimalVal.ne(null)) {
-  decimalVal(MathContext.UNLIMITED)
-} else {
-  BigDecimal(longVal, _scale)(MathContext.UNLIMITED)
-}
-  }
-
-  def toLimitedBigDecimal: BigDecimal = {
-if (decimalVal.ne(null)) {
   decimalVal
 } else {
   BigDecimal(longVal, _scale)
@@ -273,15 +263,8 @@ final class Decimal extends Ordered[Decimal] with 
Serializable {
 
   def * (that: Decimal): Decimal = Decimal(toBigDecimal * that.toBigDecimal)
 
-  def / (that: Decimal): Decimal = {
-if (that.isZero) {
-  null
-} else {
-  // To avoid non-terminating decimal expansion problem, we get scala's 
BigDecimal with limited
-  // precision and scala.
-  Decimal(toLimitedBigDecimal / that.toLimitedBigDecimal)
-}
-  }
+  def / (that: Decimal): Decimal =
+if (that.isZero) null else Decimal(toBigDecimal / that.toBigDecimal)
 
   def % (that: Decimal): Decimal =
 if (that.isZero) null else Decimal(toBigDecimal % that.toBigDecimal)

http://git-wip-us.apache.org/repos/asf/spark/blob/9c64a75b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
index f0c849d..1d297be 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
@@ -171,22 +171,4 @@ class DecimalSuite extends SparkFunSuite with 
PrivateMethodTester {
 assert(new Decimal().set(100L, 10, 0).toUnscaledLong === 100L)
 assert(Decimal(Long.MaxValue, 100, 0).toUnscaledLong === Long.MaxValue)
   }
-
-  test(accurate precision after multiplication) {
-val decimal = (Decimal(Long.MaxValue, 38, 0) * Decimal(Long.MaxValue, 38, 
0)).toJavaBigDecimal
-assert(decimal.unscaledValue.toString === 
85070591730234615847396907784232501249)
-  }
-
-  test(fix non-terminating decimal expansion problem) {
-val decimal = Decimal(1.0, 10, 3) / Decimal(3.0, 10, 3)
-// The difference between decimal should not be more than 0.001.
-assert(decimal.toDouble - 0.333  0.001)
-  }
-
-  test(fix loss of precision/scale when doing division

spark git commit: [SPARK-8972] [SQL] Incorrect result for rollup

2015-07-16 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master ba3309684 - e27212317


[SPARK-8972] [SQL] Incorrect result for rollup

We don't support the complex expression keys in the rollup/cube, and we even 
will not report it if we have the complex group by keys, that will cause very 
confusing/incorrect result.

e.g. `SELECT key%100 FROM src GROUP BY key %100 with ROLLUP`

This PR adds an additional project during the analyzing for the complex GROUP 
BY keys, and that projection will be the child of `Expand`, so to `Expand`, the 
GROUP BY KEY are always the simple key(attribute names).

Author: Cheng Hao hao.ch...@intel.com

Closes #7343 from chenghao-intel/expand and squashes the following commits:

1ebbb59 [Cheng Hao] update the comment
827873f [Cheng Hao] update as feedback
34def69 [Cheng Hao] Add more unit test and comments
c695760 [Cheng Hao] fix bug of incorrect result for rollup


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e2721231
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e2721231
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e2721231

Branch: refs/heads/master
Commit: e27212317c7341852c52d9a85137b8f94cb0d935
Parents: ba33096
Author: Cheng Hao hao.ch...@intel.com
Authored: Wed Jul 15 23:35:27 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Wed Jul 15 23:35:27 2015 -0700

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 42 +--
 ...r CUBE #1-0-63b61fb3f0e74226001ad279be440864 |  6 +++
 ...r CUBE #2-0-7a511f02a16f0af4f810b1666cfcd896 | 10 
 ...oupingSet-0-8c14c24670a4b06c440346277ce9cf1c | 10 
 ...Rollup #1-0-a78e3dbf242f240249e36b3d3fd0926a |  6 +++
 ...Rollup #2-0-bf180c9d1a18f61b9d9f31bb0115cf89 | 10 
 ...Rollup #3-0-9257085d123728730be96b6d9fbb84ce | 10 
 .../sql/hive/execution/HiveQuerySuite.scala | 54 
 8 files changed, 145 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e2721231/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 891408e..df8e7f2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -194,16 +194,52 @@ class Analyzer(
 }
 
 def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+  case a if !a.childrenResolved = a // be sure all of the children are 
resolved.
   case a: Cube =
 GroupingSets(bitmasks(a), a.groupByExprs, a.child, a.aggregations)
   case a: Rollup =
 GroupingSets(bitmasks(a), a.groupByExprs, a.child, a.aggregations)
   case x: GroupingSets =
 val gid = AttributeReference(VirtualColumn.groupingIdName, 
IntegerType, false)()
+// We will insert another Projection if the GROUP BY keys contains the
+// non-attribute expressions. And the top operators can references 
those
+// expressions by its alias.
+// e.g. SELECT key%5 as c1 FROM src GROUP BY key%5 ==
+//  SELECT a as c1 FROM (SELECT key%5 AS a FROM src) GROUP BY a
+
+// find all of the non-attribute expressions in the GROUP BY keys
+val nonAttributeGroupByExpressions = new ArrayBuffer[Alias]()
+
+// The pair of (the original GROUP BY key, associated attribute)
+val groupByExprPairs = x.groupByExprs.map(_ match {
+  case e: NamedExpression = (e, e.toAttribute)
+  case other = {
+val alias = Alias(other, other.toString)()
+nonAttributeGroupByExpressions += alias // add the non-attributes 
expression alias
+(other, alias.toAttribute)
+  }
+})
+
+// substitute the non-attribute expressions for aggregations.
+val aggregation = x.aggregations.map(expr = expr.transformDown {
+  case e = 
groupByExprPairs.find(_._1.semanticEquals(e)).map(_._2).getOrElse(e)
+}.asInstanceOf[NamedExpression])
+
+// substitute the group by expressions.
+val newGroupByExprs = groupByExprPairs.map(_._2)
+
+val child = if (nonAttributeGroupByExpressions.length  0) {
+  // insert additional projection if contains the
+  // non-attribute expressions in the GROUP BY keys
+  Project(x.child.output ++ nonAttributeGroupByExpressions, x.child)
+} else {
+  x.child
+}
+
 Aggregate(
-  x.groupByExprs :+ VirtualColumn.groupingIdAttribute,
-  x.aggregations,
-  

spark git commit: [SPARK-9102] [SQL] Improve project collapse with nondeterministic expressions

2015-07-17 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 111c05538 - 3f6d28a5c


[SPARK-9102] [SQL] Improve project collapse with nondeterministic expressions

Currently we will stop project collapse when the lower projection has 
nondeterministic expressions. However it's overkill sometimes, we should be 
able to optimize `df.select(Rand(10)).select('a)` to `df.select('a)`

Author: Wenchen Fan cloud0...@outlook.com

Closes #7445 from cloud-fan/non-deterministic and squashes the following 
commits:

0deaef6 [Wenchen Fan] Improve project collapse with nondeterministic expressions


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3f6d28a5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3f6d28a5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3f6d28a5

Branch: refs/heads/master
Commit: 3f6d28a5ca98cf7d20c2c029094350cc4f9545a0
Parents: 111c055
Author: Wenchen Fan cloud0...@outlook.com
Authored: Fri Jul 17 00:59:15 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Fri Jul 17 00:59:15 2015 -0700

--
 .../sql/catalyst/optimizer/Optimizer.scala  | 38 ++--
 .../optimizer/ProjectCollapsingSuite.scala  | 26 ++
 .../org/apache/spark/sql/DataFrameSuite.scala   | 10 +++---
 3 files changed, 51 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3f6d28a5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 2f94b45..d5beeec 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -206,31 +206,33 @@ object ColumnPruning extends Rule[LogicalPlan] {
  */
 object ProjectCollapsing extends Rule[LogicalPlan] {
 
-  /** Returns true if any expression in projectList is non-deterministic. */
-  private def hasNondeterministic(projectList: Seq[NamedExpression]): Boolean 
= {
-projectList.exists(expr = expr.find(!_.deterministic).isDefined)
-  }
-
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-// We only collapse these two Projects if the child Project's expressions 
are all
-// deterministic.
-case Project(projectList1, Project(projectList2, child))
- if !hasNondeterministic(projectList2) =
+case p @ Project(projectList1, Project(projectList2, child)) =
   // Create a map of Aliases to their values from the child projection.
   // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c - 
Alias(a + b, c)).
   val aliasMap = AttributeMap(projectList2.collect {
-case a @ Alias(e, _) = (a.toAttribute, a)
+case a: Alias = (a.toAttribute, a)
   })
 
-  // Substitute any attributes that are produced by the child projection, 
so that we safely
-  // eliminate it.
-  // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + 
b + 1 ...'
-  // TODO: Fix TransformBase to avoid the cast below.
-  val substitutedProjection = projectList1.map(_.transform {
-case a: Attribute if aliasMap.contains(a) = aliasMap(a)
-  }).asInstanceOf[Seq[NamedExpression]]
+  // We only collapse these two Projects if their overlapped expressions 
are all
+  // deterministic.
+  val hasNondeterministic = projectList1.flatMap(_.collect {
+case a: Attribute if aliasMap.contains(a) = aliasMap(a).child
+  }).exists(_.find(!_.deterministic).isDefined)
 
-  Project(substitutedProjection, child)
+  if (hasNondeterministic) {
+p
+  } else {
+// Substitute any attributes that are produced by the child 
projection, so that we safely
+// eliminate it.
+// e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a 
+ b + 1 ...'
+// TODO: Fix TransformBase to avoid the cast below.
+val substitutedProjection = projectList1.map(_.transform {
+  case a: Attribute = aliasMap.getOrElse(a, a)
+}).asInstanceOf[Seq[NamedExpression]]
+
+Project(substitutedProjection, child)
+  }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/3f6d28a5/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ProjectCollapsingSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ProjectCollapsingSuite.scala
 

spark git commit: [SPARK-8638] [SQL] Window Function Performance Improvements

2015-07-19 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 04c1b49f5 - a9a0d0ceb


[SPARK-8638] [SQL] Window Function Performance Improvements

## Description
Performance improvements for Spark Window functions. This PR will also serve as 
the basis for moving away from Hive UDAFs to Spark UDAFs. See JIRA tickets 
SPARK-8638 and SPARK-7712 for more information.

## Improvements
* Much better performance (10x) in running cases (e.g. BETWEEN UNBOUNDED 
PRECEDING AND CURRENT ROW) and UNBOUDED FOLLOWING cases. The current 
implementation in spark uses a sliding window approach in these cases. This 
means that an aggregate is maintained for every row, so space usage is N (N 
being the number of rows). This also means that all these aggregates all need 
to be updated separately, this takes N*(N-1)/2 updates. The running case 
differs from the Sliding case because we are only adding data to an aggregate 
function (no reset is required), we only need to maintain one aggregate (like 
in the UNBOUNDED PRECEDING AND UNBOUNDED case), update the aggregate for each 
row, and get the aggregate value after each update. This is what the new 
implementation does. This approach only uses 1 buffer, and only requires N 
updates; I am currently working on data with window sizes of 500-1000 doing 
running sums and this saves a lot of time. The CURRENT ROW AND UNBOUNDED 
FOLLOWING case also us
 es this approach and the fact that aggregate operations are communitative, 
there is one twist though it will process the input buffer in reverse.
* Fewer comparisons in the sliding case. The current implementation determines 
frame boundaries for every input row. The new implementation makes more use of 
the fact that the window is sorted, maintains the boundaries, and only moves 
them when the current row order changes. This is a minor improvement.
* A single Window node is able to process all types of Frames for the same 
Partitioning/Ordering. This saves a little time/memory spent buffering and 
managing partitions. This will be enabled in a follow-up PR.
* A lot of the staging code is moved from the execution phase to the 
initialization phase. Minor performance improvement, and improves readability 
of the execution code.

## Benchmarking
I have done a small benchmark using [on time 
performance](http://www.transtats.bts.gov) data of the month april. I have used 
the origin as a partioning key, as a result there is quite some variation in 
window sizes. The code for the benchmark can be found in the JIRA ticket. These 
are the results per Frame type:

Frame | Master | SPARK-8638
- | -- | --
Entire Frame | 2 s | 1 s
Sliding | 18 s | 1 s
Growing | 14 s | 0.9 s
Shrinking | 13 s | 1 s

Author: Herman van Hovell hvanhov...@questtec.nl

Closes #7057 from hvanhovell/SPARK-8638 and squashes the following commits:

3bfdc49 [Herman van Hovell] Fixed Perfomance Regression for Shrinking Window 
Frames (+Rebase)
2eb3b33 [Herman van Hovell] Corrected reverse range frame processing.
2cd2d5b [Herman van Hovell] Corrected reverse range frame processing.
b0654d7 [Herman van Hovell] Tests for exotic frame specifications.
e75b76e [Herman van Hovell] More docs, added support for reverse sliding range 
frames, and some reorganization of code.
1fdb558 [Herman van Hovell] Changed Data In HiveDataFrameWindowSuite.
ac2f682 [Herman van Hovell] Added a few more comments.
1938312 [Herman van Hovell] Added Documentation to the createBoundOrdering 
methods.
bb020e6 [Herman van Hovell] Major overhaul of Window operator.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a9a0d0ce
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a9a0d0ce
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a9a0d0ce

Branch: refs/heads/master
Commit: a9a0d0cebf8ab3c539723488e5945794ebfd6104
Parents: 04c1b49
Author: Herman van Hovell hvanhov...@questtec.nl
Authored: Sat Jul 18 23:44:38 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Sat Jul 18 23:44:38 2015 -0700

--
 .../expressions/windowExpressions.scala |   12 +
 .../org/apache/spark/sql/execution/Window.scala | 1072 +++---
 .../sql/hive/HiveDataFrameWindowSuite.scala |6 +-
 .../spark/sql/hive/execution/WindowSuite.scala  |   79 ++
 4 files changed, 765 insertions(+), 404 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a9a0d0ce/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index 50bbfd6..09ec0e3 100644
--- 

spark git commit: [SPARK-8638] [SQL] Window Function Performance Improvements - Cleanup

2015-07-19 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master a803ac3e0 - 7a8124534


[SPARK-8638] [SQL] Window Function Performance Improvements - Cleanup

This PR contains a few clean-ups that are a part of SPARK-8638: a few style 
issues got fixed, and a few tests were moved.

Git commit message is wrong BTW :(...

Author: Herman van Hovell hvanhov...@questtec.nl

Closes #7513 from hvanhovell/SPARK-8638-cleanup and squashes the following 
commits:

4e69d08 [Herman van Hovell] Fixed Perfomance Regression for Shrinking Window 
Frames (+Rebase)


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7a812453
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7a812453
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7a812453

Branch: refs/heads/master
Commit: 7a81245345f2d6124423161786bb0d9f1c278ab8
Parents: a803ac3
Author: Herman van Hovell hvanhov...@questtec.nl
Authored: Sun Jul 19 16:29:50 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Sun Jul 19 16:29:50 2015 -0700

--
 .../org/apache/spark/sql/execution/Window.scala | 14 ++--
 .../sql/hive/HiveDataFrameWindowSuite.scala | 43 +++
 .../spark/sql/hive/execution/WindowSuite.scala  | 79 
 3 files changed, 51 insertions(+), 85 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7a812453/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
index a054f52..de04132 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
@@ -118,22 +118,24 @@ case class Window(
   val exprs = windowSpec.orderSpec.map(_.child)
   val projection = newMutableProjection(exprs, child.output)
   (windowSpec.orderSpec, projection(), projection())
-}
-else if (windowSpec.orderSpec.size == 1) {
+} else if (windowSpec.orderSpec.size == 1) {
   // Use only the first order expression when the offset is non-null.
   val sortExpr = windowSpec.orderSpec.head
   val expr = sortExpr.child
   // Create the projection which returns the current 'value'.
   val current = newMutableProjection(expr :: Nil, child.output)()
   // Flip the sign of the offset when processing the order is 
descending
-  val boundOffset = if (sortExpr.direction == Descending) -offset
-  else offset
+  val boundOffset =
+if (sortExpr.direction == Descending) {
+  -offset
+} else {
+  offset
+}
   // Create the projection which returns the current 'value' modified 
by adding the offset.
   val boundExpr = Add(expr, Cast(Literal.create(boundOffset, 
IntegerType), expr.dataType))
   val bound = newMutableProjection(boundExpr :: Nil, child.output)()
   (sortExpr :: Nil, current, bound)
-}
-else {
+} else {
   sys.error(Non-Zero range offsets are not supported for windows  +
 with multiple order expressions.)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/7a812453/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameWindowSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameWindowSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameWindowSuite.scala
index 15b5f41..c177cbd 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameWindowSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameWindowSuite.scala
@@ -212,4 +212,47 @@ class HiveDataFrameWindowSuite extends QueryTest {
   |   (PARTITION BY value ORDER BY key RANGE BETWEEN 1 preceding and 
current row)
   | FROM window_table.stripMargin).collect())
   }
+
+  test(reverse sliding range frame) {
+val df = Seq(
+  (1, Thin, Cell Phone, 6000),
+  (2, Normal, Tablet, 1500),
+  (3, Mini, Tablet, 5500),
+  (4, Ultra thin, Cell Phone, 5500),
+  (5, Very thin, Cell Phone, 6000),
+  (6, Big, Tablet, 2500),
+  (7, Bendable, Cell Phone, 3000),
+  (8, Foldable, Cell Phone, 3000),
+  (9, Pro, Tablet, 4500),
+  (10, Pro2, Tablet, 6500)).
+  toDF(id, product, category, revenue)
+val window = Window.
+  partitionBy($category).
+  orderBy($revenue.desc).
+  rangeBetween(-2000L, 1000L)
+checkAnswer(
+  df.select(
+$id,
+

spark git commit: [SPARK-10144] [UI] Actually show peak execution memory by default

2015-08-24 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 9ce0c7ad3 - 662bb9667


[SPARK-10144] [UI] Actually show peak execution memory by default

The peak execution memory metric was introduced in SPARK-8735. That was before 
Tungsten was enabled by default, so it assumed that `spark.sql.unsafe.enabled` 
must be explicitly set to true. The result is that the memory is not displayed 
by default.

Author: Andrew Or and...@databricks.com

Closes #8345 from andrewor14/show-memory-default.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/662bb966
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/662bb966
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/662bb966

Branch: refs/heads/master
Commit: 662bb9667669cb07cf6d2ccee0d8e76bb561cd89
Parents: 9ce0c7a
Author: Andrew Or and...@databricks.com
Authored: Mon Aug 24 14:10:50 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Mon Aug 24 14:10:50 2015 -0700

--
 core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 6 ++
 core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala | 8 ++--
 2 files changed, 8 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/662bb966/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala 
b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index fb4556b..4adc659 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -68,8 +68,7 @@ private[ui] class StagePage(parent: StagesTab) extends 
WebUIPage(stage) {
   // if we find that it's okay.
   private val MAX_TIMELINE_TASKS = 
parent.conf.getInt(spark.ui.timeline.tasks.maximum, 1000)
 
-  private val displayPeakExecutionMemory =
-parent.conf.getOption(spark.sql.unsafe.enabled).exists(_.toBoolean)
+  private val displayPeakExecutionMemory = 
parent.conf.getBoolean(spark.sql.unsafe.enabled, true)
 
   def render(request: HttpServletRequest): Seq[Node] = {
 progressListener.synchronized {
@@ -1193,8 +1192,7 @@ private[ui] class TaskPagedTable(
 desc: Boolean) extends PagedTable[TaskTableRowData] {
 
   // We only track peak memory used for unsafe operators
-  private val displayPeakExecutionMemory =
-conf.getOption(spark.sql.unsafe.enabled).exists(_.toBoolean)
+  private val displayPeakExecutionMemory = 
conf.getBoolean(spark.sql.unsafe.enabled, true)
 
   override def tableId: String = task-table
 

http://git-wip-us.apache.org/repos/asf/spark/blob/662bb966/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
--
diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala 
b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
index 98f9314..3388c6d 100644
--- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
@@ -33,14 +33,18 @@ class StagePageSuite extends SparkFunSuite with 
LocalSparkContext {
 
   test(peak execution memory only displayed if unsafe is enabled) {
 val unsafeConf = spark.sql.unsafe.enabled
-val conf = new SparkConf().set(unsafeConf, true)
+val conf = new SparkConf(false).set(unsafeConf, true)
 val html = renderStagePage(conf).toString().toLowerCase
 val targetString = peak execution memory
 assert(html.contains(targetString))
 // Disable unsafe and make sure it's not there
-val conf2 = new SparkConf().set(unsafeConf, false)
+val conf2 = new SparkConf(false).set(unsafeConf, false)
 val html2 = renderStagePage(conf2).toString().toLowerCase
 assert(!html2.contains(targetString))
+// Avoid setting anything; it should be displayed by default
+val conf3 = new SparkConf(false)
+val html3 = renderStagePage(conf3).toString().toLowerCase
+assert(html3.contains(targetString))
   }
 
   /**


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



spark git commit: [SPARK-10144] [UI] Actually show peak execution memory by default

2015-08-24 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 43dcf95e4 - 831f78ee5


[SPARK-10144] [UI] Actually show peak execution memory by default

The peak execution memory metric was introduced in SPARK-8735. That was before 
Tungsten was enabled by default, so it assumed that `spark.sql.unsafe.enabled` 
must be explicitly set to true. The result is that the memory is not displayed 
by default.

Author: Andrew Or and...@databricks.com

Closes #8345 from andrewor14/show-memory-default.

(cherry picked from commit 662bb9667669cb07cf6d2ccee0d8e76bb561cd89)
Signed-off-by: Yin Huai yh...@databricks.com


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/831f78ee
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/831f78ee
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/831f78ee

Branch: refs/heads/branch-1.5
Commit: 831f78ee5d2deed9b529214b2613c7e972453514
Parents: 43dcf95
Author: Andrew Or and...@databricks.com
Authored: Mon Aug 24 14:10:50 2015 -0700
Committer: Yin Huai yh...@databricks.com
Committed: Mon Aug 24 14:11:03 2015 -0700

--
 core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 6 ++
 core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala | 8 ++--
 2 files changed, 8 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/831f78ee/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala 
b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index fb4556b..4adc659 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -68,8 +68,7 @@ private[ui] class StagePage(parent: StagesTab) extends 
WebUIPage(stage) {
   // if we find that it's okay.
   private val MAX_TIMELINE_TASKS = 
parent.conf.getInt(spark.ui.timeline.tasks.maximum, 1000)
 
-  private val displayPeakExecutionMemory =
-parent.conf.getOption(spark.sql.unsafe.enabled).exists(_.toBoolean)
+  private val displayPeakExecutionMemory = 
parent.conf.getBoolean(spark.sql.unsafe.enabled, true)
 
   def render(request: HttpServletRequest): Seq[Node] = {
 progressListener.synchronized {
@@ -1193,8 +1192,7 @@ private[ui] class TaskPagedTable(
 desc: Boolean) extends PagedTable[TaskTableRowData] {
 
   // We only track peak memory used for unsafe operators
-  private val displayPeakExecutionMemory =
-conf.getOption(spark.sql.unsafe.enabled).exists(_.toBoolean)
+  private val displayPeakExecutionMemory = 
conf.getBoolean(spark.sql.unsafe.enabled, true)
 
   override def tableId: String = task-table
 

http://git-wip-us.apache.org/repos/asf/spark/blob/831f78ee/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
--
diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala 
b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
index 98f9314..3388c6d 100644
--- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
@@ -33,14 +33,18 @@ class StagePageSuite extends SparkFunSuite with 
LocalSparkContext {
 
   test(peak execution memory only displayed if unsafe is enabled) {
 val unsafeConf = spark.sql.unsafe.enabled
-val conf = new SparkConf().set(unsafeConf, true)
+val conf = new SparkConf(false).set(unsafeConf, true)
 val html = renderStagePage(conf).toString().toLowerCase
 val targetString = peak execution memory
 assert(html.contains(targetString))
 // Disable unsafe and make sure it's not there
-val conf2 = new SparkConf().set(unsafeConf, false)
+val conf2 = new SparkConf(false).set(unsafeConf, false)
 val html2 = renderStagePage(conf2).toString().toLowerCase
 assert(!html2.contains(targetString))
+// Avoid setting anything; it should be displayed by default
+val conf3 = new SparkConf(false)
+val html3 = renderStagePage(conf3).toString().toLowerCase
+assert(html3.contains(targetString))
   }
 
   /**


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



spark git commit: [SPARK-11253] [SQL] reset all accumulators in physical operators before execute an action

2015-10-25 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 87f82a5fb -> 07ced4342


[SPARK-11253] [SQL] reset all accumulators in physical operators before execute 
an action

With this change, our query execution listener can get the metrics correctly.

The UI still looks good after this change.
https://cloud.githubusercontent.com/assets/3182036/10683834/d516f37e-7978-11e5-8118-343ed40eb824.png;>
https://cloud.githubusercontent.com/assets/3182036/10683837/e1fa60da-7978-11e5-8ec8-178b88f27764.png;>

Author: Wenchen Fan 

Closes #9215 from cloud-fan/metric.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/07ced434
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/07ced434
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/07ced434

Branch: refs/heads/master
Commit: 07ced43424447699e47106de9ca2fa714756bdeb
Parents: 87f82a5
Author: Wenchen Fan 
Authored: Sun Oct 25 22:47:39 2015 -0700
Committer: Yin Huai 
Committed: Sun Oct 25 22:47:39 2015 -0700

--
 .../scala/org/apache/spark/sql/DataFrame.scala  |  3 +
 .../spark/sql/execution/metric/SQLMetrics.scala |  7 +-
 .../spark/sql/util/DataFrameCallbackSuite.scala | 81 +++-
 3 files changed, 87 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/07ced434/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
index bf25bcd..25ad3bb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -1974,6 +1974,9 @@ class DataFrame private[sql](
*/
   private def withCallback[T](name: String, df: DataFrame)(action: DataFrame 
=> T) = {
 try {
+  df.queryExecution.executedPlan.foreach { plan =>
+plan.metrics.valuesIterator.foreach(_.reset())
+  }
   val start = System.nanoTime()
   val result = action(df)
   val end = System.nanoTime()

http://git-wip-us.apache.org/repos/asf/spark/blob/07ced434/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
index 075b7ad..1c253e3 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
@@ -28,7 +28,12 @@ import org.apache.spark.{Accumulable, AccumulableParam, 
SparkContext}
  */
 private[sql] abstract class SQLMetric[R <: SQLMetricValue[T], T](
 name: String, val param: SQLMetricParam[R, T])
-  extends Accumulable[R, T](param.zero, param, Some(name), true)
+  extends Accumulable[R, T](param.zero, param, Some(name), true) {
+
+  def reset(): Unit = {
+this.value = param.zero
+  }
+}
 
 /**
  * Create a layer for specialized metric. We cannot add `@specialized` to

http://git-wip-us.apache.org/repos/asf/spark/blob/07ced434/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
index eb056cd..b46b0d2 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
@@ -17,14 +17,14 @@
 
 package org.apache.spark.sql.util
 
-import org.apache.spark.SparkException
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark._
 import org.apache.spark.sql.{functions, QueryTest}
 import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Project}
 import org.apache.spark.sql.execution.QueryExecution
 import org.apache.spark.sql.test.SharedSQLContext
 
-import scala.collection.mutable.ArrayBuffer
-
 class DataFrameCallbackSuite extends QueryTest with SharedSQLContext {
   import testImplicits._
   import functions._
@@ -54,6 +54,8 @@ class DataFrameCallbackSuite extends QueryTest with 
SharedSQLContext {
 assert(metrics(1)._1 == "count")
 assert(metrics(1)._2.analyzed.isInstanceOf[Aggregate])
 assert(metrics(1)._3 > 0)
+
+sqlContext.listenerManager.unregister(listener)
   }
 
   test("execute callback functions when a DataFrame action failed") {
@@ -79,5 +81,78 @@ class 

spark git commit: [SPARK-11325] [SQL] Alias 'alias' in Scala's DataFrame API

2015-10-26 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 4bb2b3698 -> d4c397a64


[SPARK-11325] [SQL] Alias 'alias' in Scala's DataFrame API

Author: Nong Li 

Closes #9286 from nongli/spark-11325.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d4c397a6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d4c397a6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d4c397a6

Branch: refs/heads/master
Commit: d4c397a64af4cec899fdaa3e617ed20333cc567d
Parents: 4bb2b36
Author: Nong Li 
Authored: Mon Oct 26 18:27:02 2015 -0700
Committer: Yin Huai 
Committed: Mon Oct 26 18:27:02 2015 -0700

--
 .../main/scala/org/apache/spark/sql/DataFrame.scala   | 14 ++
 .../scala/org/apache/spark/sql/DataFrameSuite.scala   |  7 +++
 2 files changed, 21 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d4c397a6/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
index 25ad3bb..32d9b0b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -699,6 +699,20 @@ class DataFrame private[sql](
   def as(alias: Symbol): DataFrame = as(alias.name)
 
   /**
+   * Returns a new [[DataFrame]] with an alias set. Same as `as`.
+   * @group dfops
+   * @since 1.6.0
+   */
+  def alias(alias: String): DataFrame = as(alias)
+
+  /**
+   * (Scala-specific) Returns a new [[DataFrame]] with an alias set. Same as 
`as`.
+   * @group dfops
+   * @since 1.6.0
+   */
+  def alias(alias: Symbol): DataFrame = as(alias)
+
+  /**
* Selects a set of column based expressions.
* {{{
*   df.select($"colA", $"colB" + 1)

http://git-wip-us.apache.org/repos/asf/spark/blob/d4c397a6/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index f4c7aa3..59565a6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -105,6 +105,13 @@ class DataFrameSuite extends QueryTest with 
SharedSQLContext {
 assert(testData.head(2).head.schema === testData.schema)
   }
 
+  test("dataframe alias") {
+val df = Seq(Tuple1(1)).toDF("c").as("t")
+val dfAlias = df.alias("t2")
+df.col("t.c")
+dfAlias.col("t2.c")
+  }
+
   test("simple explode") {
 val df = Seq(Tuple1("a b c"), Tuple1("d e")).toDF("words")
 


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



spark git commit: [SPARK-10562] [SQL] support mixed case partitionBy column names for tables stored in metastore

2015-10-26 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master dc3220ce1 -> a150e6c1b


[SPARK-10562] [SQL] support mixed case partitionBy column names for tables 
stored in metastore

https://issues.apache.org/jira/browse/SPARK-10562

Author: Wenchen Fan 

Closes #9226 from cloud-fan/par.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a150e6c1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a150e6c1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a150e6c1

Branch: refs/heads/master
Commit: a150e6c1b03b64a35855b8074b2fe077a6081a34
Parents: dc3220c
Author: Wenchen Fan 
Authored: Mon Oct 26 21:14:26 2015 -0700
Committer: Yin Huai 
Committed: Mon Oct 26 21:14:26 2015 -0700

--
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 61 
 .../sql/hive/MetastoreDataSourcesSuite.scala|  9 ++-
 .../sql/hive/execution/SQLQuerySuite.scala  | 11 
 3 files changed, 54 insertions(+), 27 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a150e6c1/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index fdb576b..f4d4571 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -143,6 +143,21 @@ private[hive] class HiveMetastoreCatalog(val client: 
ClientInterface, hive: Hive
   }
 }
 
+def partColsFromParts: Option[Seq[String]] = {
+  table.properties.get("spark.sql.sources.schema.numPartCols").map { 
numPartCols =>
+(0 until numPartCols.toInt).map { index =>
+  val partCol = 
table.properties.get(s"spark.sql.sources.schema.partCol.$index").orNull
+  if (partCol == null) {
+throw new AnalysisException(
+  "Could not read partitioned columns from the metastore 
because it is corrupted " +
+s"(missing part $index of the it, $numPartCols parts are 
expected).")
+  }
+
+  partCol
+}
+  }
+}
+
 // Originally, we used spark.sql.sources.schema to store the schema of 
a data source table.
 // After SPARK-6024, we removed this flag.
 // Although we are not using spark.sql.sources.schema any more, we 
need to still support.
@@ -155,7 +170,7 @@ private[hive] class HiveMetastoreCatalog(val client: 
ClientInterface, hive: Hive
 // We only need names at here since userSpecifiedSchema we loaded from 
the metastore
 // contains partition columns. We can always get datatypes of 
partitioning columns
 // from userSpecifiedSchema.
-val partitionColumns = table.partitionColumns.map(_.name)
+val partitionColumns = partColsFromParts.getOrElse(Nil)
 
 // It does not appear that the ql client for the metastore has a way 
to enumerate all the
 // SerDe properties directly...
@@ -218,25 +233,21 @@ private[hive] class HiveMetastoreCatalog(val client: 
ClientInterface, hive: Hive
   }
 }
 
-val metastorePartitionColumns = userSpecifiedSchema.map { schema =>
-  val fields = partitionColumns.map(col => schema(col))
-  fields.map { field =>
-HiveColumn(
-  name = field.name,
-  hiveType = HiveMetastoreTypes.toMetastoreType(field.dataType),
-  comment = "")
-  }.toSeq
-}.getOrElse {
-  if (partitionColumns.length > 0) {
-// The table does not have a specified schema, which means that the 
schema will be inferred
-// when we load the table. So, we are not expecting partition columns 
and we will discover
-// partitions when we load the table. However, if there are specified 
partition columns,
-// we simply ignore them and provide a warning message.
-logWarning(
-  s"The schema and partitions of table $tableIdent will be inferred 
when it is loaded. " +
-s"Specified partition columns (${partitionColumns.mkString(",")}) 
will be ignored.")
+if (userSpecifiedSchema.isDefined && partitionColumns.length > 0) {
+  tableProperties.put("spark.sql.sources.schema.numPartCols", 
partitionColumns.length.toString)
+  partitionColumns.zipWithIndex.foreach { case (partCol, index) =>
+tableProperties.put(s"spark.sql.sources.schema.partCol.$index", 
partCol)
   }
-  Seq.empty[HiveColumn]
+}
+
+if (userSpecifiedSchema.isEmpty && partitionColumns.length > 0) 

spark git commit: [SPARK-10947] [SQL] With schema inference from JSON into a Dataframe, add option to infer all primitive object types as strings

2015-10-26 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master d4c397a64 -> 82464fb2e


[SPARK-10947] [SQL] With schema inference from JSON into a Dataframe, add 
option to infer all primitive object types as strings

Currently, when a schema is inferred from a JSON file using 
sqlContext.read.json, the primitive object types are inferred as string, long, 
boolean, etc.

However, if the inferred type is too specific (JSON obviously does not enforce 
types itself), this can cause issues with merging dataframe schemas.

This pull request adds the option "primitivesAsString" to the JSON 
DataFrameReader which when true (defaults to false if not set) will infer all 
primitives as strings.

Below is an example usage of this new functionality.
```
val jsonDf = sqlContext.read.option("primitivesAsString", 
"true").json(sampleJsonFile)

scala> jsonDf.printSchema()
root
|-- bigInteger: string (nullable = true)
|-- boolean: string (nullable = true)
|-- double: string (nullable = true)
|-- integer: string (nullable = true)
|-- long: string (nullable = true)
|-- null: string (nullable = true)
|-- string: string (nullable = true)
```

Author: Stephen De Gennaro 

Closes #9249 from stephend-realitymine/stephend-primitives.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/82464fb2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/82464fb2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/82464fb2

Branch: refs/heads/master
Commit: 82464fb2e02ca4e4d425017815090497b79dc93f
Parents: d4c397a
Author: Stephen De Gennaro 
Authored: Mon Oct 26 19:55:10 2015 -0700
Committer: Yin Huai 
Committed: Mon Oct 26 19:55:10 2015 -0700

--
 .../org/apache/spark/sql/DataFrameReader.scala  |  10 +-
 .../datasources/json/InferSchema.scala  |  20 ++-
 .../datasources/json/JSONRelation.scala |  14 +-
 .../execution/datasources/json/JsonSuite.scala  | 138 ++-
 4 files changed, 171 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/82464fb2/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
index 824220d..6a194a4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -256,8 +256,16 @@ class DataFrameReader private[sql](sqlContext: SQLContext) 
extends Logging {
*/
   def json(jsonRDD: RDD[String]): DataFrame = {
 val samplingRatio = extraOptions.getOrElse("samplingRatio", "1.0").toDouble
+val primitivesAsString = extraOptions.getOrElse("primitivesAsString", 
"false").toBoolean
 sqlContext.baseRelationToDataFrame(
-  new JSONRelation(Some(jsonRDD), samplingRatio, userSpecifiedSchema, 
None, None)(sqlContext))
+  new JSONRelation(
+Some(jsonRDD),
+samplingRatio,
+primitivesAsString,
+userSpecifiedSchema,
+None,
+None)(sqlContext)
+)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/82464fb2/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala
index d078002..b9914c5 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala
@@ -35,7 +35,8 @@ private[sql] object InferSchema {
   def apply(
   json: RDD[String],
   samplingRatio: Double = 1.0,
-  columnNameOfCorruptRecords: String): StructType = {
+  columnNameOfCorruptRecords: String,
+  primitivesAsString: Boolean = false): StructType = {
 require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be 
greater than 0")
 val schemaData = if (samplingRatio > 0.99) {
   json
@@ -50,7 +51,7 @@ private[sql] object InferSchema {
 try {
   Utils.tryWithResource(factory.createParser(row)) { parser =>
 parser.nextToken()
-inferField(parser)
+inferField(parser, primitivesAsString)
   }
 } catch {
   case _: JsonParseException =>
@@ -70,14 +71,14 @@ private[sql] object InferSchema {
   /**
* Infer the type of a json document from the parser's token stream
*/
-  private def 

spark git commit: [SPARK-11246] [SQL] Table cache for Parquet broken in 1.5

2015-10-29 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 9e3197aaa -> 76d742386


[SPARK-11246] [SQL] Table cache for Parquet broken in 1.5

The root cause is that when spark.sql.hive.convertMetastoreParquet=true by 
default, the cached InMemoryRelation of the ParquetRelation can not be looked 
up from the cachedData of CacheManager because the key comparison fails even 
though it is the same LogicalPlan representing the Subquery that wraps the 
ParquetRelation.
The solution in this PR is overriding the LogicalPlan.sameResult function in 
Subquery case class to eliminate subquery node first before directly comparing 
the child (ParquetRelation), which will find the key  to the cached 
InMemoryRelation.

Author: xin Wu 

Closes #9326 from xwu0226/spark-11246-commit.

(cherry picked from commit f7a51deebad1b4c3b970a051f25d286110b94438)
Signed-off-by: Yin Huai 

Conflicts:
sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/76d74238
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/76d74238
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/76d74238

Branch: refs/heads/branch-1.5
Commit: 76d742386cd045526969a4f8b4c2cf30d54fd30c
Parents: 9e3197a
Author: xin Wu 
Authored: Thu Oct 29 07:42:46 2015 -0700
Committer: Yin Huai 
Committed: Thu Oct 29 07:57:10 2015 -0700

--
 .../spark/sql/execution/datasources/LogicalRelation.scala |  5 +
 .../org/apache/spark/sql/hive/CachedTableSuite.scala  | 10 ++
 2 files changed, 15 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/76d74238/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
index 4069179..c9cc7d5 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
@@ -62,6 +62,11 @@ private[sql] case class LogicalRelation(
 case _ => false
   }
 
+  // When comparing two LogicalRelations from within LogicalPlan.sameResult, 
we only need
+  // LogicalRelation.cleanArgs to return Seq(relation), since 
expectedOutputAttribute's
+  // expId can be different but the relation is still the same.
+  override lazy val cleanArgs: Seq[Any] = Seq(relation)
+
   @transient override lazy val statistics: Statistics = Statistics(
 sizeInBytes = BigInt(relation.sizeInBytes)
   )

http://git-wip-us.apache.org/repos/asf/spark/blob/76d74238/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
index 39d315a..7f7b079 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
@@ -203,4 +203,14 @@ class CachedTableSuite extends QueryTest {
 sql("DROP TABLE refreshTable")
 Utils.deleteRecursively(tempPath)
   }
+
+  test("SPARK-11246 cache parquet table") {
+sql("CREATE TABLE cachedTable STORED AS PARQUET AS SELECT 1")
+
+cacheTable("cachedTable")
+val sparkPlan = sql("SELECT * FROM cachedTable").queryExecution.sparkPlan
+assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e }.size 
=== 1)
+
+sql("DROP TABLE cachedTable")
+  }
 }


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



spark git commit: [SPARK-11032] [SQL] correctly handle having

2015-10-29 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 76d742386 -> bb3b3627a


[SPARK-11032] [SQL] correctly handle having

We should not stop resolving having when the having condtion is resolved, or 
something like `count(1)` will crash.

Author: Wenchen Fan 

Closes #9105 from cloud-fan/having.

(cherry picked from commit e170c22160bb452f98c340489ebf8390116a8cbb)
Signed-off-by: Yin Huai 

Conflicts:
sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bb3b3627
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bb3b3627
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bb3b3627

Branch: refs/heads/branch-1.5
Commit: bb3b3627ac3fcd18be7fb07b6d0ba5eae0342fc3
Parents: 76d7423
Author: Wenchen Fan 
Authored: Tue Oct 13 17:11:22 2015 -0700
Committer: Yin Huai 
Committed: Thu Oct 29 08:05:31 2015 -0700

--
 .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala   | 2 +-
 .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 +
 2 files changed, 10 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/bb3b3627/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 0f459cd..4ce5b11 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -553,7 +553,7 @@ class Analyzer(
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
   case filter @ Filter(havingCondition,
  aggregate @ Aggregate(grouping, originalAggExprs, child))
-  if aggregate.resolved && !filter.resolved =>
+  if aggregate.resolved =>
 
 // Try resolving the condition of the filter as though it is in the 
aggregate clause
 val aggregatedCondition =

http://git-wip-us.apache.org/repos/asf/spark/blob/bb3b3627/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 97be3ee..c255b9e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -1765,4 +1765,13 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
   assert(sampled.count() == sampledOdd.count() + sampledEven.count())
 }
   }
+
+  test("SPARK-11032: resolve having correctly") {
+withTempTable("src") {
+  Seq(1 -> "a").toDF("i", "j").registerTempTable("src")
+  checkAnswer(
+sql("SELECT MIN(t.i) FROM (SELECT * FROM src WHERE i > 0) t 
HAVING(COUNT(1) > 0)"),
+Row(1))
+}
+  }
 }


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



spark git commit: [SPARK-11125] [SQL] Uninformative exception when running spark-sql witho…

2015-10-23 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 5e4581250 -> ffed00493


[SPARK-11125] [SQL] Uninformative exception when running spark-sql witho…

…ut building with -Phive-thriftserver and SPARK_PREPEND_CLASSES is set

This is the exception after this patch. Please help review.
```
java.lang.NoClassDefFoundError: org/apache/hadoop/hive/cli/CliDriver
at java.lang.ClassLoader.defineClass1(Native Method)
at java.lang.ClassLoader.defineClass(ClassLoader.java:800)
at 
java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
at java.net.URLClassLoader.defineClass(URLClassLoader.java:449)
at java.net.URLClassLoader.access$100(URLClassLoader.java:71)
at java.net.URLClassLoader$1.run(URLClassLoader.java:361)
at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
at java.security.AccessController.doPrivileged(Native Method)
at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
at java.lang.ClassLoader.loadClass(ClassLoader.java:412)
at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
at java.lang.Class.forName0(Native Method)
at java.lang.Class.forName(Class.java:270)
at org.apache.spark.util.Utils$.classForName(Utils.scala:173)
at 
org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:647)
at 
org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:180)
at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:205)
at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:120)
at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: java.lang.ClassNotFoundException: 
org.apache.hadoop.hive.cli.CliDriver
at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
at java.security.AccessController.doPrivileged(Native Method)
at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
... 21 more
Failed to load hive class.
You need to build Spark with -Phive and -Phive-thriftserver.
```

Author: Jeff Zhang 

Closes #9134 from zjffdu/SPARK-11125.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ffed0049
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ffed0049
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ffed0049

Branch: refs/heads/master
Commit: ffed00493a0aa2373a04e3aa374404936fbe15c7
Parents: 5e45812
Author: Jeff Zhang 
Authored: Fri Oct 23 22:56:55 2015 -0700
Committer: Yin Huai 
Committed: Fri Oct 23 22:56:55 2015 -0700

--
 .../main/scala/org/apache/spark/deploy/SparkSubmit.scala| 9 +
 1 file changed, 9 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ffed0049/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
--
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala 
b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index ad92f56..640cc32 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -655,6 +655,15 @@ object SparkSubmit {
   // scalastyle:on println
 }
 System.exit(CLASS_NOT_FOUND_EXIT_STATUS)
+  case e: NoClassDefFoundError =>
+e.printStackTrace(printStream)
+if (e.getMessage.contains("org/apache/hadoop/hive")) {
+  // scalastyle:off println
+  printStream.println(s"Failed to load hive class.")
+  printStream.println("You need to build Spark with -Phive and 
-Phive-thriftserver.")
+  // scalastyle:on println
+}
+System.exit(CLASS_NOT_FOUND_EXIT_STATUS)
 }
 
 // SPARK-4170


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



[2/2] spark git commit: [SPARK-11347] [SQL] Support for joinWith in Datasets

2015-10-27 Thread yhuai
[SPARK-11347] [SQL] Support for joinWith in Datasets

This PR adds a new operation `joinWith` to a `Dataset`, which returns a `Tuple` 
for each pair where a given `condition` evaluates to true.

```scala
case class ClassData(a: String, b: Int)

val ds1 = Seq(ClassData("a", 1), ClassData("b", 2)).toDS()
val ds2 = Seq(("a", 1), ("b", 2)).toDS()

> ds1.joinWith(ds2, $"_1" === $"a").collect()
res0: Array((ClassData("a", 1), ("a", 1)), (ClassData("b", 2), ("b", 2)))
```

This operation is similar to the relation `join` function with one important 
difference in the result schema. Since `joinWith` preserves objects present on 
either side of the join, the result schema is similarly nested into a tuple 
under the column names `_1` and `_2`.

This type of join can be useful both for preserving type-safety with the 
original object types as well as working with relational data where either side 
of the join has column names in common.

## Required Changes to Encoders
In the process of working on this patch, several deficiencies to the way that 
we were handling encoders were discovered.  Specifically, it turned out to be 
very difficult to `rebind` the non-expression based encoders to extract the 
nested objects from the results of joins (and also typed selects that return 
tuples).

As a result the following changes were made.
 - `ClassEncoder` has been renamed to `ExpressionEncoder` and has been improved 
to also handle primitive types.  Additionally, it is now possible to take 
arbitrary expression encoders and rewrite them into a single encoder that 
returns a tuple.
 - All internal operations on `Dataset`s now require an `ExpressionEncoder`.  
If the users tries to pass a non-`ExpressionEncoder` in, an error will be 
thrown.  We can relax this requirement in the future by constructing a wrapper 
class that uses expressions to project the row to the expected schema, 
shielding the users code from the required remapping.  This will give us a nice 
balance where we don't force user encoders to understand attribute references 
and binding, but still allow our native encoder to leverage runtime code 
generation to construct specific encoders for a given schema that avoid an 
extra remapping step.
 - Additionally, the semantics for different types of objects are now better 
defined.  As stated in the `ExpressionEncoder` scaladoc:
  - Classes will have their sub fields extracted by name using 
`UnresolvedAttribute` expressions
  and `UnresolvedExtractValue` expressions.
  - Tuples will have their subfields extracted by position using 
`BoundReference` expressions.
  - Primitives will have their values extracted from the first ordinal with a 
schema that defaults
  to the name `value`.
 - Finally, the binding lifecycle for `Encoders` has now been unified across 
the codebase.  Encoders are now `resolved` to the appropriate schema in the 
constructor of `Dataset`.  This process replaces an unresolved expressions with 
concrete `AttributeReference` expressions.  Binding then happens on demand, 
when an encoder is going to be used to construct an object.  This closely 
mirrors the lifecycle for standard expressions when executing normal SQL or 
`DataFrame` queries.

Author: Michael Armbrust 

Closes #9300 from marmbrus/datasets-tuples.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5a5f6590
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5a5f6590
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5a5f6590

Branch: refs/heads/master
Commit: 5a5f65905a202e59bc85170b01c57a883718ddf6
Parents: 3bdbbc6
Author: Michael Armbrust 
Authored: Tue Oct 27 13:28:52 2015 -0700
Committer: Yin Huai 
Committed: Tue Oct 27 13:28:52 2015 -0700

--
 .../spark/sql/catalyst/ScalaReflection.scala|  43 ++-
 .../sql/catalyst/encoders/ClassEncoder.scala| 101 ---
 .../spark/sql/catalyst/encoders/Encoder.scala   |  38 +--
 .../catalyst/encoders/ExpressionEncoder.scala   | 217 ++
 .../sql/catalyst/encoders/ProductEncoder.scala  |  47 ---
 .../sql/catalyst/encoders/RowEncoder.scala  |   5 +-
 .../spark/sql/catalyst/encoders/package.scala   |  26 ++
 .../sql/catalyst/encoders/primitiveTypes.scala  | 100 ---
 .../spark/sql/catalyst/encoders/tuples.scala| 173 ---
 .../catalyst/plans/logical/basicOperators.scala |  28 +-
 .../encoders/ExpressionEncoderSuite.scala   | 291 +++
 .../encoders/PrimitiveEncoderSuite.scala|  43 ---
 .../catalyst/encoders/ProductEncoderSuite.scala | 282 --
 .../scala/org/apache/spark/sql/DataFrame.scala  |   2 +-
 .../scala/org/apache/spark/sql/Dataset.scala| 190 +++-
 .../scala/org/apache/spark/sql/SQLContext.scala |   4 +-
 .../org/apache/spark/sql/SQLImplicits.scala |  13 +-
 

[1/2] spark git commit: [SPARK-11347] [SQL] Support for joinWith in Datasets

2015-10-27 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 3bdbbc6c9 -> 5a5f65905


http://git-wip-us.apache.org/repos/asf/spark/blob/5a5f6590/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 0849624..aebb390 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -34,6 +34,13 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
   data: _*)
   }
 
+  test("as tuple") {
+val data = Seq(("a", 1), ("b", 2)).toDF("a", "b")
+checkAnswer(
+  data.as[(String, Int)],
+  ("a", 1), ("b", 2))
+  }
+
   test("as case class / collect") {
 val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData]
 checkAnswer(
@@ -61,14 +68,40 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
   2, 3, 4)
   }
 
-  test("select 3") {
+  test("select 2") {
 val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS()
 checkAnswer(
   ds.select(
 expr("_1").as[String],
-expr("_2").as[Int],
-expr("_2 + 1").as[Int]),
-  ("a", 1, 2), ("b", 2, 3), ("c", 3, 4))
+expr("_2").as[Int]) : Dataset[(String, Int)],
+  ("a", 1), ("b", 2), ("c", 3))
+  }
+
+  test("select 2, primitive and tuple") {
+val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS()
+checkAnswer(
+  ds.select(
+expr("_1").as[String],
+expr("struct(_2, _2)").as[(Int, Int)]),
+  ("a", (1, 1)), ("b", (2, 2)), ("c", (3, 3)))
+  }
+
+  test("select 2, primitive and class") {
+val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS()
+checkAnswer(
+  ds.select(
+expr("_1").as[String],
+expr("named_struct('a', _1, 'b', _2)").as[ClassData]),
+  ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 
3)))
+  }
+
+  test("select 2, primitive and class, fields reordered") {
+val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS()
+checkDecoding(
+  ds.select(
+expr("_1").as[String],
+expr("named_struct('b', _2, 'a', _1)").as[ClassData]),
+  ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 
3)))
   }
 
   test("filter") {
@@ -102,6 +135,54 @@ class DatasetSuite extends QueryTest with SharedSQLContext 
{
 assert(ds.fold(("", 0))((a, b) => ("sum", a._2 + b._2)) == ("sum", 6))
   }
 
+  test("joinWith, flat schema") {
+val ds1 = Seq(1, 2, 3).toDS().as("a")
+val ds2 = Seq(1, 2).toDS().as("b")
+
+checkAnswer(
+  ds1.joinWith(ds2, $"a.value" === $"b.value"),
+  (1, 1), (2, 2))
+  }
+
+  test("joinWith, expression condition") {
+val ds1 = Seq(ClassData("a", 1), ClassData("b", 2)).toDS()
+val ds2 = Seq(("a", 1), ("b", 2)).toDS()
+
+checkAnswer(
+  ds1.joinWith(ds2, $"_1" === $"a"),
+  (ClassData("a", 1), ("a", 1)), (ClassData("b", 2), ("b", 2)))
+  }
+
+  test("joinWith tuple with primitive, expression") {
+val ds1 = Seq(1, 1, 2).toDS()
+val ds2 = Seq(("a", 1), ("b", 2)).toDS()
+
+checkAnswer(
+  ds1.joinWith(ds2, $"value" === $"_2"),
+  (1, ("a", 1)), (1, ("a", 1)), (2, ("b", 2)))
+  }
+
+  test("joinWith class with primitive, toDF") {
+val ds1 = Seq(1, 1, 2).toDS()
+val ds2 = Seq(ClassData("a", 1), ClassData("b", 2)).toDS()
+
+checkAnswer(
+  ds1.joinWith(ds2, $"value" === $"b").toDF().select($"_1", $"_2.a", 
$"_2.b"),
+  Row(1, "a", 1) :: Row(1, "a", 1) :: Row(2, "b", 2) :: Nil)
+  }
+
+  test("multi-level joinWith") {
+val ds1 = Seq(("a", 1), ("b", 2)).toDS().as("a")
+val ds2 = Seq(("a", 1), ("b", 2)).toDS().as("b")
+val ds3 = Seq(("a", 1), ("b", 2)).toDS().as("c")
+
+checkAnswer(
+  ds1.joinWith(ds2, $"a._2" === $"b._2").as("ab").joinWith(ds3, 
$"ab._1._2" === $"c._2"),
+  ((("a", 1), ("a", 1)), ("a", 1)),
+  ((("b", 2), ("b", 2)), ("b", 2)))
+
+  }
+
   test("groupBy function, keys") {
 val ds = Seq(("a", 1), ("b", 1)).toDS()
 val grouped = ds.groupBy(v => (1, v._2))

http://git-wip-us.apache.org/repos/asf/spark/blob/5a5f6590/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
index aba5675..73e02eb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
@@ -20,12 +20,11 @@ package org.apache.spark.sql
 import java.util.{Locale, TimeZone}
 
 import scala.collection.JavaConverters._
-import scala.reflect.runtime.universe._
 
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.util._
 import 

spark git commit: [SPARK-10484] [SQL] Optimize the cartesian join with broadcast join for some cases

2015-10-27 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master b960a8905 -> d9c603989


[SPARK-10484] [SQL] Optimize the cartesian join with broadcast join for some 
cases

In some cases, we can broadcast the smaller relation in cartesian join, which 
improve the performance significantly.

Author: Cheng Hao 

Closes #8652 from chenghao-intel/cartesian.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d9c60398
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d9c60398
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d9c60398

Branch: refs/heads/master
Commit: d9c6039897236c3f1e4503aa95c5c9b07b32eadd
Parents: b960a89
Author: Cheng Hao 
Authored: Tue Oct 27 20:26:38 2015 -0700
Committer: Yin Huai 
Committed: Tue Oct 27 20:26:38 2015 -0700

--
 .../spark/sql/execution/SparkPlanner.scala  |  3 +-
 .../spark/sql/execution/SparkStrategies.scala   | 38 +---
 .../joins/BroadcastNestedLoopJoin.scala |  7 +-
 .../scala/org/apache/spark/sql/JoinSuite.scala  | 92 
 .../org/apache/spark/sql/hive/HiveContext.scala |  3 +-
 ...ed JOIN #1-0-abfc0b99ee357f71639f6162345fe8e | 20 +
 ...d JOIN #2-0-8412a39ee57885ccb0aaf848db8ef1dd | 20 +
 ...d JOIN #3-0-e8a0427dbde35eea601113e5ffb4 | 20 +
 ...d JOIN #4-0-45f8602d257655322b7d18cad09f6a0f | 20 +
 .../sql/hive/execution/HiveQuerySuite.scala | 54 
 10 files changed, 261 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d9c60398/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala
index b346f43..0f98fe8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala
@@ -44,8 +44,9 @@ class SparkPlanner(val sqlContext: SQLContext) extends 
SparkStrategies {
   EquiJoinSelection ::
   InMemoryScans ::
   BasicOperators ::
+  BroadcastNestedLoop ::
   CartesianProduct ::
-  BroadcastNestedLoopJoin :: Nil)
+  DefaultJoin :: Nil)
 
   /**
* Used to build table scan operators where complex projection and filtering 
are done using

http://git-wip-us.apache.org/repos/asf/spark/blob/d9c60398/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 637deff..ee97162 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -294,25 +294,24 @@ private[sql] abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
 }
   }
 
-
-  object BroadcastNestedLoopJoin extends Strategy {
+  object BroadcastNestedLoop extends Strategy {
 def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
-  case logical.Join(left, right, joinType, condition) =>
-val buildSide =
-  if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) {
-joins.BuildRight
-  } else {
-joins.BuildLeft
-  }
-joins.BroadcastNestedLoopJoin(
-  planLater(left), planLater(right), buildSide, joinType, condition) 
:: Nil
+  case logical.Join(
+ CanBroadcast(left), right, joinType, condition) if joinType != 
LeftSemiJoin =>
+execution.joins.BroadcastNestedLoopJoin(
+  planLater(left), planLater(right), joins.BuildLeft, joinType, 
condition) :: Nil
+  case logical.Join(
+ left, CanBroadcast(right), joinType, condition) if joinType != 
LeftSemiJoin =>
+execution.joins.BroadcastNestedLoopJoin(
+  planLater(left), planLater(right), joins.BuildRight, joinType, 
condition) :: Nil
   case _ => Nil
 }
   }
 
   object CartesianProduct extends Strategy {
 def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
-  case logical.Join(left, right, _, None) =>
+  // TODO CartesianProduct doesn't support the Left Semi Join
+  case logical.Join(left, right, joinType, None) if joinType != 
LeftSemiJoin =>
 execution.joins.CartesianProduct(planLater(left), planLater(right)) :: 
Nil
   case logical.Join(left, right, Inner, Some(condition)) =>
 execution.Filter(condition,
@@ -321,6 +320,21 @@ private[sql] 

spark git commit: [SPARK-11377] [SQL] withNewChildren should not convert StructType to Seq

2015-10-28 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master f92b7b98e -> 032748bb9


[SPARK-11377] [SQL] withNewChildren should not convert StructType to Seq

This is minor, but I ran into while writing Datasets and while it wasn't needed 
for the final solution, it was super confusing so we should fix it.

Basically we recurse into `Seq` to see if they have children.  This breaks 
because we don't preserve the original subclass of `Seq` (and `StructType <:< 
Seq[StructField]`).  Since a struct can never contain children, lets just not 
recurse into it.

Author: Michael Armbrust 

Closes #9334 from marmbrus/structMakeCopy.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/032748bb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/032748bb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/032748bb

Branch: refs/heads/master
Commit: 032748bb9add096e4691551ee73834f3e5363dd5
Parents: f92b7b9
Author: Michael Armbrust 
Authored: Wed Oct 28 09:40:05 2015 -0700
Committer: Yin Huai 
Committed: Wed Oct 28 09:40:05 2015 -0700

--
 .../scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/032748bb/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index 7971e25..35f087b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.catalyst.trees
 
 import org.apache.spark.sql.catalyst.errors._
-import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.{StructType, DataType}
 
 /** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given 
number */
 private class MutableInt(var i: Int)
@@ -176,6 +176,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product {
 val remainingNewChildren = newChildren.toBuffer
 val remainingOldChildren = children.toBuffer
 val newArgs = productIterator.map {
+  case s: StructType => s // Don't convert struct types to some other type 
of Seq[StructField]
   // Handle Seq[TreeNode] in TreeNode parameters.
   case s: Seq[_] => s.map {
 case arg: TreeNode[_] if containsChild(arg) =>
@@ -337,6 +338,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product {
  |Is otherCopyArgs specified correctly for $nodeName.
  |Exception message: ${e.getMessage}
  |ctor: $defaultCtor?
+ |types: ${newArgs.map(_.getClass).mkString(", ")}
  |args: ${newArgs.mkString(", ")}
""".stripMargin)
 }


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



spark git commit: [SPARK-11292] [SQL] Python API for text data source

2015-10-28 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 032748bb9 -> 5aa052191


[SPARK-11292] [SQL] Python API for text data source

Adds DataFrameReader.text and DataFrameWriter.text.

Author: Reynold Xin 

Closes #9259 from rxin/SPARK-11292.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5aa05219
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5aa05219
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5aa05219

Branch: refs/heads/master
Commit: 5aa05219118e3d3525fb703a4716ae8e04f3da72
Parents: 032748b
Author: Reynold Xin 
Authored: Wed Oct 28 14:28:38 2015 -0700
Committer: Yin Huai 
Committed: Wed Oct 28 14:28:38 2015 -0700

--
 python/pyspark/sql/readwriter.py  | 27 +--
 python/test_support/sql/text-test.txt |  2 ++
 2 files changed, 27 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5aa05219/python/pyspark/sql/readwriter.py
--
diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py
index 93832d4..97bd90c 100644
--- a/python/pyspark/sql/readwriter.py
+++ b/python/pyspark/sql/readwriter.py
@@ -23,6 +23,7 @@ if sys.version >= '3':
 from py4j.java_gateway import JavaClass
 
 from pyspark import RDD, since
+from pyspark.rdd import ignore_unicode_prefix
 from pyspark.sql.column import _to_seq
 from pyspark.sql.types import *
 
@@ -193,10 +194,22 @@ class DataFrameReader(object):
 """
 return self._df(self._jreader.parquet(_to_seq(self._sqlContext._sc, 
paths)))
 
+@ignore_unicode_prefix
+@since(1.6)
+def text(self, path):
+"""Loads a text file and returns a [[DataFrame]] with a single string 
column named "text".
+
+Each line in the text file is a new row in the resulting DataFrame.
+
+>>> df = sqlContext.read.text('python/test_support/sql/text-test.txt')
+>>> df.collect()
+[Row(text=u'hello'), Row(text=u'this')]
+"""
+return self._df(self._jreader.text(path))
+
 @since(1.5)
 def orc(self, path):
-"""
-Loads an ORC file, returning the result as a :class:`DataFrame`.
+"""Loads an ORC file, returning the result as a :class:`DataFrame`.
 
 ::Note: Currently ORC support is only available together with
 :class:`HiveContext`.
@@ -432,6 +445,16 @@ class DataFrameWriter(object):
 self.partitionBy(partitionBy)
 self._jwrite.parquet(path)
 
+@since(1.6)
+def text(self, path):
+"""Saves the content of the DataFrame in a text file at the specified 
path.
+
+The DataFrame must have only one column that is of string type.
+Each row becomes a new line in the output file.
+"""
+self._jwrite.text(path)
+
+@since(1.5)
 def orc(self, path, mode=None, partitionBy=None):
 """Saves the content of the :class:`DataFrame` in ORC format at the 
specified path.
 

http://git-wip-us.apache.org/repos/asf/spark/blob/5aa05219/python/test_support/sql/text-test.txt
--
diff --git a/python/test_support/sql/text-test.txt 
b/python/test_support/sql/text-test.txt
new file mode 100644
index 000..ae1e76c
--- /dev/null
+++ b/python/test_support/sql/text-test.txt
@@ -0,0 +1,2 @@
+hello
+this
\ No newline at end of file


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



spark git commit: [SPARK-11363] [SQL] LeftSemiJoin should be LeftSemi in SparkStrategies

2015-10-28 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 5aa052191 -> 20dfd4674


[SPARK-11363] [SQL] LeftSemiJoin should be LeftSemi in SparkStrategies

JIRA: https://issues.apache.org/jira/browse/SPARK-11363

In SparkStrategies some places use LeftSemiJoin. It should be LeftSemi.

cc chenghao-intel liancheng

Author: Liang-Chi Hsieh 

Closes #9318 from viirya/no-left-semi-join.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/20dfd467
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/20dfd467
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/20dfd467

Branch: refs/heads/master
Commit: 20dfd46743401a528b70dfb7862e50ce9a3f8e02
Parents: 5aa0521
Author: Liang-Chi Hsieh 
Authored: Wed Oct 28 15:57:01 2015 -0700
Committer: Yin Huai 
Committed: Wed Oct 28 15:57:01 2015 -0700

--
 .../scala/org/apache/spark/sql/execution/SparkStrategies.scala | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/20dfd467/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 3206726..86d1d39 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -297,11 +297,11 @@ private[sql] abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
   object BroadcastNestedLoop extends Strategy {
 def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
   case logical.Join(
- CanBroadcast(left), right, joinType, condition) if joinType != 
LeftSemiJoin =>
+ CanBroadcast(left), right, joinType, condition) if joinType != 
LeftSemi =>
 execution.joins.BroadcastNestedLoopJoin(
   planLater(left), planLater(right), joins.BuildLeft, joinType, 
condition) :: Nil
   case logical.Join(
- left, CanBroadcast(right), joinType, condition) if joinType != 
LeftSemiJoin =>
+ left, CanBroadcast(right), joinType, condition) if joinType != 
LeftSemi =>
 execution.joins.BroadcastNestedLoopJoin(
   planLater(left), planLater(right), joins.BuildRight, joinType, 
condition) :: Nil
   case _ => Nil
@@ -311,7 +311,7 @@ private[sql] abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
   object CartesianProduct extends Strategy {
 def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
   // TODO CartesianProduct doesn't support the Left Semi Join
-  case logical.Join(left, right, joinType, None) if joinType != 
LeftSemiJoin =>
+  case logical.Join(left, right, joinType, None) if joinType != LeftSemi =>
 execution.joins.CartesianProduct(planLater(left), planLater(right)) :: 
Nil
   case logical.Join(left, right, Inner, Some(condition)) =>
 execution.Filter(condition,


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



spark git commit: [SPARK-11274] [SQL] Text data source support for Spark SQL.

2015-10-23 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 4e38defae -> e1a897b65


[SPARK-11274] [SQL] Text data source support for Spark SQL.

This adds API for reading and writing text files, similar to 
SparkContext.textFile and RDD.saveAsTextFile.
```
SQLContext.read.text("/path/to/something.txt")
DataFrame.write.text("/path/to/write.txt")
```

Using the new Dataset API, this also supports
```
val ds: Dataset[String] = 
SQLContext.read.text("/path/to/something.txt").as[String]
```

Author: Reynold Xin 

Closes #9240 from rxin/SPARK-11274.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e1a897b6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e1a897b6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e1a897b6

Branch: refs/heads/master
Commit: e1a897b657eb62e837026f7b3efafb9a6424ec4f
Parents: 4e38def
Author: Reynold Xin 
Authored: Fri Oct 23 13:04:06 2015 -0700
Committer: Yin Huai 
Committed: Fri Oct 23 13:04:06 2015 -0700

--
 apache.spark.sql.sources.DataSourceRegister |   1 +
 .../org/apache/spark/sql/DataFrameReader.scala  |  16 ++
 .../org/apache/spark/sql/DataFrameWriter.scala  |  18 +++
 .../datasources/json/JSONRelation.scala |   7 +-
 .../datasources/text/DefaultSource.scala| 160 +++
 sql/core/src/test/resources/text-suite.txt  |   4 +
 .../execution/datasources/text/TextSuite.scala  |  81 ++
 7 files changed, 283 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e1a897b6/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
--
diff --git 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index ca5..1ca2044 100644
--- 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -1,3 +1,4 @@
 org.apache.spark.sql.execution.datasources.jdbc.DefaultSource
 org.apache.spark.sql.execution.datasources.json.DefaultSource
 org.apache.spark.sql.execution.datasources.parquet.DefaultSource
+org.apache.spark.sql.execution.datasources.text.DefaultSource

http://git-wip-us.apache.org/repos/asf/spark/blob/e1a897b6/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
index e8651a3..824220d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -302,6 +302,22 @@ class DataFrameReader private[sql](sqlContext: SQLContext) 
extends Logging {
 DataFrame(sqlContext, 
sqlContext.catalog.lookupRelation(TableIdentifier(tableName)))
   }
 
+  /**
+   * Loads a text file and returns a [[DataFrame]] with a single string column 
named "text".
+   * Each line in the text file is a new row in the resulting DataFrame. For 
example:
+   * {{{
+   *   // Scala:
+   *   sqlContext.read.text("/path/to/spark/README.md")
+   *
+   *   // Java:
+   *   sqlContext.read().text("/path/to/spark/README.md")
+   * }}}
+   *
+   * @param path input path
+   * @since 1.6.0
+   */
+  def text(path: String): DataFrame = format("text").load(path)
+
   
///
   // Builder pattern config options
   
///

http://git-wip-us.apache.org/repos/asf/spark/blob/e1a897b6/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index 764510a..7887e55 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -244,6 +244,8 @@ final class DataFrameWriter private[sql](df: DataFrame) {
* @param connectionProperties JDBC database connection arguments, a list of 
arbitrary string
* tag/value. Normally at least a "user" and 
"password" property
* should be included.
+   *
+   * @since 1.4.0
*/
   def jdbc(url: String, 

spark git commit: [SPARK-11194] [SQL] Use MutableURLClassLoader for the classLoader in IsolatedClientLoader.

2015-10-23 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master e1a897b65 -> 4725cb988


[SPARK-11194] [SQL] Use MutableURLClassLoader for the classLoader in 
IsolatedClientLoader.

https://issues.apache.org/jira/browse/SPARK-11194

Author: Yin Huai <yh...@databricks.com>

Closes #9170 from yhuai/SPARK-11194.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4725cb98
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4725cb98
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4725cb98

Branch: refs/heads/master
Commit: 4725cb988b98f367c07214c4c3cfd1206fb2b5c2
Parents: e1a897b
Author: Yin Huai <yh...@databricks.com>
Authored: Fri Oct 23 17:15:13 2015 -0700
Committer: Yin Huai <yh...@databricks.com>
Committed: Fri Oct 23 17:15:13 2015 -0700

--
 .../sql/hive/client/IsolatedClientLoader.scala  | 79 +---
 1 file changed, 51 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4725cb98/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
index 567e4d7..f99c3ed 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
@@ -30,7 +30,7 @@ import org.apache.commons.io.{FileUtils, IOUtils}
 
 import org.apache.spark.Logging
 import org.apache.spark.deploy.SparkSubmitUtils
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{MutableURLClassLoader, Utils}
 
 import org.apache.spark.sql.catalyst.util.quietly
 import org.apache.spark.sql.hive.HiveContext
@@ -148,39 +148,51 @@ private[hive] class IsolatedClientLoader(
   protected def classToPath(name: String): String =
 name.replaceAll("\\.", "/") + ".class"
 
-  /** The classloader that is used to load an isolated version of Hive. */
-  private[hive] var classLoader: ClassLoader = if (isolationOn) {
-new URLClassLoader(allJars, rootClassLoader) {
-  override def loadClass(name: String, resolve: Boolean): Class[_] = {
-val loaded = findLoadedClass(name)
-if (loaded == null) doLoadClass(name, resolve) else loaded
-  }
-  def doLoadClass(name: String, resolve: Boolean): Class[_] = {
-val classFileName = name.replaceAll("\\.", "/") + ".class"
-if (isBarrierClass(name)) {
-  // For barrier classes, we construct a new copy of the class.
-  val bytes = 
IOUtils.toByteArray(baseClassLoader.getResourceAsStream(classFileName))
-  logDebug(s"custom defining: $name - ${util.Arrays.hashCode(bytes)}")
-  defineClass(name, bytes, 0, bytes.length)
-} else if (!isSharedClass(name)) {
-  logDebug(s"hive class: $name - ${getResource(classToPath(name))}")
-  super.loadClass(name, resolve)
-} else {
-  // For shared classes, we delegate to baseClassLoader.
-  logDebug(s"shared class: $name")
-  baseClassLoader.loadClass(name)
+  /**
+   * The classloader that is used to load an isolated version of Hive.
+   * This classloader is a special URLClassLoader that exposes the addURL 
method.
+   * So, when we add jar, we can add this new jar directly through the addURL 
method
+   * instead of stacking a new URLClassLoader on top of it.
+   */
+  private[hive] val classLoader: MutableURLClassLoader = {
+val isolatedClassLoader =
+  if (isolationOn) {
+new URLClassLoader(allJars, rootClassLoader) {
+  override def loadClass(name: String, resolve: Boolean): Class[_] = {
+val loaded = findLoadedClass(name)
+if (loaded == null) doLoadClass(name, resolve) else loaded
+  }
+  def doLoadClass(name: String, resolve: Boolean): Class[_] = {
+val classFileName = name.replaceAll("\\.", "/") + ".class"
+if (isBarrierClass(name)) {
+  // For barrier classes, we construct a new copy of the class.
+  val bytes = 
IOUtils.toByteArray(baseClassLoader.getResourceAsStream(classFileName))
+  logDebug(s"custom defining: $name - 
${util.Arrays.hashCode(bytes)}")
+  defineClass(name, bytes, 0, bytes.length)
+} else if (!isSharedClass(name)) {
+  logDebug(s"hive class: $name - 
${getResource(classToPath(name))}")
+  super.loadClass(name, resolve)
+} else {
+ 

spark git commit: [SPARK-11590][SQL] use native json_tuple in lateral view

2015-11-10 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 6e2e84f3e -> 5ccc1eb08


[SPARK-11590][SQL] use native json_tuple in lateral view

Author: Wenchen Fan 

Closes #9562 from cloud-fan/json-tuple.

(cherry picked from commit 53600854c270d4c953fe95fbae528740b5cf6603)
Signed-off-by: Yin Huai 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5ccc1eb0
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5ccc1eb0
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5ccc1eb0

Branch: refs/heads/branch-1.6
Commit: 5ccc1eb08c14291bb1e94b1cd9fa3bff1172529d
Parents: 6e2e84f
Author: Wenchen Fan 
Authored: Tue Nov 10 11:21:31 2015 -0800
Committer: Yin Huai 
Committed: Tue Nov 10 11:22:11 2015 -0800

--
 .../catalyst/expressions/jsonExpressions.scala  | 23 +--
 .../expressions/JsonExpressionsSuite.scala  | 30 +++
 .../scala/org/apache/spark/sql/DataFrame.scala  |  8 +++--
 .../scala/org/apache/spark/sql/functions.scala  | 12 
 .../apache/spark/sql/JsonFunctionsSuite.scala   | 23 ---
 .../org/apache/spark/sql/hive/HiveQl.scala  |  4 +++
 .../org/apache/spark/sql/hive/HiveQlSuite.scala | 13 
 .../sql/hive/execution/SQLQuerySuite.scala  | 31 
 8 files changed, 104 insertions(+), 40 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5ccc1eb0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
index 8c9853e..8cd7323 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
@@ -314,7 +314,7 @@ case class GetJsonObject(json: Expression, path: Expression)
 }
 
 case class JsonTuple(children: Seq[Expression])
-  extends Expression with CodegenFallback {
+  extends Generator with CodegenFallback {
 
   import SharedFactory._
 
@@ -324,8 +324,8 @@ case class JsonTuple(children: Seq[Expression])
   }
 
   // if processing fails this shared value will be returned
-  @transient private lazy val nullRow: InternalRow =
-new GenericInternalRow(Array.ofDim[Any](fieldExpressions.length))
+  @transient private lazy val nullRow: Seq[InternalRow] =
+new GenericInternalRow(Array.ofDim[Any](fieldExpressions.length)) :: Nil
 
   // the json body is the first child
   @transient private lazy val jsonExpr: Expression = children.head
@@ -344,15 +344,8 @@ case class JsonTuple(children: Seq[Expression])
   // and count the number of foldable fields, we'll use this later to optimize 
evaluation
   @transient private lazy val constantFields: Int = foldableFieldNames.count(_ 
!= null)
 
-  override lazy val dataType: StructType = {
-val fields = fieldExpressions.zipWithIndex.map {
-  case (_, idx) => StructField(
-name = s"c$idx", // mirroring GenericUDTFJSONTuple.initialize
-dataType = StringType,
-nullable = true)
-}
-
-StructType(fields)
+  override def elementTypes: Seq[(DataType, Boolean, String)] = 
fieldExpressions.zipWithIndex.map {
+case (_, idx) => (StringType, true, s"c$idx")
   }
 
   override def prettyName: String = "json_tuple"
@@ -367,7 +360,7 @@ case class JsonTuple(children: Seq[Expression])
 }
   }
 
-  override def eval(input: InternalRow): InternalRow = {
+  override def eval(input: InternalRow): TraversableOnce[InternalRow] = {
 val json = jsonExpr.eval(input).asInstanceOf[UTF8String]
 if (json == null) {
   return nullRow
@@ -383,7 +376,7 @@ case class JsonTuple(children: Seq[Expression])
 }
   }
 
-  private def parseRow(parser: JsonParser, input: InternalRow): InternalRow = {
+  private def parseRow(parser: JsonParser, input: InternalRow): 
Seq[InternalRow] = {
 // only objects are supported
 if (parser.nextToken() != JsonToken.START_OBJECT) {
   return nullRow
@@ -433,7 +426,7 @@ case class JsonTuple(children: Seq[Expression])
   parser.skipChildren()
 }
 
-new GenericInternalRow(row)
+new GenericInternalRow(row) :: Nil
   }
 
   private def copyCurrentStructure(generator: JsonGenerator, parser: 
JsonParser): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/5ccc1eb0/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
--
diff --git 

spark git commit: [SPARK-11590][SQL] use native json_tuple in lateral view

2015-11-10 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master dfcfcbcc0 -> 53600854c


[SPARK-11590][SQL] use native json_tuple in lateral view

Author: Wenchen Fan 

Closes #9562 from cloud-fan/json-tuple.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/53600854
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/53600854
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/53600854

Branch: refs/heads/master
Commit: 53600854c270d4c953fe95fbae528740b5cf6603
Parents: dfcfcbc
Author: Wenchen Fan 
Authored: Tue Nov 10 11:21:31 2015 -0800
Committer: Yin Huai 
Committed: Tue Nov 10 11:21:31 2015 -0800

--
 .../catalyst/expressions/jsonExpressions.scala  | 23 +--
 .../expressions/JsonExpressionsSuite.scala  | 30 +++
 .../scala/org/apache/spark/sql/DataFrame.scala  |  8 +++--
 .../scala/org/apache/spark/sql/functions.scala  | 12 
 .../apache/spark/sql/JsonFunctionsSuite.scala   | 23 ---
 .../org/apache/spark/sql/hive/HiveQl.scala  |  4 +++
 .../org/apache/spark/sql/hive/HiveQlSuite.scala | 13 
 .../sql/hive/execution/SQLQuerySuite.scala  | 31 
 8 files changed, 104 insertions(+), 40 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/53600854/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
index 8c9853e..8cd7323 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
@@ -314,7 +314,7 @@ case class GetJsonObject(json: Expression, path: Expression)
 }
 
 case class JsonTuple(children: Seq[Expression])
-  extends Expression with CodegenFallback {
+  extends Generator with CodegenFallback {
 
   import SharedFactory._
 
@@ -324,8 +324,8 @@ case class JsonTuple(children: Seq[Expression])
   }
 
   // if processing fails this shared value will be returned
-  @transient private lazy val nullRow: InternalRow =
-new GenericInternalRow(Array.ofDim[Any](fieldExpressions.length))
+  @transient private lazy val nullRow: Seq[InternalRow] =
+new GenericInternalRow(Array.ofDim[Any](fieldExpressions.length)) :: Nil
 
   // the json body is the first child
   @transient private lazy val jsonExpr: Expression = children.head
@@ -344,15 +344,8 @@ case class JsonTuple(children: Seq[Expression])
   // and count the number of foldable fields, we'll use this later to optimize 
evaluation
   @transient private lazy val constantFields: Int = foldableFieldNames.count(_ 
!= null)
 
-  override lazy val dataType: StructType = {
-val fields = fieldExpressions.zipWithIndex.map {
-  case (_, idx) => StructField(
-name = s"c$idx", // mirroring GenericUDTFJSONTuple.initialize
-dataType = StringType,
-nullable = true)
-}
-
-StructType(fields)
+  override def elementTypes: Seq[(DataType, Boolean, String)] = 
fieldExpressions.zipWithIndex.map {
+case (_, idx) => (StringType, true, s"c$idx")
   }
 
   override def prettyName: String = "json_tuple"
@@ -367,7 +360,7 @@ case class JsonTuple(children: Seq[Expression])
 }
   }
 
-  override def eval(input: InternalRow): InternalRow = {
+  override def eval(input: InternalRow): TraversableOnce[InternalRow] = {
 val json = jsonExpr.eval(input).asInstanceOf[UTF8String]
 if (json == null) {
   return nullRow
@@ -383,7 +376,7 @@ case class JsonTuple(children: Seq[Expression])
 }
   }
 
-  private def parseRow(parser: JsonParser, input: InternalRow): InternalRow = {
+  private def parseRow(parser: JsonParser, input: InternalRow): 
Seq[InternalRow] = {
 // only objects are supported
 if (parser.nextToken() != JsonToken.START_OBJECT) {
   return nullRow
@@ -433,7 +426,7 @@ case class JsonTuple(children: Seq[Expression])
   parser.skipChildren()
 }
 
-new GenericInternalRow(row)
+new GenericInternalRow(row) :: Nil
   }
 
   private def copyCurrentStructure(generator: JsonGenerator, parser: 
JsonParser): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/53600854/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
 

spark git commit: [SPARK-10371][SQL][FOLLOW-UP] fix code style

2015-11-11 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 7de8abd6f -> 0d637571d


[SPARK-10371][SQL][FOLLOW-UP] fix code style

Author: Wenchen Fan 

Closes #9627 from cloud-fan/follow.

(cherry picked from commit 1510c527b4f5ee0953ae42313ef9e16d2f5864c4)
Signed-off-by: Yin Huai 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0d637571
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0d637571
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0d637571

Branch: refs/heads/branch-1.6
Commit: 0d637571d7705dca3ac63d8bf57224290223c0ff
Parents: 7de8abd
Author: Wenchen Fan 
Authored: Wed Nov 11 09:33:07 2015 -0800
Committer: Yin Huai 
Committed: Wed Nov 11 09:34:08 2015 -0800

--
 .../expressions/EquivalentExpressions.scala | 22 +---
 .../sql/catalyst/expressions/Expression.scala   | 21 +--
 .../expressions/codegen/CodeGenerator.scala | 20 +-
 3 files changed, 30 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0d637571/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
index e7380d2..f83df49 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
@@ -29,29 +29,27 @@ class EquivalentExpressions {
* Wrapper around an Expression that provides semantic equality.
*/
   case class Expr(e: Expression) {
-val hash = e.semanticHash()
 override def equals(o: Any): Boolean = o match {
   case other: Expr => e.semanticEquals(other.e)
   case _ => false
 }
-override def hashCode: Int = hash
+override val hashCode: Int = e.semanticHash()
   }
 
   // For each expression, the set of equivalent expressions.
-  private val equivalenceMap: mutable.HashMap[Expr, 
mutable.MutableList[Expression]] =
-  new mutable.HashMap[Expr, mutable.MutableList[Expression]]
+  private val equivalenceMap = mutable.HashMap.empty[Expr, 
mutable.MutableList[Expression]]
 
   /**
* Adds each expression to this data structure, grouping them with existing 
equivalent
* expressions. Non-recursive.
-   * Returns if there was already a matching expression.
+   * Returns true if there was already a matching expression.
*/
   def addExpr(expr: Expression): Boolean = {
 if (expr.deterministic) {
   val e: Expr = Expr(expr)
   val f = equivalenceMap.get(e)
   if (f.isDefined) {
-f.get.+= (expr)
+f.get += expr
 true
   } else {
 equivalenceMap.put(e, mutable.MutableList(expr))
@@ -63,19 +61,19 @@ class EquivalentExpressions {
   }
 
   /**
-   * Adds the expression to this datastructure recursively. Stops if a 
matching expression
+   * Adds the expression to this data structure recursively. Stops if a 
matching expression
* is found. That is, if `expr` has already been added, its children are not 
added.
* If ignoreLeaf is true, leaf nodes are ignored.
*/
   def addExprTree(root: Expression, ignoreLeaf: Boolean = true): Unit = {
 val skip = root.isInstanceOf[LeafExpression] && ignoreLeaf
-if (!skip && root.deterministic && !addExpr(root)) {
- root.children.foreach(addExprTree(_, ignoreLeaf))
+if (!skip && !addExpr(root)) {
+  root.children.foreach(addExprTree(_, ignoreLeaf))
 }
   }
 
   /**
-   * Returns all fo the expression trees that are equivalent to `e`. Returns
+   * Returns all of the expression trees that are equivalent to `e`. Returns
* an empty collection if there are none.
*/
   def getEquivalentExprs(e: Expression): Seq[Expression] = {
@@ -90,8 +88,8 @@ class EquivalentExpressions {
   }
 
   /**
-   * Returns the state of the datastructure as a string. If all is false, 
skips sets of equivalent
-   * expressions with cardinality 1.
+   * Returns the state of the data structure as a string. If `all` is false, 
skips sets of
+   * equivalent expressions with cardinality 1.
*/
   def debugString(all: Boolean = false): String = {
 val sb: mutable.StringBuilder = new StringBuilder()

http://git-wip-us.apache.org/repos/asf/spark/blob/0d637571/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
--
diff --git 

spark git commit: [SPARK-10371][SQL][FOLLOW-UP] fix code style

2015-11-11 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 1bc41125e -> 1510c527b


[SPARK-10371][SQL][FOLLOW-UP] fix code style

Author: Wenchen Fan 

Closes #9627 from cloud-fan/follow.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1510c527
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1510c527
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1510c527

Branch: refs/heads/master
Commit: 1510c527b4f5ee0953ae42313ef9e16d2f5864c4
Parents: 1bc4112
Author: Wenchen Fan 
Authored: Wed Nov 11 09:33:07 2015 -0800
Committer: Yin Huai 
Committed: Wed Nov 11 09:33:41 2015 -0800

--
 .../expressions/EquivalentExpressions.scala | 22 +---
 .../sql/catalyst/expressions/Expression.scala   | 21 +--
 .../expressions/codegen/CodeGenerator.scala | 20 +-
 3 files changed, 30 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1510c527/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
index e7380d2..f83df49 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
@@ -29,29 +29,27 @@ class EquivalentExpressions {
* Wrapper around an Expression that provides semantic equality.
*/
   case class Expr(e: Expression) {
-val hash = e.semanticHash()
 override def equals(o: Any): Boolean = o match {
   case other: Expr => e.semanticEquals(other.e)
   case _ => false
 }
-override def hashCode: Int = hash
+override val hashCode: Int = e.semanticHash()
   }
 
   // For each expression, the set of equivalent expressions.
-  private val equivalenceMap: mutable.HashMap[Expr, 
mutable.MutableList[Expression]] =
-  new mutable.HashMap[Expr, mutable.MutableList[Expression]]
+  private val equivalenceMap = mutable.HashMap.empty[Expr, 
mutable.MutableList[Expression]]
 
   /**
* Adds each expression to this data structure, grouping them with existing 
equivalent
* expressions. Non-recursive.
-   * Returns if there was already a matching expression.
+   * Returns true if there was already a matching expression.
*/
   def addExpr(expr: Expression): Boolean = {
 if (expr.deterministic) {
   val e: Expr = Expr(expr)
   val f = equivalenceMap.get(e)
   if (f.isDefined) {
-f.get.+= (expr)
+f.get += expr
 true
   } else {
 equivalenceMap.put(e, mutable.MutableList(expr))
@@ -63,19 +61,19 @@ class EquivalentExpressions {
   }
 
   /**
-   * Adds the expression to this datastructure recursively. Stops if a 
matching expression
+   * Adds the expression to this data structure recursively. Stops if a 
matching expression
* is found. That is, if `expr` has already been added, its children are not 
added.
* If ignoreLeaf is true, leaf nodes are ignored.
*/
   def addExprTree(root: Expression, ignoreLeaf: Boolean = true): Unit = {
 val skip = root.isInstanceOf[LeafExpression] && ignoreLeaf
-if (!skip && root.deterministic && !addExpr(root)) {
- root.children.foreach(addExprTree(_, ignoreLeaf))
+if (!skip && !addExpr(root)) {
+  root.children.foreach(addExprTree(_, ignoreLeaf))
 }
   }
 
   /**
-   * Returns all fo the expression trees that are equivalent to `e`. Returns
+   * Returns all of the expression trees that are equivalent to `e`. Returns
* an empty collection if there are none.
*/
   def getEquivalentExprs(e: Expression): Seq[Expression] = {
@@ -90,8 +88,8 @@ class EquivalentExpressions {
   }
 
   /**
-   * Returns the state of the datastructure as a string. If all is false, 
skips sets of equivalent
-   * expressions with cardinality 1.
+   * Returns the state of the data structure as a string. If `all` is false, 
skips sets of
+   * equivalent expressions with cardinality 1.
*/
   def debugString(all: Boolean = false): String = {
 val sb: mutable.StringBuilder = new StringBuilder()

http://git-wip-us.apache.org/repos/asf/spark/blob/1510c527/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
 

spark git commit: [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)

2015-11-10 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 ff7d869c4 -> ce5aba32f


[SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)

This PR is a 2nd follow-up for 
[SPARK-9241](https://issues.apache.org/jira/browse/SPARK-9241). It contains the 
following improvements:
* Fix for a potential bug in distinct child expression and attribute alignment.
* Improved handling of duplicate distinct child expressions.
* Added test for distinct UDAF with multiple children.

cc yhuai

Author: Herman van Hovell <hvanhov...@questtec.nl>

Closes #9566 from hvanhovell/SPARK-9241-followup-2.

(cherry picked from commit 21c562fa03430365f5c2b7d6de1f8f60ab2140d4)
Signed-off-by: Yin Huai <yh...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ce5aba32
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ce5aba32
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ce5aba32

Branch: refs/heads/branch-1.6
Commit: ce5aba32f0376c99a81b170ff10acb209afaa795
Parents: ff7d869
Author: Herman van Hovell <hvanhov...@questtec.nl>
Authored: Tue Nov 10 16:28:21 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Tue Nov 10 16:28:37 2015 -0800

--
 .../analysis/DistinctAggregationRewriter.scala  |  9 +++--
 .../hive/execution/AggregationQuerySuite.scala  | 41 ++--
 2 files changed, 42 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ce5aba32/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
index 397eff0..c0c9604 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
@@ -151,11 +151,12 @@ case class DistinctAggregationRewriter(conf: 
CatalystConf) extends Rule[LogicalP
   }
 
   // Setup unique distinct aggregate children.
-  val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq
-  val distinctAggChildAttrMap = 
distinctAggChildren.map(expressionAttributePair).toMap
-  val distinctAggChildAttrs = distinctAggChildAttrMap.values.toSeq
+  val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
+  val distinctAggChildAttrMap = 
distinctAggChildren.map(expressionAttributePair)
+  val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
 
   // Setup expand & aggregate operators for distinct aggregate expressions.
+  val distinctAggChildAttrLookup = distinctAggChildAttrMap.toMap
   val distinctAggOperatorMap = distinctAggGroups.toSeq.zipWithIndex.map {
 case ((group, expressions), i) =>
   val id = Literal(i + 1)
@@ -170,7 +171,7 @@ case class DistinctAggregationRewriter(conf: CatalystConf) 
extends Rule[LogicalP
   val operators = expressions.map { e =>
 val af = e.aggregateFunction
 val naf = patchAggregateFunctionChildren(af) { x =>
-  evalWithinGroup(id, distinctAggChildAttrMap(x))
+  evalWithinGroup(id, distinctAggChildAttrLookup(x))
 }
 (e, e.copy(aggregateFunction = naf, isDistinct = false))
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/ce5aba32/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
index 6bf2c53..8253921 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
@@ -66,6 +66,36 @@ class ScalaAggregateFunction(schema: StructType) extends 
UserDefinedAggregateFun
   }
 }
 
+class LongProductSum extends UserDefinedAggregateFunction {
+  def inputSchema: StructType = new StructType()
+.add("a", LongType)
+.add("b", LongType)
+
+  def bufferSchema: StructType = new StructType()
+.add("product", LongType)
+
+  def dataType: DataType = LongType
+
+  def deterministic: Boolean = true
+
+  def initialize(buffer: MutableAggregationBuffer): Unit = {
+buffer(0) = 0L
+  }
+
+  def update(buffer: MutableAggregationBuffer, 

spark git commit: [SPARK-11451][SQL] Support single distinct count on multiple columns.

2015-11-08 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 7b3736098 -> 41b2bb1c3


[SPARK-11451][SQL] Support single distinct count on multiple columns.

This PR adds support for multiple column in a single count distinct aggregate 
to the new aggregation path.

cc yhuai

Author: Herman van Hovell <hvanhov...@questtec.nl>

Closes #9409 from hvanhovell/SPARK-11451.

(cherry picked from commit 30c8ba71a76788cbc6916bc1ba6bc8522925fc2b)
Signed-off-by: Yin Huai <yh...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/41b2bb1c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/41b2bb1c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/41b2bb1c

Branch: refs/heads/branch-1.6
Commit: 41b2bb1c3792850d0992c4209bdc825f8c6c585a
Parents: 7b37360
Author: Herman van Hovell <hvanhov...@questtec.nl>
Authored: Sun Nov 8 11:06:10 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Sun Nov 8 11:06:22 2015 -0800

--
 .../catalyst/expressions/aggregate/Utils.scala  | 44 +++-
 .../expressions/conditionalExpressions.scala| 30 -
 .../catalyst/plans/logical/basicOperators.scala |  3 ++
 .../ConditionalExpressionSuite.scala| 14 +++
 .../spark/sql/DataFrameAggregateSuite.scala | 25 +++
 .../hive/execution/AggregationQuerySuite.scala  | 37 +---
 6 files changed, 127 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/41b2bb1c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
index ac23f72..9b22ce2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
@@ -22,26 +22,27 @@ import org.apache.spark.sql.catalyst._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{Expand, Aggregate, 
LogicalPlan}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.types.{IntegerType, StructType, MapType, ArrayType}
+import org.apache.spark.sql.types._
 
 /**
  * Utility functions used by the query planner to convert our plan to new 
aggregation code path.
  */
 object Utils {
-  // Right now, we do not support complex types in the grouping key schema.
-  private def supportsGroupingKeySchema(aggregate: Aggregate): Boolean = {
-val hasComplexTypes = aggregate.groupingExpressions.map(_.dataType).exists 
{
-  case array: ArrayType => true
-  case map: MapType => true
-  case struct: StructType => true
-  case _ => false
-}
 
-!hasComplexTypes
+  // Check if the DataType given cannot be part of a group by clause.
+  private def isUnGroupable(dt: DataType): Boolean = dt match {
+case _: ArrayType | _: MapType => true
+case s: StructType => s.fields.exists(f => isUnGroupable(f.dataType))
+case _ => false
   }
 
+  // Right now, we do not support complex types in the grouping key schema.
+  private def supportsGroupingKeySchema(aggregate: Aggregate): Boolean =
+!aggregate.groupingExpressions.exists(e => isUnGroupable(e.dataType))
+
   private def doConvert(plan: LogicalPlan): Option[Aggregate] = plan match {
 case p: Aggregate if supportsGroupingKeySchema(p) =>
+
   val converted = 
MultipleDistinctRewriter.rewrite(p.transformExpressionsDown {
 case expressions.Average(child) =>
   aggregate.AggregateExpression2(
@@ -55,10 +56,14 @@ object Utils {
 mode = aggregate.Complete,
 isDistinct = false)
 
-// We do not support multiple COUNT DISTINCT columns for now.
-case expressions.CountDistinct(children) if children.length == 1 =>
+case expressions.CountDistinct(children) =>
+  val child = if (children.size > 1) {
+DropAnyNull(CreateStruct(children))
+  } else {
+children.head
+  }
   aggregate.AggregateExpression2(
-aggregateFunction = aggregate.Count(children.head),
+aggregateFunction = aggregate.Count(child),
 mode = aggregate.Complete,
 isDistinct = true)
 
@@ -320,7 +325,7 @@ object MultipleDistinctRewriter extends Rule[LogicalPlan] {
   val gid = new AttributeReference("gid", IntegerType, false)()
   val groupByMap = a.groupingExpressions.collect {
 case ne: NamedEx

spark git commit: [SPARK-11451][SQL] Support single distinct count on multiple columns.

2015-11-08 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 5c4e6d7ec -> 30c8ba71a


[SPARK-11451][SQL] Support single distinct count on multiple columns.

This PR adds support for multiple column in a single count distinct aggregate 
to the new aggregation path.

cc yhuai

Author: Herman van Hovell <hvanhov...@questtec.nl>

Closes #9409 from hvanhovell/SPARK-11451.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/30c8ba71
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/30c8ba71
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/30c8ba71

Branch: refs/heads/master
Commit: 30c8ba71a76788cbc6916bc1ba6bc8522925fc2b
Parents: 5c4e6d7
Author: Herman van Hovell <hvanhov...@questtec.nl>
Authored: Sun Nov 8 11:06:10 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Sun Nov 8 11:06:10 2015 -0800

--
 .../catalyst/expressions/aggregate/Utils.scala  | 44 +++-
 .../expressions/conditionalExpressions.scala| 30 -
 .../catalyst/plans/logical/basicOperators.scala |  3 ++
 .../ConditionalExpressionSuite.scala| 14 +++
 .../spark/sql/DataFrameAggregateSuite.scala | 25 +++
 .../hive/execution/AggregationQuerySuite.scala  | 37 +---
 6 files changed, 127 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/30c8ba71/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
index ac23f72..9b22ce2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
@@ -22,26 +22,27 @@ import org.apache.spark.sql.catalyst._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{Expand, Aggregate, 
LogicalPlan}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.types.{IntegerType, StructType, MapType, ArrayType}
+import org.apache.spark.sql.types._
 
 /**
  * Utility functions used by the query planner to convert our plan to new 
aggregation code path.
  */
 object Utils {
-  // Right now, we do not support complex types in the grouping key schema.
-  private def supportsGroupingKeySchema(aggregate: Aggregate): Boolean = {
-val hasComplexTypes = aggregate.groupingExpressions.map(_.dataType).exists 
{
-  case array: ArrayType => true
-  case map: MapType => true
-  case struct: StructType => true
-  case _ => false
-}
 
-!hasComplexTypes
+  // Check if the DataType given cannot be part of a group by clause.
+  private def isUnGroupable(dt: DataType): Boolean = dt match {
+case _: ArrayType | _: MapType => true
+case s: StructType => s.fields.exists(f => isUnGroupable(f.dataType))
+case _ => false
   }
 
+  // Right now, we do not support complex types in the grouping key schema.
+  private def supportsGroupingKeySchema(aggregate: Aggregate): Boolean =
+!aggregate.groupingExpressions.exists(e => isUnGroupable(e.dataType))
+
   private def doConvert(plan: LogicalPlan): Option[Aggregate] = plan match {
 case p: Aggregate if supportsGroupingKeySchema(p) =>
+
   val converted = 
MultipleDistinctRewriter.rewrite(p.transformExpressionsDown {
 case expressions.Average(child) =>
   aggregate.AggregateExpression2(
@@ -55,10 +56,14 @@ object Utils {
 mode = aggregate.Complete,
 isDistinct = false)
 
-// We do not support multiple COUNT DISTINCT columns for now.
-case expressions.CountDistinct(children) if children.length == 1 =>
+case expressions.CountDistinct(children) =>
+  val child = if (children.size > 1) {
+DropAnyNull(CreateStruct(children))
+  } else {
+children.head
+  }
   aggregate.AggregateExpression2(
-aggregateFunction = aggregate.Count(children.head),
+aggregateFunction = aggregate.Count(child),
 mode = aggregate.Complete,
 isDistinct = true)
 
@@ -320,7 +325,7 @@ object MultipleDistinctRewriter extends Rule[LogicalPlan] {
   val gid = new AttributeReference("gid", IntegerType, false)()
   val groupByMap = a.groupingExpressions.collect {
 case ne: NamedExpression => ne -> ne.toAttribute
-case e => e -> new AttributeReference(e.prettyName, e.dataType, 
e.nullable)()
+

spark git commit: [SPARK-11453][SQL] append data to partitioned table will messes up the result

2015-11-08 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 fddf0c413 -> 7eaf48eeb


[SPARK-11453][SQL] append data to partitioned table will messes up the result

The reason is that:

1. For partitioned hive table, we will move the partitioned columns after data 
columns. (e.g. `` partition by `a` will become ``)
2. When append data to table, we use position to figure out how to match input 
columns to table's columns.

So when we append data to partitioned table, we will match wrong columns 
between input and table. A solution is reordering the input columns before 
match by position, like what we did for 
[`InsertIntoHadoopFsRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala#L101-L105)

Author: Wenchen Fan 

Closes #9408 from cloud-fan/append.

(cherry picked from commit d8b50f70298dbf45e91074ee2d751fee7eecb119)
Signed-off-by: Yin Huai 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7eaf48ee
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7eaf48ee
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7eaf48ee

Branch: refs/heads/branch-1.6
Commit: 7eaf48eeb00442d530061adbd7adefbcd5bb74bf
Parents: fddf0c4
Author: Wenchen Fan 
Authored: Sun Nov 8 21:01:53 2015 -0800
Committer: Yin Huai 
Committed: Sun Nov 8 21:02:07 2015 -0800

--
 .../org/apache/spark/sql/DataFrameWriter.scala  | 29 +---
 .../sql/sources/PartitionedWriteSuite.scala |  8 ++
 .../sql/hive/execution/SQLQuerySuite.scala  | 20 ++
 3 files changed, 53 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7eaf48ee/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index 7887e55..e63a4d5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -23,8 +23,8 @@ import scala.collection.JavaConverters._
 
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier}
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, 
UnresolvedRelation}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, InsertIntoTable}
 import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils
 import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, 
ResolvedDataSource}
 import org.apache.spark.sql.sources.HadoopFsRelation
@@ -167,17 +167,38 @@ final class DataFrameWriter private[sql](df: DataFrame) {
   }
 
   private def insertInto(tableIdent: TableIdentifier): Unit = {
-val partitions = partitioningColumns.map(_.map(col => col -> (None: 
Option[String])).toMap)
+val partitions = normalizedParCols.map(_.map(col => col -> (None: 
Option[String])).toMap)
 val overwrite = mode == SaveMode.Overwrite
+
+// A partitioned relation's schema can be different from the input 
logicalPlan, since
+// partition columns are all moved after data columns. We Project to 
adjust the ordering.
+// TODO: this belongs to the analyzer.
+val input = normalizedParCols.map { parCols =>
+  val (inputPartCols, inputDataCols) = df.logicalPlan.output.partition { 
attr =>
+parCols.contains(attr.name)
+  }
+  Project(inputDataCols ++ inputPartCols, df.logicalPlan)
+}.getOrElse(df.logicalPlan)
+
 df.sqlContext.executePlan(
   InsertIntoTable(
 UnresolvedRelation(tableIdent),
 partitions.getOrElse(Map.empty[String, Option[String]]),
-df.logicalPlan,
+input,
 overwrite,
 ifNotExists = false)).toRdd
   }
 
+  private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map 
{ parCols =>
+parCols.map { col =>
+  df.logicalPlan.output
+.map(_.name)
+.find(df.sqlContext.analyzer.resolver(_, col))
+.getOrElse(throw new AnalysisException(s"Partition column $col not 
found in existing " +
+  s"columns (${df.logicalPlan.output.map(_.name).mkString(", ")})"))
+}
+  }
+
   /**
* Saves the content of the [[DataFrame]] as the specified table.
*

http://git-wip-us.apache.org/repos/asf/spark/blob/7eaf48ee/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala

spark git commit: [SPARK-11453][SQL] append data to partitioned table will messes up the result

2015-11-08 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 97b7080cf -> d8b50f702


[SPARK-11453][SQL] append data to partitioned table will messes up the result

The reason is that:

1. For partitioned hive table, we will move the partitioned columns after data 
columns. (e.g. `` partition by `a` will become ``)
2. When append data to table, we use position to figure out how to match input 
columns to table's columns.

So when we append data to partitioned table, we will match wrong columns 
between input and table. A solution is reordering the input columns before 
match by position, like what we did for 
[`InsertIntoHadoopFsRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala#L101-L105)

Author: Wenchen Fan 

Closes #9408 from cloud-fan/append.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d8b50f70
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d8b50f70
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d8b50f70

Branch: refs/heads/master
Commit: d8b50f70298dbf45e91074ee2d751fee7eecb119
Parents: 97b7080
Author: Wenchen Fan 
Authored: Sun Nov 8 21:01:53 2015 -0800
Committer: Yin Huai 
Committed: Sun Nov 8 21:01:53 2015 -0800

--
 .../org/apache/spark/sql/DataFrameWriter.scala  | 29 +---
 .../sql/sources/PartitionedWriteSuite.scala |  8 ++
 .../sql/hive/execution/SQLQuerySuite.scala  | 20 ++
 3 files changed, 53 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d8b50f70/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index 7887e55..e63a4d5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -23,8 +23,8 @@ import scala.collection.JavaConverters._
 
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier}
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, 
UnresolvedRelation}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, InsertIntoTable}
 import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils
 import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, 
ResolvedDataSource}
 import org.apache.spark.sql.sources.HadoopFsRelation
@@ -167,17 +167,38 @@ final class DataFrameWriter private[sql](df: DataFrame) {
   }
 
   private def insertInto(tableIdent: TableIdentifier): Unit = {
-val partitions = partitioningColumns.map(_.map(col => col -> (None: 
Option[String])).toMap)
+val partitions = normalizedParCols.map(_.map(col => col -> (None: 
Option[String])).toMap)
 val overwrite = mode == SaveMode.Overwrite
+
+// A partitioned relation's schema can be different from the input 
logicalPlan, since
+// partition columns are all moved after data columns. We Project to 
adjust the ordering.
+// TODO: this belongs to the analyzer.
+val input = normalizedParCols.map { parCols =>
+  val (inputPartCols, inputDataCols) = df.logicalPlan.output.partition { 
attr =>
+parCols.contains(attr.name)
+  }
+  Project(inputDataCols ++ inputPartCols, df.logicalPlan)
+}.getOrElse(df.logicalPlan)
+
 df.sqlContext.executePlan(
   InsertIntoTable(
 UnresolvedRelation(tableIdent),
 partitions.getOrElse(Map.empty[String, Option[String]]),
-df.logicalPlan,
+input,
 overwrite,
 ifNotExists = false)).toRdd
   }
 
+  private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map 
{ parCols =>
+parCols.map { col =>
+  df.logicalPlan.output
+.map(_.name)
+.find(df.sqlContext.analyzer.resolver(_, col))
+.getOrElse(throw new AnalysisException(s"Partition column $col not 
found in existing " +
+  s"columns (${df.logicalPlan.output.map(_.name).mkString(", ")})"))
+}
+  }
+
   /**
* Saves the content of the [[DataFrame]] as the specified table.
*

http://git-wip-us.apache.org/repos/asf/spark/blob/d8b50f70/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala
--
diff --git 

spark git commit: [SPARK-11690][PYSPARK] Add pivot to python api

2015-11-13 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 99693fef0 -> a24477996


[SPARK-11690][PYSPARK] Add pivot to python api

This PR adds pivot to the python api of GroupedData with the same syntax as 
Scala/Java.

Author: Andrew Ray 

Closes #9653 from aray/sql-pivot-python.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a2447799
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a2447799
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a2447799

Branch: refs/heads/master
Commit: a24477996e936b0861819ffb420f763f80f0b1da
Parents: 99693fe
Author: Andrew Ray 
Authored: Fri Nov 13 10:31:17 2015 -0800
Committer: Yin Huai 
Committed: Fri Nov 13 10:31:17 2015 -0800

--
 python/pyspark/sql/group.py | 24 +++-
 1 file changed, 23 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a2447799/python/pyspark/sql/group.py
--
diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py
index 71c0bcc..227f40b 100644
--- a/python/pyspark/sql/group.py
+++ b/python/pyspark/sql/group.py
@@ -17,7 +17,7 @@
 
 from pyspark import since
 from pyspark.rdd import ignore_unicode_prefix
-from pyspark.sql.column import Column, _to_seq
+from pyspark.sql.column import Column, _to_seq, _to_java_column, 
_create_column_from_literal
 from pyspark.sql.dataframe import DataFrame
 from pyspark.sql.types import *
 
@@ -167,6 +167,23 @@ class GroupedData(object):
 [Row(sum(age)=7, sum(height)=165)]
 """
 
+@since(1.6)
+def pivot(self, pivot_col, *values):
+"""Pivots a column of the current DataFrame and preform the specified 
aggregation.
+
+:param pivot_col: Column to pivot
+:param values: Optional list of values of pivotColumn that will be 
translated to columns in
+the output data frame. If values are not provided the method with 
do an immediate call
+to .distinct() on the pivot column.
+>>> df4.groupBy("year").pivot("course", "dotNET", 
"Java").sum("earnings").collect()
+[Row(year=2012, dotNET=15000, Java=2), Row(year=2013, 
dotNET=48000, Java=3)]
+>>> df4.groupBy("year").pivot("course").sum("earnings").collect()
+[Row(year=2012, Java=2, dotNET=15000), Row(year=2013, Java=3, 
dotNET=48000)]
+"""
+jgd = self._jdf.pivot(_to_java_column(pivot_col),
+  _to_seq(self.sql_ctx._sc, values, 
_create_column_from_literal))
+return GroupedData(jgd, self.sql_ctx)
+
 
 def _test():
 import doctest
@@ -182,6 +199,11 @@ def _test():
   StructField('name', StringType())]))
 globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80),
Row(name='Bob', age=5, height=85)]).toDF()
+globs['df4'] = sc.parallelize([Row(course="dotNET", year=2012, 
earnings=1),
+   Row(course="Java",   year=2012, 
earnings=2),
+   Row(course="dotNET", year=2012, 
earnings=5000),
+   Row(course="dotNET", year=2013, 
earnings=48000),
+   Row(course="Java",   year=2013, 
earnings=3)]).toDF()
 
 (failure_count, test_count) = doctest.testmod(
 pyspark.sql.group, globs=globs,


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



spark git commit: [SPARK-11690][PYSPARK] Add pivot to python api

2015-11-13 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 4a1bcb26d -> 6459a6747


[SPARK-11690][PYSPARK] Add pivot to python api

This PR adds pivot to the python api of GroupedData with the same syntax as 
Scala/Java.

Author: Andrew Ray 

Closes #9653 from aray/sql-pivot-python.

(cherry picked from commit a24477996e936b0861819ffb420f763f80f0b1da)
Signed-off-by: Yin Huai 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6459a674
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6459a674
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6459a674

Branch: refs/heads/branch-1.6
Commit: 6459a6747bc6ead87c21f649347292ec79a3f40d
Parents: 4a1bcb2
Author: Andrew Ray 
Authored: Fri Nov 13 10:31:17 2015 -0800
Committer: Yin Huai 
Committed: Fri Nov 13 10:31:29 2015 -0800

--
 python/pyspark/sql/group.py | 24 +++-
 1 file changed, 23 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6459a674/python/pyspark/sql/group.py
--
diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py
index 71c0bcc..227f40b 100644
--- a/python/pyspark/sql/group.py
+++ b/python/pyspark/sql/group.py
@@ -17,7 +17,7 @@
 
 from pyspark import since
 from pyspark.rdd import ignore_unicode_prefix
-from pyspark.sql.column import Column, _to_seq
+from pyspark.sql.column import Column, _to_seq, _to_java_column, 
_create_column_from_literal
 from pyspark.sql.dataframe import DataFrame
 from pyspark.sql.types import *
 
@@ -167,6 +167,23 @@ class GroupedData(object):
 [Row(sum(age)=7, sum(height)=165)]
 """
 
+@since(1.6)
+def pivot(self, pivot_col, *values):
+"""Pivots a column of the current DataFrame and preform the specified 
aggregation.
+
+:param pivot_col: Column to pivot
+:param values: Optional list of values of pivotColumn that will be 
translated to columns in
+the output data frame. If values are not provided the method with 
do an immediate call
+to .distinct() on the pivot column.
+>>> df4.groupBy("year").pivot("course", "dotNET", 
"Java").sum("earnings").collect()
+[Row(year=2012, dotNET=15000, Java=2), Row(year=2013, 
dotNET=48000, Java=3)]
+>>> df4.groupBy("year").pivot("course").sum("earnings").collect()
+[Row(year=2012, Java=2, dotNET=15000), Row(year=2013, Java=3, 
dotNET=48000)]
+"""
+jgd = self._jdf.pivot(_to_java_column(pivot_col),
+  _to_seq(self.sql_ctx._sc, values, 
_create_column_from_literal))
+return GroupedData(jgd, self.sql_ctx)
+
 
 def _test():
 import doctest
@@ -182,6 +199,11 @@ def _test():
   StructField('name', StringType())]))
 globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80),
Row(name='Bob', age=5, height=85)]).toDF()
+globs['df4'] = sc.parallelize([Row(course="dotNET", year=2012, 
earnings=1),
+   Row(course="Java",   year=2012, 
earnings=2),
+   Row(course="dotNET", year=2012, 
earnings=5000),
+   Row(course="dotNET", year=2013, 
earnings=48000),
+   Row(course="Java",   year=2013, 
earnings=3)]).toDF()
 
 (failure_count, test_count) = doctest.testmod(
 pyspark.sql.group, globs=globs,


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



spark git commit: [SPARK-11522][SQL] input_file_name() returns "" for external tables

2015-11-16 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 a0f9cd77a -> c37ed52ec


[SPARK-11522][SQL] input_file_name() returns "" for external tables

When computing partition for non-parquet relation, `HadoopRDD.compute` is used. 
but it does not set the thread local variable `inputFileName` in 
`NewSqlHadoopRDD`, like `NewSqlHadoopRDD.compute` does.. Yet, when getting the 
`inputFileName`, `NewSqlHadoopRDD.inputFileName` is exptected, which is empty 
now.
Adding the setting inputFileName in HadoopRDD.compute resolves this issue.

Author: xin Wu 

Closes #9542 from xwu0226/SPARK-11522.

(cherry picked from commit 0e79604aed116bdcb40e03553a2d103b5b1cdbae)
Signed-off-by: Yin Huai 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c37ed52e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c37ed52e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c37ed52e

Branch: refs/heads/branch-1.6
Commit: c37ed52ecd38a400f99cc82f07440e455b772db2
Parents: a0f9cd7
Author: xin Wu 
Authored: Mon Nov 16 08:10:48 2015 -0800
Committer: Yin Huai 
Committed: Mon Nov 16 08:10:59 2015 -0800

--
 .../scala/org/apache/spark/rdd/HadoopRDD.scala  |  7 ++
 .../spark/sql/hive/execution/HiveUDFSuite.scala | 93 +++-
 2 files changed, 98 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c37ed52e/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala 
b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 0453614..7db5834 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -213,6 +213,12 @@ class HadoopRDD[K, V](
 
   val inputMetrics = 
context.taskMetrics.getInputMetricsForReadMethod(DataReadMethod.Hadoop)
 
+  // Sets the thread local variable for the file's name
+  split.inputSplit.value match {
+case fs: FileSplit => 
SqlNewHadoopRDD.setInputFileName(fs.getPath.toString)
+case _ => SqlNewHadoopRDD.unsetInputFileName()
+  }
+
   // Find a function that will return the FileSystem bytes read by this 
thread. Do this before
   // creating RecordReader, because RecordReader's constructor might read 
some bytes
   val bytesReadCallback = inputMetrics.bytesReadCallback.orElse {
@@ -250,6 +256,7 @@ class HadoopRDD[K, V](
 
   override def close() {
 if (reader != null) {
+  SqlNewHadoopRDD.unsetInputFileName()
   // Close the reader and release it. Note: it's very important that 
we don't close the
   // reader more than once, since that exposes us to MAPREDUCE-5918 
when running against
   // Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to 
non-deterministic

http://git-wip-us.apache.org/repos/asf/spark/blob/c37ed52e/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
index 5ab477e..9deb1a6 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.hive.execution
 
-import java.io.{DataInput, DataOutput}
+import java.io.{PrintWriter, File, DataInput, DataOutput}
 import java.util.{ArrayList, Arrays, Properties}
 
 import org.apache.hadoop.conf.Configuration
@@ -28,6 +28,7 @@ import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, 
ObjectInspectorFactory}
 import org.apache.hadoop.hive.serde2.{AbstractSerDe, SerDeStats}
 import org.apache.hadoop.io.Writable
+import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
 import org.apache.spark.sql.hive.test.TestHiveSingleton
 import org.apache.spark.util.Utils
@@ -44,7 +45,7 @@ case class ListStringCaseClass(l: Seq[String])
 /**
  * A test suite for Hive custom UDFs.
  */
-class HiveUDFSuite extends QueryTest with TestHiveSingleton {
+class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils {
 
   import hiveContext.{udf, sql}
   import hiveContext.implicits._
@@ -348,6 +349,94 @@ class HiveUDFSuite extends QueryTest with 
TestHiveSingleton {
 
 sqlContext.dropTempTable("testUDF")
   }
+
+  test("SPARK-11522 

spark git commit: [SPARK-10181][SQL] Do kerberos login for credentials during hive client initialization

2015-11-16 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 bf79a171e -> 51fc152b7


[SPARK-10181][SQL] Do kerberos login for credentials during hive client 
initialization

On driver process start up, UserGroupInformation.loginUserFromKeytab is called 
with the principal and keytab passed in, and therefore static var 
UserGroupInfomation,loginUser is set to that principal with kerberos 
credentials saved in its private credential set, and all threads within the 
driver process are supposed to see and use this login credentials to 
authenticate with Hive and Hadoop. However, because of IsolatedClientLoader, 
UserGroupInformation class is not shared for hive metastore clients, and 
instead it is loaded separately and of course not able to see the prepared 
kerberos login credentials in the main thread.

The first proposed fix would cause other classloader conflict errors, and is 
not an appropriate solution. This new change does kerberos login during hive 
client initialization, which will make credentials ready for the particular 
hive client instance.

 yhuai Please take a look and let me know. If you are not the right person to 
talk to, could you point me to someone responsible for this?

Author: Yu Gao <y...@us.ibm.com>
Author: gaoyu <ga...@gaoyu-macbookpro.roam.corp.google.com>
Author: Yu Gao <crystalga...@gmail.com>

Closes #9272 from yolandagao/master.

(cherry picked from commit 72c1d68b4ab6acb3f85971e10947caabb4bd846d)
Signed-off-by: Yin Huai <yh...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/51fc152b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/51fc152b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/51fc152b

Branch: refs/heads/branch-1.5
Commit: 51fc152b7e194282940eab29fe0069edef8a67a5
Parents: bf79a17
Author: Yu Gao <y...@us.ibm.com>
Authored: Sun Nov 15 14:53:59 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Mon Nov 16 10:29:39 2015 -0800

--
 .../org/apache/spark/deploy/SparkSubmit.scala   | 17 +++---
 .../spark/sql/hive/client/ClientWrapper.scala   | 24 +++-
 2 files changed, 37 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/51fc152b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
--
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala 
b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index fefbba9..dc555cb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -39,7 +39,7 @@ import org.apache.ivy.plugins.matcher.GlobPatternMatcher
 import org.apache.ivy.plugins.repository.file.FileRepository
 import org.apache.ivy.plugins.resolver.{FileSystemResolver, ChainResolver, 
IBiblioResolver}
 
-import org.apache.spark.{SparkUserAppException, SPARK_VERSION}
+import org.apache.spark.{SparkException, SparkUserAppException, SPARK_VERSION}
 import org.apache.spark.api.r.RUtils
 import org.apache.spark.deploy.rest._
 import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, 
Utils}
@@ -521,8 +521,19 @@ object SparkSubmit {
 sysProps.put("spark.yarn.isPython", "true")
   }
   if (args.principal != null) {
-require(args.keytab != null, "Keytab must be specified when the keytab 
is specified")
-UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
+require(args.keytab != null, "Keytab must be specified when principal 
is specified")
+if (!new File(args.keytab).exists()) {
+  throw new SparkException(s"Keytab file: ${args.keytab} does not 
exist")
+} else {
+  // Add keytab and principal configurations in sysProps to make them 
available
+  // for later use; e.g. in spark sql, the isolated class loader used 
to talk
+  // to HiveMetastore will use these settings. They will be set as 
Java system
+  // properties and then loaded by SparkConf
+  sysProps.put("spark.yarn.keytab", args.keytab)
+  sysProps.put("spark.yarn.principal", args.principal)
+
+  UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
+}
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/51fc152b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/Clie

spark git commit: [SPARK-11191][SPARK-11311][SQL] Backports #9664 and #9277 to branch-1.5

2015-11-15 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 330961bbf -> b767ceeb2


[SPARK-11191][SPARK-11311][SQL] Backports #9664 and #9277 to branch-1.5

The main purpose of this PR is to backport #9664, which depends on #9277.

Author: Cheng Lian 

Closes #9671 from liancheng/spark-11191.fix-temp-function.branch-1_5.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b767ceeb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b767ceeb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b767ceeb

Branch: refs/heads/branch-1.5
Commit: b767ceeb25e9b4948c93475c6382b16f26dbfc6e
Parents: 330961b
Author: Cheng Lian 
Authored: Sun Nov 15 13:16:49 2015 -0800
Committer: Yin Huai 
Committed: Sun Nov 15 13:16:49 2015 -0800

--
 .../thriftserver/HiveThriftServer2Suites.scala  | 53 ++--
 .../org/apache/spark/sql/hive/HiveContext.scala | 18 ---
 .../spark/sql/hive/client/ClientWrapper.scala   |  2 +-
 .../org/apache/spark/sql/hive/hiveUDFs.scala| 20 ++--
 4 files changed, 77 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b767ceeb/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
--
diff --git 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index 19b2f24..597bf60 100644
--- 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -256,9 +256,9 @@ class HiveThriftBinaryServerSuite extends 
HiveThriftJdbcTest {
   { statement =>
 
 val queries = Seq(
-s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}=291",
-"SET hive.cli.print.header=true"
-)
+  s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}=291",
+  "SET hive.cli.print.header=true"
+)
 
 queries.map(statement.execute)
 val rs1 = statement.executeQuery(s"SET 
${SQLConf.SHUFFLE_PARTITIONS.key}")
@@ -458,6 +458,53 @@ class HiveThriftBinaryServerSuite extends 
HiveThriftJdbcTest {
   assert(conf.get("spark.sql.hive.version") === Some("1.2.1"))
 }
   }
+
+  test("SPARK-11595 ADD JAR with input path having URL scheme") {
+withJdbcStatement { statement =>
+  statement.executeQuery("SET spark.sql.hive.thriftServer.async=true")
+
+  val jarPath = "../hive/src/test/resources/TestUDTF.jar"
+  val jarURL = s"file://${System.getProperty("user.dir")}/$jarPath"
+
+  Seq(
+s"ADD JAR $jarURL",
+s"""CREATE TEMPORARY FUNCTION udtf_count2
+|AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
+ """.stripMargin
+  ).foreach(statement.execute)
+
+  val rs1 = statement.executeQuery("DESCRIBE FUNCTION udtf_count2")
+
+  assert(rs1.next())
+  assert(rs1.getString(1) === "Function: udtf_count2")
+
+  assert(rs1.next())
+  assertResult("Class: 
org.apache.spark.sql.hive.execution.GenericUDTFCount2") {
+rs1.getString(1)
+  }
+
+  assert(rs1.next())
+  assert(rs1.getString(1) === "Usage: To be added.")
+
+  val dataPath = "../hive/src/test/resources/data/files/kv1.txt"
+
+  Seq(
+s"CREATE TABLE test_udtf(key INT, value STRING)",
+s"LOAD DATA LOCAL INPATH '$dataPath' OVERWRITE INTO TABLE test_udtf"
+  ).foreach(statement.execute)
+
+  val rs2 = statement.executeQuery(
+"SELECT key, cc FROM test_udtf LATERAL VIEW udtf_count2(value) dd AS 
cc")
+
+  assert(rs2.next())
+  assert(rs2.getInt(1) === 97)
+  assert(rs2.getInt(2) === 500)
+
+  assert(rs2.next())
+  assert(rs2.getInt(1) === 97)
+  assert(rs2.getInt(2) === 500)
+}
+  }
 }
 
 class HiveThriftHttpServerSuite extends HiveThriftJdbcTest {

http://git-wip-us.apache.org/repos/asf/spark/blob/b767ceeb/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 5dc3d81..b3ba444 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -57,9 +57,11 @@ import org.apache.spark.util.Utils
 /**
  * This is the HiveQL Dialect, this dialect is strongly bind with HiveContext
  */

spark git commit: [SPARK-11672][ML] set active SQLContext in JavaDefaultReadWriteSuite

2015-11-15 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master d22fc1088 -> 64e555110


[SPARK-11672][ML] set active SQLContext in JavaDefaultReadWriteSuite

The same as #9694, but for Java test suite. yhuai

Author: Xiangrui Meng <m...@databricks.com>

Closes #9719 from mengxr/SPARK-11672.4.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/64e55511
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/64e55511
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/64e55511

Branch: refs/heads/master
Commit: 64e55511033afb6ef42be142eb371bfbc31f5230
Parents: d22fc10
Author: Xiangrui Meng <m...@databricks.com>
Authored: Sun Nov 15 13:23:05 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Sun Nov 15 13:23:05 2015 -0800

--
 .../org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java   | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/64e55511/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java
--
diff --git 
a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java 
b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java
index c395380..01ff1ea 100644
--- 
a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java
+++ 
b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java
@@ -32,17 +32,23 @@ import org.apache.spark.util.Utils;
 public class JavaDefaultReadWriteSuite {
 
   JavaSparkContext jsc = null;
+  SQLContext sqlContext = null;
   File tempDir = null;
 
   @Before
   public void setUp() {
 jsc = new JavaSparkContext("local[2]", "JavaDefaultReadWriteSuite");
+SQLContext.clearActive();
+sqlContext = new SQLContext(jsc);
+SQLContext.setActive(sqlContext);
 tempDir = Utils.createTempDir(
   System.getProperty("java.io.tmpdir"), "JavaDefaultReadWriteSuite");
   }
 
   @After
   public void tearDown() {
+sqlContext = null;
+SQLContext.clearActive();
 if (jsc != null) {
   jsc.stop();
   jsc = null;
@@ -64,7 +70,6 @@ public class JavaDefaultReadWriteSuite {
 } catch (IOException e) {
   // expected
 }
-SQLContext sqlContext = new SQLContext(jsc);
 instance.write().context(sqlContext).overwrite().save(outputPath);
 MyParams newInstance = MyParams.load(outputPath);
 Assert.assertEquals("UID should match.", instance.uid(), 
newInstance.uid());


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



spark git commit: [SPARK-11672][ML] set active SQLContext in JavaDefaultReadWriteSuite

2015-11-15 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 6f98d47f8 -> 07af78221


[SPARK-11672][ML] set active SQLContext in JavaDefaultReadWriteSuite

The same as #9694, but for Java test suite. yhuai

Author: Xiangrui Meng <m...@databricks.com>

Closes #9719 from mengxr/SPARK-11672.4.

(cherry picked from commit 64e55511033afb6ef42be142eb371bfbc31f5230)
Signed-off-by: Yin Huai <yh...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/07af7822
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/07af7822
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/07af7822

Branch: refs/heads/branch-1.6
Commit: 07af7822138a5ee5a39e3b82f14b409c408388eb
Parents: 6f98d47
Author: Xiangrui Meng <m...@databricks.com>
Authored: Sun Nov 15 13:23:05 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Sun Nov 15 13:23:14 2015 -0800

--
 .../org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java   | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/07af7822/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java
--
diff --git 
a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java 
b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java
index c395380..01ff1ea 100644
--- 
a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java
+++ 
b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java
@@ -32,17 +32,23 @@ import org.apache.spark.util.Utils;
 public class JavaDefaultReadWriteSuite {
 
   JavaSparkContext jsc = null;
+  SQLContext sqlContext = null;
   File tempDir = null;
 
   @Before
   public void setUp() {
 jsc = new JavaSparkContext("local[2]", "JavaDefaultReadWriteSuite");
+SQLContext.clearActive();
+sqlContext = new SQLContext(jsc);
+SQLContext.setActive(sqlContext);
 tempDir = Utils.createTempDir(
   System.getProperty("java.io.tmpdir"), "JavaDefaultReadWriteSuite");
   }
 
   @After
   public void tearDown() {
+sqlContext = null;
+SQLContext.clearActive();
 if (jsc != null) {
   jsc.stop();
   jsc = null;
@@ -64,7 +70,6 @@ public class JavaDefaultReadWriteSuite {
 } catch (IOException e) {
   // expected
 }
-SQLContext sqlContext = new SQLContext(jsc);
 instance.write().context(sqlContext).overwrite().save(outputPath);
 MyParams newInstance = MyParams.load(outputPath);
 Assert.assertEquals("UID should match.", instance.uid(), 
newInstance.uid());


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



spark git commit: [SPARK-10181][SQL] Do kerberos login for credentials during hive client initialization

2015-11-15 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 84f457ad0 -> 2f0f8bb66


[SPARK-10181][SQL] Do kerberos login for credentials during hive client 
initialization

On driver process start up, UserGroupInformation.loginUserFromKeytab is called 
with the principal and keytab passed in, and therefore static var 
UserGroupInfomation,loginUser is set to that principal with kerberos 
credentials saved in its private credential set, and all threads within the 
driver process are supposed to see and use this login credentials to 
authenticate with Hive and Hadoop. However, because of IsolatedClientLoader, 
UserGroupInformation class is not shared for hive metastore clients, and 
instead it is loaded separately and of course not able to see the prepared 
kerberos login credentials in the main thread.

The first proposed fix would cause other classloader conflict errors, and is 
not an appropriate solution. This new change does kerberos login during hive 
client initialization, which will make credentials ready for the particular 
hive client instance.

 yhuai Please take a look and let me know. If you are not the right person to 
talk to, could you point me to someone responsible for this?

Author: Yu Gao <y...@us.ibm.com>
Author: gaoyu <ga...@gaoyu-macbookpro.roam.corp.google.com>
Author: Yu Gao <crystalga...@gmail.com>

Closes #9272 from yolandagao/master.

(cherry picked from commit 72c1d68b4ab6acb3f85971e10947caabb4bd846d)
Signed-off-by: Yin Huai <yh...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2f0f8bb6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2f0f8bb6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2f0f8bb6

Branch: refs/heads/branch-1.6
Commit: 2f0f8bb66f0d968e3008305983f91adb76c78508
Parents: 84f457a
Author: Yu Gao <y...@us.ibm.com>
Authored: Sun Nov 15 14:53:59 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Sun Nov 15 14:54:08 2015 -0800

--
 .../org/apache/spark/deploy/SparkSubmit.scala   | 17 +++---
 .../spark/sql/hive/client/ClientWrapper.scala   | 24 +++-
 2 files changed, 37 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2f0f8bb6/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
--
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala 
b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 84ae122..09d2ec9 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -39,7 +39,7 @@ import org.apache.ivy.plugins.matcher.GlobPatternMatcher
 import org.apache.ivy.plugins.repository.file.FileRepository
 import org.apache.ivy.plugins.resolver.{FileSystemResolver, ChainResolver, 
IBiblioResolver}
 
-import org.apache.spark.{SparkUserAppException, SPARK_VERSION}
+import org.apache.spark.{SparkException, SparkUserAppException, SPARK_VERSION}
 import org.apache.spark.api.r.RUtils
 import org.apache.spark.deploy.rest._
 import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, 
Utils}
@@ -521,8 +521,19 @@ object SparkSubmit {
 sysProps.put("spark.yarn.isPython", "true")
   }
   if (args.principal != null) {
-require(args.keytab != null, "Keytab must be specified when the keytab 
is specified")
-UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
+require(args.keytab != null, "Keytab must be specified when principal 
is specified")
+if (!new File(args.keytab).exists()) {
+  throw new SparkException(s"Keytab file: ${args.keytab} does not 
exist")
+} else {
+  // Add keytab and principal configurations in sysProps to make them 
available
+  // for later use; e.g. in spark sql, the isolated class loader used 
to talk
+  // to HiveMetastore will use these settings. They will be set as 
Java system
+  // properties and then loaded by SparkConf
+  sysProps.put("spark.yarn.keytab", args.keytab)
+  sysProps.put("spark.yarn.principal", args.principal)
+
+  UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
+}
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2f0f8bb6/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/Clie

spark git commit: [SPARK-10181][SQL] Do kerberos login for credentials during hive client initialization

2015-11-15 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 3e2e1873b -> 72c1d68b4


[SPARK-10181][SQL] Do kerberos login for credentials during hive client 
initialization

On driver process start up, UserGroupInformation.loginUserFromKeytab is called 
with the principal and keytab passed in, and therefore static var 
UserGroupInfomation,loginUser is set to that principal with kerberos 
credentials saved in its private credential set, and all threads within the 
driver process are supposed to see and use this login credentials to 
authenticate with Hive and Hadoop. However, because of IsolatedClientLoader, 
UserGroupInformation class is not shared for hive metastore clients, and 
instead it is loaded separately and of course not able to see the prepared 
kerberos login credentials in the main thread.

The first proposed fix would cause other classloader conflict errors, and is 
not an appropriate solution. This new change does kerberos login during hive 
client initialization, which will make credentials ready for the particular 
hive client instance.

 yhuai Please take a look and let me know. If you are not the right person to 
talk to, could you point me to someone responsible for this?

Author: Yu Gao <y...@us.ibm.com>
Author: gaoyu <ga...@gaoyu-macbookpro.roam.corp.google.com>
Author: Yu Gao <crystalga...@gmail.com>

Closes #9272 from yolandagao/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/72c1d68b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/72c1d68b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/72c1d68b

Branch: refs/heads/master
Commit: 72c1d68b4ab6acb3f85971e10947caabb4bd846d
Parents: 3e2e187
Author: Yu Gao <y...@us.ibm.com>
Authored: Sun Nov 15 14:53:59 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Sun Nov 15 14:53:59 2015 -0800

--
 .../org/apache/spark/deploy/SparkSubmit.scala   | 17 +++---
 .../spark/sql/hive/client/ClientWrapper.scala   | 24 +++-
 2 files changed, 37 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/72c1d68b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
--
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala 
b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 84ae122..09d2ec9 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -39,7 +39,7 @@ import org.apache.ivy.plugins.matcher.GlobPatternMatcher
 import org.apache.ivy.plugins.repository.file.FileRepository
 import org.apache.ivy.plugins.resolver.{FileSystemResolver, ChainResolver, 
IBiblioResolver}
 
-import org.apache.spark.{SparkUserAppException, SPARK_VERSION}
+import org.apache.spark.{SparkException, SparkUserAppException, SPARK_VERSION}
 import org.apache.spark.api.r.RUtils
 import org.apache.spark.deploy.rest._
 import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, 
Utils}
@@ -521,8 +521,19 @@ object SparkSubmit {
 sysProps.put("spark.yarn.isPython", "true")
   }
   if (args.principal != null) {
-require(args.keytab != null, "Keytab must be specified when the keytab 
is specified")
-UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
+require(args.keytab != null, "Keytab must be specified when principal 
is specified")
+if (!new File(args.keytab).exists()) {
+  throw new SparkException(s"Keytab file: ${args.keytab} does not 
exist")
+} else {
+  // Add keytab and principal configurations in sysProps to make them 
available
+  // for later use; e.g. in spark sql, the isolated class loader used 
to talk
+  // to HiveMetastore will use these settings. They will be set as 
Java system
+  // properties and then loaded by SparkConf
+  sysProps.put("spark.yarn.keytab", args.keytab)
+  sysProps.put("spark.yarn.principal", args.principal)
+
+  UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
+}
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/72c1d68b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index f1c2489..598ccde 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ 
b

spark git commit: [SPARK-9928][SQL] Removal of LogicalLocalTable

2015-11-15 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 b56aaa9be -> eced2766b


[SPARK-9928][SQL] Removal of LogicalLocalTable

LogicalLocalTable in ExistingRDD.scala is replaced by localRelation in 
LocalRelation.scala?

Do you know any reason why we still keep this class?

Author: gatorsmile 

Closes #9717 from gatorsmile/LogicalLocalTable.

(cherry picked from commit b58765caa6d7e6933050565c5d423c45e7e70ba6)
Signed-off-by: Yin Huai 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/eced2766
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/eced2766
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/eced2766

Branch: refs/heads/branch-1.6
Commit: eced2766b18b3cc4a3e2fe23a46664b4f6ec72a3
Parents: b56aaa9
Author: gatorsmile 
Authored: Sun Nov 15 21:10:46 2015 -0800
Committer: Yin Huai 
Committed: Sun Nov 15 21:10:57 2015 -0800

--
 .../spark/sql/execution/ExistingRDD.scala   | 22 
 1 file changed, 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/eced2766/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index 7a466cf..8b41d3d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -110,25 +110,3 @@ private[sql] object PhysicalRDD {
 PhysicalRDD(output, rdd, relation.toString, 
relation.isInstanceOf[HadoopFsRelation])
   }
 }
-
-/** Logical plan node for scanning data from a local collection. */
-private[sql]
-case class LogicalLocalTable(output: Seq[Attribute], rows: 
Seq[InternalRow])(sqlContext: SQLContext)
-   extends LogicalPlan with MultiInstanceRelation {
-
-  override def children: Seq[LogicalPlan] = Nil
-
-  override def newInstance(): this.type =
-LogicalLocalTable(output.map(_.newInstance()), 
rows)(sqlContext).asInstanceOf[this.type]
-
-  override def sameResult(plan: LogicalPlan): Boolean = plan match {
-case LogicalRDD(_, otherRDD) => rows == rows
-case _ => false
-  }
-
-  @transient override lazy val statistics: Statistics = Statistics(
-// TODO: Improve the statistics estimation.
-// This is made small enough so it can be broadcasted.
-sizeInBytes = sqlContext.conf.autoBroadcastJoinThreshold - 1
-  )
-}


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



spark git commit: [SPARK-9928][SQL] Removal of LogicalLocalTable

2015-11-15 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 835a79d78 -> b58765caa


[SPARK-9928][SQL] Removal of LogicalLocalTable

LogicalLocalTable in ExistingRDD.scala is replaced by localRelation in 
LocalRelation.scala?

Do you know any reason why we still keep this class?

Author: gatorsmile 

Closes #9717 from gatorsmile/LogicalLocalTable.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b58765ca
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b58765ca
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b58765ca

Branch: refs/heads/master
Commit: b58765caa6d7e6933050565c5d423c45e7e70ba6
Parents: 835a79d
Author: gatorsmile 
Authored: Sun Nov 15 21:10:46 2015 -0800
Committer: Yin Huai 
Committed: Sun Nov 15 21:10:46 2015 -0800

--
 .../spark/sql/execution/ExistingRDD.scala   | 22 
 1 file changed, 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b58765ca/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index 7a466cf..8b41d3d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -110,25 +110,3 @@ private[sql] object PhysicalRDD {
 PhysicalRDD(output, rdd, relation.toString, 
relation.isInstanceOf[HadoopFsRelation])
   }
 }
-
-/** Logical plan node for scanning data from a local collection. */
-private[sql]
-case class LogicalLocalTable(output: Seq[Attribute], rows: 
Seq[InternalRow])(sqlContext: SQLContext)
-   extends LogicalPlan with MultiInstanceRelation {
-
-  override def children: Seq[LogicalPlan] = Nil
-
-  override def newInstance(): this.type =
-LogicalLocalTable(output.map(_.newInstance()), 
rows)(sqlContext).asInstanceOf[this.type]
-
-  override def sameResult(plan: LogicalPlan): Boolean = plan match {
-case LogicalRDD(_, otherRDD) => rows == rows
-case _ => false
-  }
-
-  @transient override lazy val statistics: Statistics = Statistics(
-// TODO: Improve the statistics estimation.
-// This is made small enough so it can be broadcasted.
-sizeInBytes = sqlContext.conf.autoBroadcastJoinThreshold - 1
-  )
-}


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



spark git commit: [SPARK-8992][SQL] Add pivot to dataframe api

2015-11-11 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 1a21be15f -> b8ff6888e


[SPARK-8992][SQL] Add pivot to dataframe api

This adds a pivot method to the dataframe api.

Following the lead of cube and rollup this adds a Pivot operator that is 
translated into an Aggregate by the analyzer.

Currently the syntax is like:
~~courseSales.pivot(Seq($"year"), $"course", Seq("dotNET", "Java"), 
sum($"earnings"))~~

~~Would we be interested in the following syntax also/alternatively? and~~

courseSales.groupBy($"year").pivot($"course", "dotNET", 
"Java").agg(sum($"earnings"))
//or
courseSales.groupBy($"year").pivot($"course").agg(sum($"earnings"))

Later we can add it to `SQLParser`, but as Hive doesn't support it we cant add 
it there, right?

~~Also what would be the suggested Java friendly method signature for this?~~

Author: Andrew Ray 

Closes #7841 from aray/sql-pivot.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b8ff6888
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b8ff6888
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b8ff6888

Branch: refs/heads/master
Commit: b8ff6888e76b437287d7d6bf2d4b9c759710a195
Parents: 1a21be1
Author: Andrew Ray 
Authored: Wed Nov 11 16:23:24 2015 -0800
Committer: Yin Huai 
Committed: Wed Nov 11 16:23:24 2015 -0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  42 
 .../catalyst/plans/logical/basicOperators.scala |  14 +++
 .../org/apache/spark/sql/GroupedData.scala  | 103 +--
 .../scala/org/apache/spark/sql/SQLConf.scala|   7 ++
 .../apache/spark/sql/DataFramePivotSuite.scala  |  87 
 .../org/apache/spark/sql/test/SQLTestData.scala |  12 +++
 6 files changed, 255 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b8ff6888/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index a9cd9a7..2f4670b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -72,6 +72,7 @@ class Analyzer(
   ResolveRelations ::
   ResolveReferences ::
   ResolveGroupingAnalytics ::
+  ResolvePivot ::
   ResolveSortReferences ::
   ResolveGenerate ::
   ResolveFunctions ::
@@ -166,6 +167,10 @@ class Analyzer(
   case g: GroupingAnalytics if g.child.resolved && 
hasUnresolvedAlias(g.aggregations) =>
 g.withNewAggs(assignAliases(g.aggregations))
 
+  case Pivot(groupByExprs, pivotColumn, pivotValues, aggregates, child)
+if child.resolved && hasUnresolvedAlias(groupByExprs) =>
+Pivot(assignAliases(groupByExprs), pivotColumn, pivotValues, 
aggregates, child)
+
   case Project(projectList, child) if child.resolved && 
hasUnresolvedAlias(projectList) =>
 Project(assignAliases(projectList), child)
 }
@@ -248,6 +253,43 @@ class Analyzer(
 }
   }
 
+  object ResolvePivot extends Rule[LogicalPlan] {
+def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+  case p: Pivot if !p.childrenResolved => p
+  case Pivot(groupByExprs, pivotColumn, pivotValues, aggregates, child) =>
+val singleAgg = aggregates.size == 1
+val pivotAggregates: Seq[NamedExpression] = pivotValues.flatMap { 
value =>
+  def ifExpr(expr: Expression) = {
+If(EqualTo(pivotColumn, value), expr, Literal(null))
+  }
+  aggregates.map { aggregate =>
+val filteredAggregate = aggregate.transformDown {
+  // Assumption is the aggregate function ignores nulls. This is 
true for all current
+  // AggregateFunction's with the exception of First and Last in 
their default mode
+  // (which we handle) and possibly some Hive UDAF's.
+  case First(expr, _) =>
+First(ifExpr(expr), Literal(true))
+  case Last(expr, _) =>
+Last(ifExpr(expr), Literal(true))
+  case a: AggregateFunction =>
+a.withNewChildren(a.children.map(ifExpr))
+}
+if (filteredAggregate.fastEquals(aggregate)) {
+  throw new AnalysisException(
+s"Aggregate expression required for pivot, found '$aggregate'")
+}
+val name = if (singleAgg) value.toString else value + "_" + 
aggregate.prettyString
+Alias(filteredAggregate, name)()
+ 

spark git commit: [SPARK-8992][SQL] Add pivot to dataframe api

2015-11-11 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 4151afbf5 -> 5940fc71d


[SPARK-8992][SQL] Add pivot to dataframe api

This adds a pivot method to the dataframe api.

Following the lead of cube and rollup this adds a Pivot operator that is 
translated into an Aggregate by the analyzer.

Currently the syntax is like:
~~courseSales.pivot(Seq($"year"), $"course", Seq("dotNET", "Java"), 
sum($"earnings"))~~

~~Would we be interested in the following syntax also/alternatively? and~~

courseSales.groupBy($"year").pivot($"course", "dotNET", 
"Java").agg(sum($"earnings"))
//or
courseSales.groupBy($"year").pivot($"course").agg(sum($"earnings"))

Later we can add it to `SQLParser`, but as Hive doesn't support it we cant add 
it there, right?

~~Also what would be the suggested Java friendly method signature for this?~~

Author: Andrew Ray 

Closes #7841 from aray/sql-pivot.

(cherry picked from commit b8ff6888e76b437287d7d6bf2d4b9c759710a195)
Signed-off-by: Yin Huai 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5940fc71
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5940fc71
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5940fc71

Branch: refs/heads/branch-1.6
Commit: 5940fc71d2a245cc6e50edb455c3dd3dbb8de43a
Parents: 4151afb
Author: Andrew Ray 
Authored: Wed Nov 11 16:23:24 2015 -0800
Committer: Yin Huai 
Committed: Wed Nov 11 16:23:39 2015 -0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  42 
 .../catalyst/plans/logical/basicOperators.scala |  14 +++
 .../org/apache/spark/sql/GroupedData.scala  | 103 +--
 .../scala/org/apache/spark/sql/SQLConf.scala|   7 ++
 .../apache/spark/sql/DataFramePivotSuite.scala  |  87 
 .../org/apache/spark/sql/test/SQLTestData.scala |  12 +++
 6 files changed, 255 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5940fc71/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index a9cd9a7..2f4670b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -72,6 +72,7 @@ class Analyzer(
   ResolveRelations ::
   ResolveReferences ::
   ResolveGroupingAnalytics ::
+  ResolvePivot ::
   ResolveSortReferences ::
   ResolveGenerate ::
   ResolveFunctions ::
@@ -166,6 +167,10 @@ class Analyzer(
   case g: GroupingAnalytics if g.child.resolved && 
hasUnresolvedAlias(g.aggregations) =>
 g.withNewAggs(assignAliases(g.aggregations))
 
+  case Pivot(groupByExprs, pivotColumn, pivotValues, aggregates, child)
+if child.resolved && hasUnresolvedAlias(groupByExprs) =>
+Pivot(assignAliases(groupByExprs), pivotColumn, pivotValues, 
aggregates, child)
+
   case Project(projectList, child) if child.resolved && 
hasUnresolvedAlias(projectList) =>
 Project(assignAliases(projectList), child)
 }
@@ -248,6 +253,43 @@ class Analyzer(
 }
   }
 
+  object ResolvePivot extends Rule[LogicalPlan] {
+def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+  case p: Pivot if !p.childrenResolved => p
+  case Pivot(groupByExprs, pivotColumn, pivotValues, aggregates, child) =>
+val singleAgg = aggregates.size == 1
+val pivotAggregates: Seq[NamedExpression] = pivotValues.flatMap { 
value =>
+  def ifExpr(expr: Expression) = {
+If(EqualTo(pivotColumn, value), expr, Literal(null))
+  }
+  aggregates.map { aggregate =>
+val filteredAggregate = aggregate.transformDown {
+  // Assumption is the aggregate function ignores nulls. This is 
true for all current
+  // AggregateFunction's with the exception of First and Last in 
their default mode
+  // (which we handle) and possibly some Hive UDAF's.
+  case First(expr, _) =>
+First(ifExpr(expr), Literal(true))
+  case Last(expr, _) =>
+Last(ifExpr(expr), Literal(true))
+  case a: AggregateFunction =>
+a.withNewChildren(a.children.map(ifExpr))
+}
+if (filteredAggregate.fastEquals(aggregate)) {
+  throw new AnalysisException(
+s"Aggregate expression required for pivot, found '$aggregate'")
+}
+val 

spark git commit: [SPARK-11595][SQL][BRANCH-1.5] Fixes ADD JAR when the input path contains URL scheme

2015-11-12 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 6e823b4d7 -> b478ee374


[SPARK-11595][SQL][BRANCH-1.5] Fixes ADD JAR when the input path contains URL 
scheme

This PR backports #9569 to branch-1.5.

Author: Cheng Lian 

Closes #9570 from liancheng/spark-11595.for-branch-1.5.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b478ee37
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b478ee37
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b478ee37

Branch: refs/heads/branch-1.5
Commit: b478ee3743dd542004f2715e6d00eb999d1a06dd
Parents: 6e823b4
Author: Cheng Lian 
Authored: Thu Nov 12 09:31:43 2015 -0800
Committer: Yin Huai 
Committed: Thu Nov 12 09:31:43 2015 -0800

--
 .../apache/spark/sql/hive/execution/commands.scala   | 15 ++-
 .../spark/sql/hive/execution/SQLQuerySuite.scala |  7 ++-
 2 files changed, 20 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b478ee37/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
index 9f654ee..1615803 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
@@ -17,6 +17,9 @@
 
 package org.apache.spark.sql.hive.execution
 
+import java.io.File
+
+import org.apache.hadoop.fs.Path
 import org.apache.hadoop.hive.metastore.MetaStoreUtils
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.{TableIdentifier, SqlParser}
@@ -90,7 +93,17 @@ case class AddJar(path: String) extends RunnableCommand {
 val currentClassLoader = Utils.getContextOrSparkClassLoader
 
 // Add jar to current context
-val jarURL = new java.io.File(path).toURI.toURL
+val jarURL = {
+  val uri = new Path(path).toUri
+  if (uri.getScheme == null) {
+// `path` is a local file path without a URL scheme
+new File(path).toURI.toURL
+  } else {
+// `path` is a URL with a scheme
+uri.toURL
+  }
+}
+
 val newClassLoader = new java.net.URLClassLoader(Array(jarURL), 
currentClassLoader)
 Thread.currentThread.setContextClassLoader(newClassLoader)
 // We need to explicitly set the class loader associated with the conf in 
executionHive's

http://git-wip-us.apache.org/repos/asf/spark/blob/b478ee37/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index c98fddb..05504f1 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -70,7 +70,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils {
   private val sqlContext = _sqlContext
 
   test("UDTF") {
-sql(s"ADD JAR ${TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath()}")
+val jarPath = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath
+
+// SPARK-11595 Fixes ADD JAR when input path contains URL scheme
+val jarURL = s"file://$jarPath"
+
+sql(s"ADD JAR $jarURL")
 // The function source code can be found at:
 // https://cwiki.apache.org/confluence/display/Hive/DeveloperGuide+UDTF
 sql(


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



spark git commit: [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)

2015-11-10 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 3121e7816 -> 21c562fa0


[SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)

This PR is a 2nd follow-up for 
[SPARK-9241](https://issues.apache.org/jira/browse/SPARK-9241). It contains the 
following improvements:
* Fix for a potential bug in distinct child expression and attribute alignment.
* Improved handling of duplicate distinct child expressions.
* Added test for distinct UDAF with multiple children.

cc yhuai

Author: Herman van Hovell <hvanhov...@questtec.nl>

Closes #9566 from hvanhovell/SPARK-9241-followup-2.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/21c562fa
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/21c562fa
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/21c562fa

Branch: refs/heads/master
Commit: 21c562fa03430365f5c2b7d6de1f8f60ab2140d4
Parents: 3121e78
Author: Herman van Hovell <hvanhov...@questtec.nl>
Authored: Tue Nov 10 16:28:21 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Tue Nov 10 16:28:21 2015 -0800

--
 .../analysis/DistinctAggregationRewriter.scala  |  9 +++--
 .../hive/execution/AggregationQuerySuite.scala  | 41 ++--
 2 files changed, 42 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/21c562fa/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
index 397eff0..c0c9604 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala
@@ -151,11 +151,12 @@ case class DistinctAggregationRewriter(conf: 
CatalystConf) extends Rule[LogicalP
   }
 
   // Setup unique distinct aggregate children.
-  val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq
-  val distinctAggChildAttrMap = 
distinctAggChildren.map(expressionAttributePair).toMap
-  val distinctAggChildAttrs = distinctAggChildAttrMap.values.toSeq
+  val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
+  val distinctAggChildAttrMap = 
distinctAggChildren.map(expressionAttributePair)
+  val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
 
   // Setup expand & aggregate operators for distinct aggregate expressions.
+  val distinctAggChildAttrLookup = distinctAggChildAttrMap.toMap
   val distinctAggOperatorMap = distinctAggGroups.toSeq.zipWithIndex.map {
 case ((group, expressions), i) =>
   val id = Literal(i + 1)
@@ -170,7 +171,7 @@ case class DistinctAggregationRewriter(conf: CatalystConf) 
extends Rule[LogicalP
   val operators = expressions.map { e =>
 val af = e.aggregateFunction
 val naf = patchAggregateFunctionChildren(af) { x =>
-  evalWithinGroup(id, distinctAggChildAttrMap(x))
+  evalWithinGroup(id, distinctAggChildAttrLookup(x))
 }
 (e, e.copy(aggregateFunction = naf, isDistinct = false))
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/21c562fa/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
index 6bf2c53..8253921 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
@@ -66,6 +66,36 @@ class ScalaAggregateFunction(schema: StructType) extends 
UserDefinedAggregateFun
   }
 }
 
+class LongProductSum extends UserDefinedAggregateFunction {
+  def inputSchema: StructType = new StructType()
+.add("a", LongType)
+.add("b", LongType)
+
+  def bufferSchema: StructType = new StructType()
+.add("product", LongType)
+
+  def dataType: DataType = LongType
+
+  def deterministic: Boolean = true
+
+  def initialize(buffer: MutableAggregationBuffer): Unit = {
+buffer(0) = 0L
+  }
+
+  def update(buffer: MutableAggregationBuffer, input: Row): Unit = {
+if (!(input.isNullAt(0) || input.isNullAt(1))) {
+  buffer(0) = buffer.getLong(0) + input.

spark git commit: [SPARK-9034][SQL] Reflect field names defined in GenericUDTF

2015-11-02 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 9cf56c96b -> c34c27fe9


[SPARK-9034][SQL] Reflect field names defined in GenericUDTF

Hive GenericUDTF#initialize() defines field names in a returned schema though,
the current HiveGenericUDTF drops these names.
We might need to reflect these in a logical plan tree.

Author: navis.ryu 

Closes #8456 from navis/SPARK-9034.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c34c27fe
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c34c27fe
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c34c27fe

Branch: refs/heads/master
Commit: c34c27fe9244939d8c905cd689536dfb81c74d7d
Parents: 9cf56c9
Author: navis.ryu 
Authored: Mon Nov 2 23:52:36 2015 -0800
Committer: Yin Huai 
Committed: Mon Nov 2 23:52:36 2015 -0800

--
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala   | 11 +--
 .../spark/sql/catalyst/expressions/generators.scala | 12 +++-
 .../src/main/scala/org/apache/spark/sql/DataFrame.scala | 10 +-
 .../sql/hive/execution/HiveCompatibilitySuite.scala |  1 +
 .../main/scala/org/apache/spark/sql/hive/hiveUDFs.scala |  2 +-
 ...in GenericUDTF #1-0-ff502d8c06f4b32f57aa45057b7fab0e |  1 +
 ...in GenericUDTF #2-0-d6d0def30a7fad5f90fd835361820c30 |  1 +
 ...eral_view_noalias-0-50131c0ba7b7a6b65c789a5a8497bada |  1 +
 ...eral_view_noalias-1-72509f06e1f7c5d5ccc292f775f8eea7 |  0
 ...eral_view_noalias-2-6d5806dd1d2511911a5de1e205523f42 |  2 ++
 ...eral_view_noalias-3-155b3cc2f5054725a9c2acca3c38c00a |  0
 ...eral_view_noalias-4-3b7045ace234af8e5e86d8ac23ccee56 |  2 ++
 ...teral_view_noalias-5-e1eca4e08216897d090259d4fd1e3fe |  0
 ...eral_view_noalias-6-16d227442dd775615c6ecfceedc6c612 |  0
 ...eral_view_noalias-7-66cb5ab20690dd85b2ed95bbfb9481d3 |  2 ++
 .../spark/sql/hive/execution/HiveQuerySuite.scala   |  6 ++
 16 files changed, 34 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c34c27fe/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 912c967..899ee67 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -147,7 +147,7 @@ class Analyzer(
 case u @ UnresolvedAlias(child) => child match {
   case ne: NamedExpression => ne
   case e if !e.resolved => u
-  case g: Generator if g.elementTypes.size > 1 => MultiAlias(g, 
Nil)
+  case g: Generator => MultiAlias(g, Nil)
   case c @ Cast(ne: NamedExpression, _) => Alias(c, ne.name)()
   case other => Alias(other, s"_c$i")()
 }
@@ -722,7 +722,7 @@ class Analyzer(
 
 /**
  * Construct the output attributes for a [[Generator]], given a list of 
names.  If the list of
- * names is empty names are assigned by ordinal (i.e., _c0, _c1, ...) to 
match Hive's defaults.
+ * names is empty names are assigned from field names in generator.
  */
 private def makeGeneratorOutput(
 generator: Generator,
@@ -731,13 +731,12 @@ class Analyzer(
 
   if (names.length == elementTypes.length) {
 names.zip(elementTypes).map {
-  case (name, (t, nullable)) =>
+  case (name, (t, nullable, _)) =>
 AttributeReference(name, t, nullable)()
 }
   } else if (names.isEmpty) {
-elementTypes.zipWithIndex.map {
-  // keep the default column names as Hive does _c0, _c1, _cN
-  case ((t, nullable), i) => AttributeReference(s"_c$i", t, nullable)()
+elementTypes.map {
+  case (t, nullable, name) => AttributeReference(name, t, nullable)()
 }
   } else {
 failAnalysis(

http://git-wip-us.apache.org/repos/asf/spark/blob/c34c27fe/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
index 1a2092c..894a073 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
@@ -53,7 +53,7 @@ trait Generator extends Expression {
* The 

spark git commit: [SPARK-11469][SQL] Allow users to define nondeterministic udfs.

2015-11-02 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master efaa4721b -> 9cf56c96b


[SPARK-11469][SQL] Allow users to define nondeterministic udfs.

This is the first task (https://issues.apache.org/jira/browse/SPARK-11469) of 
https://issues.apache.org/jira/browse/SPARK-11438

Author: Yin Huai <yh...@databricks.com>

Closes #9393 from yhuai/udfNondeterministic.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9cf56c96
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9cf56c96
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9cf56c96

Branch: refs/heads/master
Commit: 9cf56c96b7d02a14175d40b336da14c2e1c88339
Parents: efaa472
Author: Yin Huai <yh...@databricks.com>
Authored: Mon Nov 2 21:18:38 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Mon Nov 2 21:18:38 2015 -0800

--
 project/MimaExcludes.scala  |  47 ++
 .../sql/catalyst/expressions/ScalaUDF.scala |   7 +-
 .../org/apache/spark/sql/UDFRegistration.scala  | 164 ++-
 .../apache/spark/sql/UserDefinedFunction.scala  |  13 +-
 .../scala/org/apache/spark/sql/UDFSuite.scala   | 105 
 .../datasources/parquet/ParquetIOSuite.scala|   4 +-
 6 files changed, 262 insertions(+), 78 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9cf56c96/project/MimaExcludes.scala
--
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 8282f7e..ec0e44b 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -112,6 +112,53 @@ object MimaExcludes {
   "org.apache.spark.rdd.MapPartitionsWithPreparationRDD"),
 ProblemFilters.exclude[MissingClassProblem](
   "org.apache.spark.rdd.MapPartitionsWithPreparationRDD$")
+  ) ++ Seq(
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$2"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$3"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$4"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$5"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$6"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$7"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$8"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$9"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$10"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$11"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$12"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$13"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$14"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$15"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$16"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$17"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$18"),
+ProblemFilters.exclude[MissingMethodProblem](
+  
"org.apache.spark.sql.UD

spark git commit: Revert "[SPARK-11236][CORE] Update Tachyon dependency from 0.7.1 -> 0.8.0."

2015-10-30 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 45029bfde -> e8ec2a7b0


Revert "[SPARK-11236][CORE] Update Tachyon dependency from 0.7.1 -> 0.8.0."

This reverts commit 4f5e60c647d7d6827438721b7fabbc3a57b81023.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e8ec2a7b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e8ec2a7b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e8ec2a7b

Branch: refs/heads/master
Commit: e8ec2a7b01cc86329a6fbafc3d371bdfd79fc1d6
Parents: 45029bf
Author: Yin Huai 
Authored: Fri Oct 30 16:12:33 2015 -0700
Committer: Yin Huai 
Committed: Fri Oct 30 16:12:33 2015 -0700

--
 core/pom.xml | 6 +-
 make-distribution.sh | 8 
 2 files changed, 9 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e8ec2a7b/core/pom.xml
--
diff --git a/core/pom.xml b/core/pom.xml
index dff40e9..319a500 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -266,7 +266,7 @@
 
   org.tachyonproject
   tachyon-client
-  0.8.0
+  0.7.1
   
 
   org.apache.hadoop
@@ -288,6 +288,10 @@
   org.tachyonproject
   tachyon-underfs-glusterfs
 
+
+  org.tachyonproject
+  tachyon-underfs-s3
+
   
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e8ec2a7b/make-distribution.sh
--
diff --git a/make-distribution.sh b/make-distribution.sh
index f676678..24418ac 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -33,9 +33,9 @@ SPARK_HOME="$(cd "`dirname "$0"`"; pwd)"
 DISTDIR="$SPARK_HOME/dist"
 
 SPARK_TACHYON=false
-TACHYON_VERSION="0.8.0"
+TACHYON_VERSION="0.7.1"
 TACHYON_TGZ="tachyon-${TACHYON_VERSION}-bin.tar.gz"
-TACHYON_URL="http://tachyon-project.org/downloads/files/${TACHYON_VERSION}/${TACHYON_TGZ};
+TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/${TACHYON_TGZ};
 
 MAKE_TGZ=false
 NAME=none
@@ -240,10 +240,10 @@ if [ "$SPARK_TACHYON" == "true" ]; then
   fi
 
   tar xzf "${TACHYON_TGZ}"
-  cp 
"tachyon-${TACHYON_VERSION}/assembly/target/tachyon-assemblies-${TACHYON_VERSION}-jar-with-dependencies.jar"
 "$DISTDIR/lib"
+  cp 
"tachyon-${TACHYON_VERSION}/core/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar"
 "$DISTDIR/lib"
   mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web"
   cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon"
-  cp -r "tachyon-${TACHYON_VERSION}"/servers/src/main/java/tachyon/web 
"$DISTDIR/tachyon/src/main/java/tachyon/web"
+  cp -r "tachyon-${TACHYON_VERSION}"/core/src/main/java/tachyon/web 
"$DISTDIR/tachyon/src/main/java/tachyon/web"
 
   if [[ `uname -a` == Darwin* ]]; then
 # need to run sed differently on osx


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



spark git commit: [SPARK-9298][SQL] Add pearson correlation aggregation function

2015-11-01 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master f8d93edec -> 3e770a64a


[SPARK-9298][SQL] Add pearson correlation aggregation function

JIRA: https://issues.apache.org/jira/browse/SPARK-9298

This patch adds pearson correlation aggregation function based on 
`AggregateExpression2`.

Author: Liang-Chi Hsieh 

Closes #8587 from viirya/corr_aggregation.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3e770a64
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3e770a64
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3e770a64

Branch: refs/heads/master
Commit: 3e770a64a48c271c5829d2bcbdc1d6430cda2ac9
Parents: f8d93ed
Author: Liang-Chi Hsieh 
Authored: Sun Nov 1 18:37:27 2015 -0800
Committer: Yin Huai 
Committed: Sun Nov 1 18:37:27 2015 -0800

--
 .../catalyst/analysis/FunctionRegistry.scala|   1 +
 .../expressions/aggregate/functions.scala   | 159 +++
 .../catalyst/expressions/aggregate/utils.scala  |   6 +
 .../sql/catalyst/expressions/aggregates.scala   |  18 +++
 .../scala/org/apache/spark/sql/functions.scala  |  18 +++
 .../hive/execution/HiveCompatibilitySuite.scala |   7 +-
 .../hive/execution/AggregationQuerySuite.scala  | 104 
 7 files changed, 311 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3e770a64/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index ed9fcfe..5f3ec74 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -178,6 +178,7 @@ object FunctionRegistry {
 
 // aggregate functions
 expression[Average]("avg"),
+expression[Corr]("corr"),
 expression[Count]("count"),
 expression[First]("first"),
 expression[First]("first_value"),

http://git-wip-us.apache.org/repos/asf/spark/blob/3e770a64/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
index 281404f..5d2eb7b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
@@ -23,6 +23,7 @@ import java.util
 import com.clearspring.analytics.hash.MurmurHash
 
 import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst._
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
@@ -524,6 +525,164 @@ case class Sum(child: Expression) extends 
DeclarativeAggregate {
   override val evaluateExpression = Cast(currentSum, resultType)
 }
 
+/**
+ * Compute Pearson correlation between two expressions.
+ * When applied on empty data (i.e., count is zero), it returns NULL.
+ *
+ * Definition of Pearson correlation can be found at
+ * http://en.wikipedia.org/wiki/Pearson_product-moment_correlation_coefficient
+ *
+ * @param left one of the expressions to compute correlation with.
+ * @param right another expression to compute correlation with.
+ */
+case class Corr(
+left: Expression,
+right: Expression,
+mutableAggBufferOffset: Int = 0,
+inputAggBufferOffset: Int = 0)
+  extends ImperativeAggregate {
+
+  def children: Seq[Expression] = Seq(left, right)
+
+  def nullable: Boolean = false
+
+  def dataType: DataType = DoubleType
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType, DoubleType)
+
+  def aggBufferSchema: StructType = 
StructType.fromAttributes(aggBufferAttributes)
+
+  def inputAggBufferAttributes: Seq[AttributeReference] = 
aggBufferAttributes.map(_.newInstance())
+
+  val aggBufferAttributes: Seq[AttributeReference] = Seq(
+AttributeReference("xAvg", DoubleType)(),
+AttributeReference("yAvg", DoubleType)(),
+AttributeReference("Ck", DoubleType)(),
+AttributeReference("MkX", DoubleType)(),
+AttributeReference("MkY", DoubleType)(),
+AttributeReference("count", LongType)())
+
+  // Local cache of mutableAggBufferOffset(s) that will be used in update and 
merge
+  

spark git commit: [SPARK-11434][SPARK-11103][SQL] Fix test ": Filter applied on merged Parquet schema with new column fails"

2015-10-30 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 6b10ea5d5 -> c9ac0e918


[SPARK-11434][SPARK-11103][SQL] Fix test ": Filter applied on merged Parquet 
schema with new column fails"

https://issues.apache.org/jira/browse/SPARK-11434

Author: Yin Huai <yh...@databricks.com>

Closes #9387 from yhuai/SPARK-11434.

(cherry picked from commit 3c471885dc4f86bea95ab542e0d48d22ae748404)
Signed-off-by: Yin Huai <yh...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c9ac0e91
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c9ac0e91
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c9ac0e91

Branch: refs/heads/branch-1.5
Commit: c9ac0e9180badf28050ab0909d6d191d0718622d
Parents: 6b10ea5
Author: Yin Huai <yh...@databricks.com>
Authored: Fri Oct 30 20:05:07 2015 -0700
Committer: Yin Huai <yh...@databricks.com>
Committed: Fri Oct 30 20:05:20 2015 -0700

--
 .../sql/execution/datasources/parquet/ParquetFilterSuite.scala | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c9ac0e91/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
index b2101be..f88ddc7 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
@@ -323,15 +323,15 @@ class ParquetFilterSuite extends QueryTest with 
ParquetTest with SharedSQLContex
 withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true",
   SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") {
   withTempPath { dir =>
-var pathOne = s"${dir.getCanonicalPath}/table1"
+val pathOne = s"${dir.getCanonicalPath}/table1"
 (1 to 3).map(i => (i, i.toString)).toDF("a", 
"b").write.parquet(pathOne)
-var pathTwo = s"${dir.getCanonicalPath}/table2"
+val pathTwo = s"${dir.getCanonicalPath}/table2"
 (1 to 3).map(i => (i, i.toString)).toDF("c", 
"b").write.parquet(pathTwo)
 
 // If the "c = 1" filter gets pushed down, this query will throw an 
exception which
 // Parquet emits. This is a Parquet issue (PARQUET-389).
 checkAnswer(
-  sqlContext.read.parquet(pathOne, pathTwo).filter("c = 1"),
+  sqlContext.read.parquet(pathOne, pathTwo).filter("c = 
1").selectExpr("c", "b", "a"),
   (1 to 1).map(i => Row(i, i.toString, null)))
   }
 }


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



spark git commit: [SPARK-11504][SQL] API audit for distributeBy and localSort

2015-11-04 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master de289bf27 -> abf5e4285


[SPARK-11504][SQL] API audit for distributeBy and localSort

1. Renamed localSort -> sortWithinPartitions to avoid ambiguity in "local"
2. distributeBy -> repartition to match the existing repartition.

Author: Reynold Xin 

Closes #9470 from rxin/SPARK-11504.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/abf5e428
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/abf5e428
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/abf5e428

Branch: refs/heads/master
Commit: abf5e4285d97b148a32cf22f5287511198175cb6
Parents: de289bf
Author: Reynold Xin 
Authored: Wed Nov 4 12:33:47 2015 -0800
Committer: Yin Huai 
Committed: Wed Nov 4 12:33:47 2015 -0800

--
 .../scala/org/apache/spark/sql/DataFrame.scala  | 132 +++
 .../org/apache/spark/sql/CachedTableSuite.scala |  20 ++-
 .../org/apache/spark/sql/DataFrameSuite.scala   |  44 +++
 3 files changed, 113 insertions(+), 83 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/abf5e428/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
index 5e9c7ef..d3a2249 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -241,18 +241,6 @@ class DataFrame private[sql](
 sb.toString()
   }
 
-  private[sql] def sortInternal(global: Boolean, sortExprs: Seq[Column]): 
DataFrame = {
-val sortOrder: Seq[SortOrder] = sortExprs.map { col =>
-  col.expr match {
-case expr: SortOrder =>
-  expr
-case expr: Expression =>
-  SortOrder(expr, Ascending)
-  }
-}
-Sort(sortOrder, global = global, logicalPlan)
-  }
-
   override def toString: String = {
 try {
   schema.map(f => s"${f.name}: ${f.dataType.simpleString}").mkString("[", 
", ", "]")
@@ -620,6 +608,32 @@ class DataFrame private[sql](
   }
 
   /**
+   * Returns a new [[DataFrame]] with each partition sorted by the given 
expressions.
+   *
+   * This is the same operation as "SORT BY" in SQL (Hive QL).
+   *
+   * @group dfops
+   * @since 1.6.0
+   */
+  @scala.annotation.varargs
+  def sortWithinPartitions(sortCol: String, sortCols: String*): DataFrame = {
+sortWithinPartitions(sortCol, sortCols : _*)
+  }
+
+  /**
+   * Returns a new [[DataFrame]] with each partition sorted by the given 
expressions.
+   *
+   * This is the same operation as "SORT BY" in SQL (Hive QL).
+   *
+   * @group dfops
+   * @since 1.6.0
+   */
+  @scala.annotation.varargs
+  def sortWithinPartitions(sortExprs: Column*): DataFrame = {
+sortInternal(global = false, sortExprs)
+  }
+
+  /**
* Returns a new [[DataFrame]] sorted by the specified column, all in 
ascending order.
* {{{
*   // The following 3 are equivalent
@@ -645,7 +659,7 @@ class DataFrame private[sql](
*/
   @scala.annotation.varargs
   def sort(sortExprs: Column*): DataFrame = {
-sortInternal(true, sortExprs)
+sortInternal(global = true, sortExprs)
   }
 
   /**
@@ -667,44 +681,6 @@ class DataFrame private[sql](
   def orderBy(sortExprs: Column*): DataFrame = sort(sortExprs : _*)
 
   /**
-   * Returns a new [[DataFrame]] partitioned by the given partitioning 
expressions into
-   * `numPartitions`. The resulting DataFrame is hash partitioned.
-   * @group dfops
-   * @since 1.6.0
-   */
-  def distributeBy(partitionExprs: Seq[Column], numPartitions: Int): DataFrame 
= {
-RepartitionByExpression(partitionExprs.map { _.expr }, logicalPlan, 
Some(numPartitions))
-  }
-
-  /**
-   * Returns a new [[DataFrame]] partitioned by the given partitioning 
expressions preserving
-   * the existing number of partitions. The resulting DataFrame is hash 
partitioned.
-   * @group dfops
-   * @since 1.6.0
-   */
-  def distributeBy(partitionExprs: Seq[Column]): DataFrame = {
-RepartitionByExpression(partitionExprs.map { _.expr }, logicalPlan, None)
-  }
-
-  /**
-   * Returns a new [[DataFrame]] with each partition sorted by the given 
expressions.
-   * @group dfops
-   * @since 1.6.0
-   */
-  @scala.annotation.varargs
-  def localSort(sortCol: String, sortCols: String*): DataFrame = 
localSort(sortCol, sortCols : _*)
-
-  /**
-   * Returns a new [[DataFrame]] with each partition sorted by the given 
expressions.
-   * @group dfops
-   * @since 1.6.0
-   */
-  @scala.annotation.varargs
-  def localSort(sortExprs: Column*): DataFrame = {
-sortInternal(false, sortExprs)
-  }
-
-  /**
* Selects column 

spark git commit: [SPARK-11510][SQL] Remove SQL aggregation tests for higher order statistics

2015-11-04 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 411ff6afb -> b6e0a5ae6


[SPARK-11510][SQL] Remove SQL aggregation tests for higher order statistics

We have some aggregate function tests in both DataFrameAggregateSuite and 
SQLQuerySuite. The two have almost the same coverage and we should just remove 
the SQL one.

Author: Reynold Xin 

Closes #9475 from rxin/SPARK-11510.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b6e0a5ae
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b6e0a5ae
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b6e0a5ae

Branch: refs/heads/master
Commit: b6e0a5ae6f243139f11c9cbbf18cddd3f25db208
Parents: 411ff6a
Author: Reynold Xin 
Authored: Wed Nov 4 16:49:25 2015 -0800
Committer: Yin Huai 
Committed: Wed Nov 4 16:49:25 2015 -0800

--
 .../spark/sql/DataFrameAggregateSuite.scala | 97 ++--
 .../org/apache/spark/sql/SQLQuerySuite.scala| 77 
 .../apache/spark/sql/StringFunctionsSuite.scala |  1 -
 3 files changed, 28 insertions(+), 147 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b6e0a5ae/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
index b0e2ffa..2e679e7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
@@ -83,13 +83,8 @@ class DataFrameAggregateSuite extends QueryTest with 
SharedSQLContext {
 
   test("average") {
 checkAnswer(
-  testData2.agg(avg('a)),
-  Row(2.0))
-
-// Also check mean
-checkAnswer(
-  testData2.agg(mean('a)),
-  Row(2.0))
+  testData2.agg(avg('a), mean('a)),
+  Row(2.0, 2.0))
 
 checkAnswer(
   testData2.agg(avg('a), sumDistinct('a)), // non-partial
@@ -98,6 +93,7 @@ class DataFrameAggregateSuite extends QueryTest with 
SharedSQLContext {
 checkAnswer(
   decimalData.agg(avg('a)),
   Row(new java.math.BigDecimal(2.0)))
+
 checkAnswer(
   decimalData.agg(avg('a), sumDistinct('a)), // non-partial
   Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil)
@@ -168,44 +164,23 @@ class DataFrameAggregateSuite extends QueryTest with 
SharedSQLContext {
 
   test("zero count") {
 val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b")
-assert(emptyTableData.count() === 0)
-
 checkAnswer(
   emptyTableData.agg(count('a), sumDistinct('a)), // non-partial
   Row(0, null))
   }
 
   test("stddev") {
-val testData2ADev = math.sqrt(4/5.0)
-
+val testData2ADev = math.sqrt(4 / 5.0)
 checkAnswer(
-  testData2.agg(stddev('a)),
-  Row(testData2ADev))
-
-checkAnswer(
-  testData2.agg(stddev_pop('a)),
-  Row(math.sqrt(4/6.0)))
-
-checkAnswer(
-  testData2.agg(stddev_samp('a)),
-  Row(testData2ADev))
+  testData2.agg(stddev('a), stddev_pop('a), stddev_samp('a)),
+  Row(testData2ADev, math.sqrt(4 / 6.0), testData2ADev))
   }
 
   test("zero stddev") {
 val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b")
-assert(emptyTableData.count() == 0)
-
-checkAnswer(
-emptyTableData.agg(stddev('a)),
-Row(null))
-
 checkAnswer(
-emptyTableData.agg(stddev_pop('a)),
-Row(null))
-
-checkAnswer(
-emptyTableData.agg(stddev_samp('a)),
-Row(null))
+emptyTableData.agg(stddev('a), stddev_pop('a), stddev_samp('a)),
+Row(null, null, null))
   }
 
   test("zero sum") {
@@ -227,6 +202,7 @@ class DataFrameAggregateSuite extends QueryTest with 
SharedSQLContext {
 
 val sparkVariance = testData2.agg(variance('a))
 checkAggregatesWithTol(sparkVariance, Row(4.0 / 5.0), absTol)
+
 val sparkVariancePop = testData2.agg(var_pop('a))
 checkAggregatesWithTol(sparkVariancePop, Row(4.0 / 6.0), absTol)
 
@@ -241,52 +217,35 @@ class DataFrameAggregateSuite extends QueryTest with 
SharedSQLContext {
   }
 
   test("zero moments") {
-val emptyTableData = Seq((1, 2)).toDF("a", "b")
-assert(emptyTableData.count() === 1)
-
-checkAnswer(
-  emptyTableData.agg(variance('a)),
-  Row(Double.NaN))
-
-checkAnswer(
-  emptyTableData.agg(var_samp('a)),
-  Row(Double.NaN))
-
+val input = Seq((1, 2)).toDF("a", "b")
 checkAnswer(
-  emptyTableData.agg(var_pop('a)),
-  Row(0.0))
+  input.agg(variance('a), var_samp('a), var_pop('a), skewness('a), 
kurtosis('a)),
+  Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN))
 
   

spark git commit: [SPARK-10978][SQL] Allow data sources to eliminate filters

2015-11-03 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master b2e4b314d -> ebf8b0b48


[SPARK-10978][SQL] Allow data sources to eliminate filters

This PR adds a new method `unhandledFilters` to `BaseRelation`. Data sources 
which implement this method properly may avoid the overhead of defensive 
filtering done by Spark SQL.

Author: Cheng Lian 

Closes #9399 from liancheng/spark-10978.unhandled-filters.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ebf8b0b4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ebf8b0b4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ebf8b0b4

Branch: refs/heads/master
Commit: ebf8b0b48deaad64f7ca27051caee763451e2623
Parents: b2e4b31
Author: Cheng Lian 
Authored: Tue Nov 3 10:07:45 2015 -0800
Committer: Yin Huai 
Committed: Tue Nov 3 10:07:45 2015 -0800

--
 .../datasources/DataSourceStrategy.scala| 131 +++
 .../apache/spark/sql/sources/interfaces.scala   |   9 ++
 .../parquet/ParquetFilterSuite.scala|   2 +-
 .../spark/sql/sources/FilteredScanSuite.scala   | 129 +-
 .../SimpleTextHadoopFsRelationSuite.scala   |  47 ++-
 .../spark/sql/sources/SimpleTextRelation.scala  |  65 -
 6 files changed, 315 insertions(+), 68 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ebf8b0b4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
index 6585986..7265d6a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
@@ -43,7 +43,8 @@ private[sql] object DataSourceStrategy extends Strategy with 
Logging {
 l,
 projects,
 filters,
-(a, f) => toCatalystRDD(l, a, t.buildScan(a, f))) :: Nil
+(requestedColumns, allPredicates, _) =>
+  toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, 
allPredicates))) :: Nil
 
 case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
PrunedFilteredScan, _)) =>
   pruneFilterProject(
@@ -266,47 +267,81 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
   relation,
   projects,
   filterPredicates,
-  (requestedColumns, pushedFilters) => {
-scanBuilder(requestedColumns, selectFilters(pushedFilters).toArray)
+  (requestedColumns, _, pushedFilters) => {
+scanBuilder(requestedColumns, pushedFilters.toArray)
   })
   }
 
-  // Based on Catalyst expressions.
+  // Based on Catalyst expressions. The `scanBuilder` function accepts three 
arguments:
+  //
+  //  1. A `Seq[Attribute]`, containing all required column attributes. Used 
to handle relation
+  // traits that support column pruning (e.g. `PrunedScan` and 
`PrunedFilteredScan`).
+  //
+  //  2. A `Seq[Expression]`, containing all gathered Catalyst filter 
expressions, only used for
+  // `CatalystScan`.
+  //
+  //  3. A `Seq[Filter]`, containing all data source `Filter`s that are 
converted from (possibly a
+  // subset of) Catalyst filter expressions and can be handled by 
`relation`.  Used to handle
+  // relation traits (`CatalystScan` excluded) that support filter 
push-down (e.g.
+  // `PrunedFilteredScan` and `HadoopFsRelation`).
+  //
+  // Note that 2 and 3 shouldn't be used together.
   protected def pruneFilterProjectRaw(
-  relation: LogicalRelation,
-  projects: Seq[NamedExpression],
-  filterPredicates: Seq[Expression],
-  scanBuilder: (Seq[Attribute], Seq[Expression]) => RDD[InternalRow]) = {
+relation: LogicalRelation,
+projects: Seq[NamedExpression],
+filterPredicates: Seq[Expression],
+scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter]) => 
RDD[InternalRow]) = {
 
 val projectSet = AttributeSet(projects.flatMap(_.references))
 val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
-val filterCondition = filterPredicates.reduceLeftOption(expressions.And)
 
-val pushedFilters = filterPredicates.map { _ transform {
+val candidatePredicates = filterPredicates.map { _ transform {
   case a: AttributeReference => relation.attributeMap(a) // Match original 
case of attributes.
 }}
 
+val (unhandledPredicates, pushedFilters) =
+  selectFilters(relation.relation, candidatePredicates)
+
+// A set of column attributes that are only 

spark git commit: [SPARK-10648][SQL][BRANCH-1.4] Oracle dialect to handle nonspecific numeric types

2015-11-05 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 8e1bd6089 -> 0eb233507


[SPARK-10648][SQL][BRANCH-1.4] Oracle dialect to handle nonspecific numeric 
types

This backports https://github.com/apache/spark/pull/9495 to branch-1.4 (in case 
anyone needs this).

Author: Yin Huai <yh...@databricks.com>

Closes #9498 from yhuai/OracleDialect-1.4.

(cherry picked from commit 6c5e9a3a056cc8ee660a2b22a0a5ff17d674b68d)
Signed-off-by: Yin Huai <yh...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0eb23350
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0eb23350
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0eb23350

Branch: refs/heads/branch-1.5
Commit: 0eb2335071b216b13ef68ac92d6898490331c758
Parents: 8e1bd60
Author: Yin Huai <yh...@databricks.com>
Authored: Thu Nov 5 11:44:34 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Thu Nov 5 11:46:39 2015 -0800

--
 .../apache/spark/sql/jdbc/JdbcDialects.scala| 26 
 1 file changed, 26 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0eb23350/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
index 8849fc2..8ae87d8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
@@ -125,6 +125,7 @@ object JdbcDialects {
 
   registerDialect(MySQLDialect)
   registerDialect(PostgresDialect)
+  registerDialect(OracleDialect)
 
   /**
* Fetch the JdbcDialect class corresponding to a given database url.
@@ -222,3 +223,28 @@ case object MySQLDialect extends JdbcDialect {
 s"`$colName`"
   }
 }
+
+/**
+ * :: DeveloperApi ::
+ * Default Oracle dialect, mapping a nonspecific numeric type to a general 
decimal type.
+ */
+@DeveloperApi
+case object OracleDialect extends JdbcDialect {
+  override def canHandle(url: String): Boolean = url.startsWith("jdbc:oracle")
+  override def getCatalystType(
+  sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): 
Option[DataType] = {
+// Handle NUMBER fields that have no precision/scale in special way
+// because JDBC ResultSetMetaData converts this to 0 precision and -127 
scale
+// For more details, please see
+// https://github.com/apache/spark/pull/8780#issuecomment-145598968
+// and
+// https://github.com/apache/spark/pull/8780#issuecomment-144541760
+if (sqlType == Types.NUMERIC && size == 0) {
+  // This is sub-optimal as we have to pick a precision/scale in advance 
whereas the data
+  // in Oracle is allowed to have different precision/scale for each value.
+  Some(DecimalType(38, 10))
+} else {
+  None
+}
+  }
+}


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



spark git commit: [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up

2015-11-07 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 2ff0e79a8 -> ef362846e


[SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up

This PR is a follow up for PR https://github.com/apache/spark/pull/9406. It 
adds more documentation to the rewriting rule, removes a redundant if 
expression in the non-distinct aggregation path and adds a multiple distinct 
test to the AggregationQuerySuite.

cc yhuai marmbrus

Author: Herman van Hovell <hvanhov...@questtec.nl>

Closes #9541 from hvanhovell/SPARK-9241-followup.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ef362846
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ef362846
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ef362846

Branch: refs/heads/master
Commit: ef362846eb448769bcf774fc9090a5013d459464
Parents: 2ff0e79
Author: Herman van Hovell <hvanhov...@questtec.nl>
Authored: Sat Nov 7 13:37:37 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Sat Nov 7 13:37:37 2015 -0800

--
 .../catalyst/expressions/aggregate/Utils.scala  | 114 +++
 .../hive/execution/AggregationQuerySuite.scala  |  17 +++
 2 files changed, 108 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ef362846/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
index 39010c3..ac23f72 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
@@ -222,10 +222,76 @@ object Utils {
  * aggregation in which the regular aggregation expressions and every distinct 
clause is aggregated
  * in a separate group. The results are then combined in a second aggregate.
  *
- * TODO Expression cannocalization
- * TODO Eliminate foldable expressions from distinct clauses.
- * TODO This eliminates all distinct expressions. We could safely pass one to 
the aggregate
- *  operator. Perhaps this is a good thing? It is much simpler to plan 
later on...
+ * For example (in scala):
+ * {{{
+ *   val data = Seq(
+ * ("a", "ca1", "cb1", 10),
+ * ("a", "ca1", "cb2", 5),
+ * ("b", "ca1", "cb1", 13))
+ * .toDF("key", "cat1", "cat2", "value")
+ *   data.registerTempTable("data")
+ *
+ *   val agg = data.groupBy($"key")
+ * .agg(
+ *   countDistinct($"cat1").as("cat1_cnt"),
+ *   countDistinct($"cat2").as("cat2_cnt"),
+ *   sum($"value").as("total"))
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *key = ['key]
+ *functions = [COUNT(DISTINCT 'cat1),
+ * COUNT(DISTINCT 'cat2),
+ * sum('value)]
+ *output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *key = ['key]
+ *functions = [count(if (('gid = 1)) 'cat1 else null),
+ * count(if (('gid = 2)) 'cat2 else null),
+ * first(if (('gid = 0)) 'total else null) ignore nulls]
+ *output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   Aggregate(
+ *  key = ['key, 'cat1, 'cat2, 'gid]
+ *  functions = [sum('value)]
+ *  output = ['key, 'cat1, 'cat2, 'gid, 'total])
+ * Expand(
+ *projections = [('key, null, null, 0, cast('value as bigint)),
+ *   ('key, 'cat1, null, 1, null),
+ *   ('key, null, 'cat2, 2, null)]
+ *output = ['key, 'cat1, 'cat2, 'gid, 'value])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * The rule does the following things here:
+ * 1. Expand the data. There are three aggregation groups in this query:
+ *i. the non-distinct group;
+ *ii. the distinct 'cat1 group;
+ *iii. the distinct 'cat2 group.
+ *An expand operator is inserted to expand the child data for each group. 
The expand will null
+ *out all unused columns for the given group; this must be done in order 
to ensure correctness
+ *later on. Groups can by identified by a group id (gid) column added by 
the expand operator.
+ * 2. De-duplicate the distinct paths and aggregate the non-aggregate path. 
The group by clause of
+ *this a

spark git commit: [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up

2015-11-07 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 aede729a9 -> 696d4a52d


[SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up

This PR is a follow up for PR https://github.com/apache/spark/pull/9406. It 
adds more documentation to the rewriting rule, removes a redundant if 
expression in the non-distinct aggregation path and adds a multiple distinct 
test to the AggregationQuerySuite.

cc yhuai marmbrus

Author: Herman van Hovell <hvanhov...@questtec.nl>

Closes #9541 from hvanhovell/SPARK-9241-followup.

(cherry picked from commit ef362846eb448769bcf774fc9090a5013d459464)
Signed-off-by: Yin Huai <yh...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/696d4a52
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/696d4a52
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/696d4a52

Branch: refs/heads/branch-1.6
Commit: 696d4a52d8ee5c1c736ce470ac87255fe58e78c3
Parents: aede729
Author: Herman van Hovell <hvanhov...@questtec.nl>
Authored: Sat Nov 7 13:37:37 2015 -0800
Committer: Yin Huai <yh...@databricks.com>
Committed: Sat Nov 7 13:38:08 2015 -0800

--
 .../catalyst/expressions/aggregate/Utils.scala  | 114 +++
 .../hive/execution/AggregationQuerySuite.scala  |  17 +++
 2 files changed, 108 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/696d4a52/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
index 39010c3..ac23f72 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Utils.scala
@@ -222,10 +222,76 @@ object Utils {
  * aggregation in which the regular aggregation expressions and every distinct 
clause is aggregated
  * in a separate group. The results are then combined in a second aggregate.
  *
- * TODO Expression cannocalization
- * TODO Eliminate foldable expressions from distinct clauses.
- * TODO This eliminates all distinct expressions. We could safely pass one to 
the aggregate
- *  operator. Perhaps this is a good thing? It is much simpler to plan 
later on...
+ * For example (in scala):
+ * {{{
+ *   val data = Seq(
+ * ("a", "ca1", "cb1", 10),
+ * ("a", "ca1", "cb2", 5),
+ * ("b", "ca1", "cb1", 13))
+ * .toDF("key", "cat1", "cat2", "value")
+ *   data.registerTempTable("data")
+ *
+ *   val agg = data.groupBy($"key")
+ * .agg(
+ *   countDistinct($"cat1").as("cat1_cnt"),
+ *   countDistinct($"cat2").as("cat2_cnt"),
+ *   sum($"value").as("total"))
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *key = ['key]
+ *functions = [COUNT(DISTINCT 'cat1),
+ * COUNT(DISTINCT 'cat2),
+ * sum('value)]
+ *output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *key = ['key]
+ *functions = [count(if (('gid = 1)) 'cat1 else null),
+ * count(if (('gid = 2)) 'cat2 else null),
+ * first(if (('gid = 0)) 'total else null) ignore nulls]
+ *output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   Aggregate(
+ *  key = ['key, 'cat1, 'cat2, 'gid]
+ *  functions = [sum('value)]
+ *  output = ['key, 'cat1, 'cat2, 'gid, 'total])
+ * Expand(
+ *projections = [('key, null, null, 0, cast('value as bigint)),
+ *   ('key, 'cat1, null, 1, null),
+ *   ('key, null, 'cat2, 2, null)]
+ *output = ['key, 'cat1, 'cat2, 'gid, 'value])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * The rule does the following things here:
+ * 1. Expand the data. There are three aggregation groups in this query:
+ *i. the non-distinct group;
+ *ii. the distinct 'cat1 group;
+ *iii. the distinct 'cat2 group.
+ *An expand operator is inserted to expand the child data for each group. 
The expand will null
+ *out all unused columns for the given group; this must be done in order 
to ensure correctness
+ *later on. Groups can by identified by a group id (gid) c

spark git commit: [SPARK-11329] [SQL] Cleanup from spark-11329 fix.

2015-11-03 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master d648a4ad5 -> e352de0db


[SPARK-11329] [SQL] Cleanup from spark-11329 fix.

Author: Nong 

Closes #9442 from nongli/spark-11483.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e352de0d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e352de0d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e352de0d

Branch: refs/heads/master
Commit: e352de0db2789919e1e0385b79f29b508a6b2b77
Parents: d648a4a
Author: Nong 
Authored: Tue Nov 3 16:44:37 2015 -0800
Committer: Yin Huai 
Committed: Tue Nov 3 16:44:37 2015 -0800

--
 .../apache/spark/sql/catalyst/SqlParser.scala   |  4 +-
 .../sql/catalyst/analysis/unresolved.scala  | 18 +
 .../scala/org/apache/spark/sql/Column.scala |  6 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala| 79 
 4 files changed, 55 insertions(+), 52 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e352de0d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
index 1ba559d..440e9e2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -477,8 +477,8 @@ object SqlParser extends AbstractSparkSQLParser with 
DataTypeParser {
 
   protected lazy val baseExpression: Parser[Expression] =
 ( "*" ^^^ UnresolvedStar(None)
-| (ident <~ "."). + <~ "*" ^^ { case target => { 
UnresolvedStar(Option(target)) }
-} | primary
+| (ident <~ "."). + <~ "*" ^^ { case target => 
UnresolvedStar(Option(target))}
+| primary
)
 
   protected lazy val signedPrimary: Parser[Expression] =

http://git-wip-us.apache.org/repos/asf/spark/blob/e352de0d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
index 6975662..eae17c8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
@@ -183,28 +183,16 @@ case class UnresolvedStar(target: Option[Seq[String]]) 
extends Star with Unevalu
   case None => input.output
   // If there is a table, pick out attributes that are part of this table.
   case Some(t) => if (t.size == 1) {
-input.output.filter(_.qualifiers.filter(resolver(_, t.head)).nonEmpty)
+input.output.filter(_.qualifiers.exists(resolver(_, t.head)))
   } else {
 List()
   }
 }
-if (!expandedAttributes.isEmpty) {
-  if (expandedAttributes.forall(_.isInstanceOf[NamedExpression])) {
-return expandedAttributes
-  } else {
-require(expandedAttributes.size == input.output.size)
-expandedAttributes.zip(input.output).map {
-  case (e, originalAttribute) =>
-Alias(e, originalAttribute.name)(qualifiers = 
originalAttribute.qualifiers)
-}
-  }
-  return expandedAttributes
-}
-
-require(target.isDefined)
+if (expandedAttributes.nonEmpty) return expandedAttributes
 
 // Try to resolve it as a struct expansion. If there is a conflict and 
both are possible,
 // (i.e. [name].* is both a table and a struct), the struct path can 
always be qualified.
+require(target.isDefined)
 val attribute = input.resolve(target.get, resolver)
 if (attribute.isDefined) {
   // This target resolved to an attribute in child. It must be a struct. 
Expand it.

http://git-wip-us.apache.org/repos/asf/spark/blob/e352de0d/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
index 3cde9d6..c73f696 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
@@ -60,8 +60,10 @@ class Column(protected[sql] val expr: Expression) extends 
Logging {
 
   def this(name: String) = this(name match {
 case "*" => UnresolvedStar(None)
-case _ if name.endsWith(".*") => 
UnresolvedStar(Some(UnresolvedAttribute.parseAttributeName(
-  name.substring(0, 

spark git commit: [SPARK-11490][SQL] variance should alias var_samp instead of var_pop.

2015-11-04 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master e0fc9c7e5 -> 3bd6f5d2a


[SPARK-11490][SQL] variance should alias var_samp instead of var_pop.

stddev is an alias for stddev_samp. variance should be consistent with stddev.

Also took the chance to remove internal Stddev and Variance, and only kept 
StddevSamp/StddevPop and VarianceSamp/VariancePop.

Author: Reynold Xin 

Closes #9449 from rxin/SPARK-11490.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3bd6f5d2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3bd6f5d2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3bd6f5d2

Branch: refs/heads/master
Commit: 3bd6f5d2ae503468de0e218d51c331e249a862bb
Parents: e0fc9c7
Author: Reynold Xin 
Authored: Wed Nov 4 09:34:52 2015 -0800
Committer: Yin Huai 
Committed: Wed Nov 4 09:34:52 2015 -0800

--
 .../catalyst/analysis/FunctionRegistry.scala|  4 +-
 .../catalyst/analysis/HiveTypeCoercion.scala|  2 -
 .../apache/spark/sql/catalyst/dsl/package.scala |  8 
 .../expressions/aggregate/functions.scala   | 29 -
 .../catalyst/expressions/aggregate/utils.scala  | 12 --
 .../sql/catalyst/expressions/aggregates.scala   | 45 +---
 .../scala/org/apache/spark/sql/DataFrame.scala  |  2 +-
 .../org/apache/spark/sql/GroupedData.scala  |  4 +-
 .../scala/org/apache/spark/sql/functions.scala  |  9 ++--
 .../spark/sql/DataFrameAggregateSuite.scala | 17 +++-
 .../org/apache/spark/sql/SQLQuerySuite.scala| 14 +++---
 11 files changed, 32 insertions(+), 114 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3bd6f5d2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 24c1a7b..d4334d1 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -187,11 +187,11 @@ object FunctionRegistry {
 expression[Max]("max"),
 expression[Average]("mean"),
 expression[Min]("min"),
-expression[Stddev]("stddev"),
+expression[StddevSamp]("stddev"),
 expression[StddevPop]("stddev_pop"),
 expression[StddevSamp]("stddev_samp"),
 expression[Sum]("sum"),
-expression[Variance]("variance"),
+expression[VarianceSamp]("variance"),
 expression[VariancePop]("var_pop"),
 expression[VarianceSamp]("var_samp"),
 expression[Skewness]("skewness"),

http://git-wip-us.apache.org/repos/asf/spark/blob/3bd6f5d2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index 3c67567..84e2b13 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -297,10 +297,8 @@ object HiveTypeCoercion {
   case Sum(e @ StringType()) => Sum(Cast(e, DoubleType))
   case SumDistinct(e @ StringType()) => Sum(Cast(e, DoubleType))
   case Average(e @ StringType()) => Average(Cast(e, DoubleType))
-  case Stddev(e @ StringType()) => Stddev(Cast(e, DoubleType))
   case StddevPop(e @ StringType()) => StddevPop(Cast(e, DoubleType))
   case StddevSamp(e @ StringType()) => StddevSamp(Cast(e, DoubleType))
-  case Variance(e @ StringType()) => Variance(Cast(e, DoubleType))
   case VariancePop(e @ StringType()) => VariancePop(Cast(e, DoubleType))
   case VarianceSamp(e @ StringType()) => VarianceSamp(Cast(e, DoubleType))
   case Skewness(e @ StringType()) => Skewness(Cast(e, DoubleType))

http://git-wip-us.apache.org/repos/asf/spark/blob/3bd6f5d2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index 787f67a..d8df664 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ 

spark git commit: [SPARK-11455][SQL] fix case sensitivity of partition by

2015-11-03 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master e352de0db -> 2692bdb7d


[SPARK-11455][SQL] fix case sensitivity of partition by

depend on `caseSensitive` to do column name equality check, instead of just `==`

Author: Wenchen Fan 

Closes #9410 from cloud-fan/partition.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2692bdb7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2692bdb7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2692bdb7

Branch: refs/heads/master
Commit: 2692bdb7dbf36d6247f595d5fd0cb9cda89e1fdd
Parents: e352de0
Author: Wenchen Fan 
Authored: Tue Nov 3 20:25:58 2015 -0800
Committer: Yin Huai 
Committed: Tue Nov 3 20:25:58 2015 -0800

--
 .../datasources/PartitioningUtils.scala |  7 ++---
 .../datasources/ResolvedDataSource.scala| 27 +++-
 .../spark/sql/execution/datasources/rules.scala |  6 +++--
 .../org/apache/spark/sql/DataFrameSuite.scala   | 10 
 4 files changed, 39 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2692bdb7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
index 628c5e1..16dc236 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
@@ -287,10 +287,11 @@ private[sql] object PartitioningUtils {
 
   def validatePartitionColumnDataTypes(
   schema: StructType,
-  partitionColumns: Array[String]): Unit = {
+  partitionColumns: Array[String],
+  caseSensitive: Boolean): Unit = {
 
-ResolvedDataSource.partitionColumnsSchema(schema, 
partitionColumns).foreach { field =>
-  field.dataType match {
+ResolvedDataSource.partitionColumnsSchema(schema, partitionColumns, 
caseSensitive).foreach {
+  field => field.dataType match {
 case _: AtomicType => // OK
 case _ => throw new AnalysisException(s"Cannot use ${field.dataType} 
for partition column")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/2692bdb7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala
index 54beabb..86a306b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala
@@ -99,7 +99,8 @@ object ResolvedDataSource extends Logging {
   val maybePartitionsSchema = if (partitionColumns.isEmpty) {
 None
   } else {
-Some(partitionColumnsSchema(schema, partitionColumns))
+Some(partitionColumnsSchema(
+  schema, partitionColumns, sqlContext.conf.caseSensitiveAnalysis))
   }
 
   val caseInsensitiveOptions = new CaseInsensitiveMap(options)
@@ -172,14 +173,24 @@ object ResolvedDataSource extends Logging {
 
   def partitionColumnsSchema(
   schema: StructType,
-  partitionColumns: Array[String]): StructType = {
+  partitionColumns: Array[String],
+  caseSensitive: Boolean): StructType = {
+val equality = columnNameEquality(caseSensitive)
 StructType(partitionColumns.map { col =>
-  schema.find(_.name == col).getOrElse {
+  schema.find(f => equality(f.name, col)).getOrElse {
 throw new RuntimeException(s"Partition column $col not found in schema 
$schema")
   }
 }).asNullable
   }
 
+  private def columnNameEquality(caseSensitive: Boolean): (String, String) => 
Boolean = {
+if (caseSensitive) {
+  org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution
+} else {
+  org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution
+}
+  }
+
   /** Create a [[ResolvedDataSource]] for saving the content of the given 
DataFrame. */
   def apply(
   sqlContext: SQLContext,
@@ -207,14 +218,18 @@ object ResolvedDataSource extends Logging {
   path.makeQualified(fs.getUri, fs.getWorkingDirectory)
 }
 
-PartitioningUtils.validatePartitionColumnDataTypes(data.schema, 
partitionColumns)
+val caseSensitive = 

spark git commit: [SPARK-10304][SQL] Following up checking valid dir structure for partition discovery

2015-11-04 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 987df4bfc -> de289bf27


[SPARK-10304][SQL] Following up checking valid dir structure for partition 
discovery

This patch follows up #8840.

Author: Liang-Chi Hsieh 

Closes #9459 from viirya/detect_invalid_part_dir_following.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/de289bf2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/de289bf2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/de289bf2

Branch: refs/heads/master
Commit: de289bf279e14e47859b5fbcd70e97b9d0759f14
Parents: 987df4b
Author: Liang-Chi Hsieh 
Authored: Wed Nov 4 10:56:32 2015 -0800
Committer: Yin Huai 
Committed: Wed Nov 4 10:56:32 2015 -0800

--
 .../execution/datasources/PartitioningUtils.scala   | 14 +-
 .../parquet/ParquetPartitionDiscoverySuite.scala| 16 
 2 files changed, 29 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/de289bf2/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
index 16dc236..86bc3a1 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
@@ -81,6 +81,8 @@ private[sql] object PartitioningUtils {
   parsePartition(path, defaultPartitionName, typeInference)
 }.unzip
 
+// We create pairs of (path -> path's partition value) here
+// If the corresponding partition value is None, the pair will be skiped
 val pathsWithPartitionValues = paths.zip(partitionValues).flatMap(x => 
x._2.map(x._1 -> _))
 
 if (pathsWithPartitionValues.isEmpty) {
@@ -89,11 +91,21 @@ private[sql] object PartitioningUtils {
 } else {
   // This dataset is partitioned. We need to check whether all partitions 
have the same
   // partition columns and resolve potential type conflicts.
+
+  // Check if there is conflicting directory structure.
+  // For the paths such as:
+  // var paths = Seq(
+  //   "hdfs://host:9000/invalidPath",
+  //   "hdfs://host:9000/path/a=10/b=20",
+  //   "hdfs://host:9000/path/a=10.5/b=hello")
+  // It will be recognised as conflicting directory structure:
+  //   "hdfs://host:9000/invalidPath"
+  //   "hdfs://host:9000/path"
   val basePaths = optBasePaths.flatMap(x => x)
   assert(
 basePaths.distinct.size == 1,
 "Conflicting directory structures detected. Suspicious paths:\b" +
-  basePaths.mkString("\n\t", "\n\t", "\n\n"))
+  basePaths.distinct.mkString("\n\t", "\n\t", "\n\n"))
 
   val resolvedPartitionValues = resolvePartitions(pathsWithPartitionValues)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/de289bf2/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
index 67b6a37..61cc0da 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
@@ -88,6 +88,22 @@ class ParquetPartitionDiscoverySuite extends QueryTest with 
ParquetTest with Sha
   parsePartitions(paths.map(new Path(_)), defaultPartitionName, true)
 }
 assert(exception.getMessage().contains("Conflicting directory structures 
detected"))
+
+// Invalid
+// Conflicting directory structure:
+// "hdfs://host:9000/tmp/tables/partitionedTable"
+// "hdfs://host:9000/tmp/tables/nonPartitionedTable1"
+// "hdfs://host:9000/tmp/tables/nonPartitionedTable2"
+paths = Seq(
+  "hdfs://host:9000/tmp/tables/partitionedTable",
+  "hdfs://host:9000/tmp/tables/partitionedTable/p=1/",
+  "hdfs://host:9000/tmp/tables/nonPartitionedTable1",
+  "hdfs://host:9000/tmp/tables/nonPartitionedTable2")
+
+exception = intercept[AssertionError] {
+  parsePartitions(paths.map(new Path(_)), defaultPartitionName, true)
+}
+assert(exception.getMessage().contains("Conflicting directory structures 

spark git commit: [SPARK-11371] Make "mean" an alias for "avg" operator

2015-11-02 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 33ae7a35d -> db11ee5e5


[SPARK-11371] Make "mean" an alias for "avg" operator

>From Reynold in the thread 'Exception when using some aggregate operators' 
>(http://search-hadoop.com/m/q3RTt0xFr22nXB4/):

I don't think these are bugs. The SQL standard for average is "avg", not 
"mean". Similarly, a distinct count is supposed to be written as 
"count(distinct col)", not "countDistinct(col)".
We can, however, make "mean" an alias for "avg" to improve compatibility 
between DataFrame and SQL.

Author: tedyu 

Closes #9332 from ted-yu/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/db11ee5e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/db11ee5e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/db11ee5e

Branch: refs/heads/master
Commit: db11ee5e56e5fac59895c772a9a87c5ac86888ef
Parents: 33ae7a3
Author: tedyu 
Authored: Mon Nov 2 13:51:53 2015 -0800
Committer: Yin Huai 
Committed: Mon Nov 2 13:51:53 2015 -0800

--
 .../spark/sql/catalyst/analysis/FunctionRegistry.scala  | 1 +
 .../spark/sql/hive/execution/AggregationQuerySuite.scala| 9 +
 2 files changed, 10 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/db11ee5e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 5f3ec74..24c1a7b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -185,6 +185,7 @@ object FunctionRegistry {
 expression[Last]("last"),
 expression[Last]("last_value"),
 expression[Max]("max"),
+expression[Average]("mean"),
 expression[Min]("min"),
 expression[Stddev]("stddev"),
 expression[StddevPop]("stddev_pop"),

http://git-wip-us.apache.org/repos/asf/spark/blob/db11ee5e/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
index 0cf0e0a..74061db 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
@@ -301,6 +301,15 @@ abstract class AggregationQuerySuite extends QueryTest 
with SQLTestUtils with Te
 checkAnswer(
   sqlContext.sql(
 """
+  |SELECT key, mean(value)
+  |FROM agg1
+  |GROUP BY key
+""".stripMargin),
+  Row(1, 20.0) :: Row(2, -0.5) :: Row(3, null) :: Row(null, 10.0) :: Nil)
+
+checkAnswer(
+  sqlContext.sql(
+"""
   |SELECT avg(value), key
   |FROM agg1
   |GROUP BY key


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



spark git commit: [SPARK-11329][SQL] Support star expansion for structs.

2015-11-02 Thread yhuai
Repository: spark
Updated Branches:
  refs/heads/master 2cef1bb0b -> 9cb5c731d


[SPARK-11329][SQL] Support star expansion for structs.

1. Supporting expanding structs in Projections. i.e.
  "SELECT s.*" where s is a struct type.
  This is fixed by allowing the expand function to handle structs in addition 
to tables.

2. Supporting expanding * inside aggregate functions of structs.
   "SELECT max(struct(col1, structCol.*))"
   This requires recursively expanding the expressions. In this case, it it the 
aggregate
   expression "max(...)" and we need to recursively expand its children inputs.

Author: Nong Li 

Closes #9343 from nongli/spark-11329.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9cb5c731
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9cb5c731
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9cb5c731

Branch: refs/heads/master
Commit: 9cb5c731dadff9539126362827a258d6b65754bb
Parents: 2cef1bb
Author: Nong Li 
Authored: Mon Nov 2 20:32:08 2015 -0800
Committer: Yin Huai 
Committed: Mon Nov 2 20:32:08 2015 -0800

--
 .../apache/spark/sql/catalyst/SqlParser.scala   |   6 +-
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  46 ---
 .../sql/catalyst/analysis/unresolved.scala  |  78 ---
 .../scala/org/apache/spark/sql/Column.scala |   3 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala| 133 +++
 .../org/apache/spark/sql/hive/HiveQl.scala  |   2 +-
 6 files changed, 230 insertions(+), 38 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9cb5c731/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
index 0fef043..d7567e8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -466,9 +466,9 @@ object SqlParser extends AbstractSparkSQLParser with 
DataTypeParser {
 
   protected lazy val baseExpression: Parser[Expression] =
 ( "*" ^^^ UnresolvedStar(None)
-| ident <~ "." ~ "*" ^^ { case tableName => 
UnresolvedStar(Option(tableName)) }
-| primary
-)
+| (ident <~ "."). + <~ "*" ^^ { case target => { 
UnresolvedStar(Option(target)) }
+} | primary
+   )
 
   protected lazy val signedPrimary: Parser[Expression] =
 sign ~ primary ^^ { case s ~ e => if (s == "-") UnaryMinus(e) else e }

http://git-wip-us.apache.org/repos/asf/spark/blob/9cb5c731/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index beabacf..912c967 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -279,6 +279,24 @@ class Analyzer(
* a logical plan node's children.
*/
   object ResolveReferences extends Rule[LogicalPlan] {
+/**
+ * Foreach expression, expands the matching attribute.*'s in `child`'s 
input for the subtree
+ * rooted at each expression.
+ */
+def expandStarExpressions(exprs: Seq[Expression], child: LogicalPlan): 
Seq[Expression] = {
+  exprs.flatMap {
+case s: Star => s.expand(child, resolver)
+case e =>
+  e.transformDown {
+case f1: UnresolvedFunction if containsStar(f1.children) =>
+  f1.copy(children = f1.children.flatMap {
+case s: Star => s.expand(child, resolver)
+case o => o :: Nil
+  })
+  } :: Nil
+  }
+}
+
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
   case p: LogicalPlan if !p.childrenResolved => p
 
@@ -286,44 +304,42 @@ class Analyzer(
   case p @ Project(projectList, child) if containsStar(projectList) =>
 Project(
   projectList.flatMap {
-case s: Star => s.expand(child.output, resolver)
+case s: Star => s.expand(child, resolver)
 case UnresolvedAlias(f @ UnresolvedFunction(_, args, _)) if 
containsStar(args) =>
-  val expandedArgs = args.flatMap {
-case s: Star => s.expand(child.output, resolver)
-case o => o :: Nil
-  }
-  UnresolvedAlias(child = f.copy(children = 

<    1   2   3   4   5   6   7   8   >