spark git commit: [SPARK-5454] More robust handling of self joins

2015-02-11 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 03bf704bf -> a60d2b70a


[SPARK-5454] More robust handling of self joins

Also I fix a bunch of bad output in test cases.

Author: Michael Armbrust 

Closes #4520 from marmbrus/selfJoin and squashes the following commits:

4f4a85c [Michael Armbrust] comments
49c8e26 [Michael Armbrust] fix tests
6fc38de [Michael Armbrust] fix style
55d64b3 [Michael Armbrust] fix dataframe selfjoins


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

Branch: refs/heads/master
Commit: a60d2b70adff3a8fb3bdfac226b1d86fdb443da4
Parents: 03bf704
Author: Michael Armbrust 
Authored: Wed Feb 11 12:31:56 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 11 12:31:56 2015 -0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 26 +---
 .../analysis/MultiInstanceRelation.scala| 21 +---
 .../spark/sql/catalyst/plans/PlanTest.scala |  4 +--
 .../scala/org/apache/spark/sql/SQLContext.scala |  2 ++
 sql/core/src/test/resources/log4j.properties|  3 +++
 .../org/apache/spark/sql/DataFrameSuite.scala   | 10 
 .../spark/sql/catalyst/plans/PlanTest.scala |  4 +--
 7 files changed, 40 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a60d2b70/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 3f0d77a..2d1fa10 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
@@ -53,14 +53,11 @@ class Analyzer(catalog: Catalog,
   val extendedRules: Seq[Rule[LogicalPlan]] = Nil
 
   lazy val batches: Seq[Batch] = Seq(
-Batch("MultiInstanceRelations", Once,
-  NewRelationInstances),
 Batch("Resolution", fixedPoint,
-  ResolveReferences ::
   ResolveRelations ::
+  ResolveReferences ::
   ResolveGroupingAnalytics ::
   ResolveSortReferences ::
-  NewRelationInstances ::
   ImplicitGenerate ::
   ResolveFunctions ::
   GlobalAggregates ::
@@ -285,6 +282,27 @@ class Analyzer(catalog: Catalog,
   }
 )
 
+  // Special handling for cases when self-join introduce duplicate 
expression ids.
+  case j @ Join(left, right, _, _) if 
left.outputSet.intersect(right.outputSet).nonEmpty =>
+val conflictingAttributes = left.outputSet.intersect(right.outputSet)
+
+val (oldRelation, newRelation, attributeRewrites) = right.collect {
+  case oldVersion: MultiInstanceRelation
+  if 
oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty =>
+val newVersion = oldVersion.newInstance()
+val newAttributes = 
AttributeMap(oldVersion.output.zip(newVersion.output))
+(oldVersion, newVersion, newAttributes)
+}.head // Only handle first case found, others will be fixed on the 
next pass.
+
+val newRight = right transformUp {
+  case r if r == oldRelation => newRelation
+  case other => other transformExpressions {
+case a: Attribute => attributeRewrites.get(a).getOrElse(a)
+  }
+}
+
+j.copy(right = newRight)
+
   case q: LogicalPlan =>
 logTrace(s"Attempting to resolve ${q.simpleString}")
 q transformExpressionsUp  {

http://git-wip-us.apache.org/repos/asf/spark/blob/a60d2b70/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
index 4c5fb3f..894c350 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
@@ -26,28 +26,9 @@ import 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
  * produced by distinct operators in a query tree as this breaks the guarantee 
that expression
  * ids, which are used to differentiate attributes, are unique.
  *
- * Before analysis, all operators that include this trait will be asked to 
p

spark git commit: [SPARK-3688][SQL]LogicalPlan can't resolve column correctlly

2015-02-11 Thread marmbrus
-
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-0-c6d02549aec166e16bfc44d5905fa33a 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-0-c6d02549aec166e16bfc44d5905fa33a
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/44b2311d/sql/hive/src/test/resources/golden/test
 ambiguousReferences resolved as hive-1-a8987ff8c7b9ca95bf8b32314694ed1f
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-1-a8987ff8c7b9ca95bf8b32314694ed1f 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-1-a8987ff8c7b9ca95bf8b32314694ed1f
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/44b2311d/sql/hive/src/test/resources/golden/test
 ambiguousReferences resolved as hive-2-26f54240cf5b909086fc34a34d7fdb56
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-2-26f54240cf5b909086fc34a34d7fdb56 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-2-26f54240cf5b909086fc34a34d7fdb56
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/44b2311d/sql/hive/src/test/resources/golden/test
 ambiguousReferences resolved as hive-3-d08d5280027adea681001ad82a5a6974
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-3-d08d5280027adea681001ad82a5a6974 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-3-d08d5280027adea681001ad82a5a6974
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/44b2311d/sql/hive/src/test/resources/golden/test
 ambiguousReferences resolved as hive-4-22eb25b5be6daf72a6649adfe5041749
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-4-22eb25b5be6daf72a6649adfe5041749 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-4-22eb25b5be6daf72a6649adfe5041749
new file mode 100644
index 000..d00491f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-4-22eb25b5be6daf72a6649adfe5041749   
@@ -0,0 +1 @@
+1

http://git-wip-us.apache.org/repos/asf/spark/blob/44b2311d/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index ab5f9cd..029c36a 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -99,6 +99,15 @@ class HiveResolutionSuite extends HiveComparisonTest {
 assert(sql("SELECT nestedArray[0].a FROM 
nestedRepeatedTest").collect().head(0) === 1)
   }
 
+  createQueryTest("test ambiguousReferences resolved as hive",
+"""
+  |CREATE TABLE t1(x INT);
+  |CREATE TABLE t2(a STRUCT, k INT);
+  |INSERT OVERWRITE TABLE t1 SELECT 1 FROM src LIMIT 1;
+  |INSERT OVERWRITE TABLE t2 SELECT named_struct("x",1),1 FROM src LIMIT 1;
+  |SELECT a.x FROM t1 a JOIN t2 b ON a.x = b.k;
+""".stripMargin)
+
   /**
* Negative examples.  Currently only left here for documentation purposes.
* TODO(marmbrus): Test that catalyst fails on these queries.


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



spark git commit: [SPARK-3688][SQL]LogicalPlan can't resolve column correctlly

2015-02-11 Thread marmbrus
rces/golden/test
 ambiguousReferences resolved as hive-0-c6d02549aec166e16bfc44d5905fa33a
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-0-c6d02549aec166e16bfc44d5905fa33a 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-0-c6d02549aec166e16bfc44d5905fa33a
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/e136f477/sql/hive/src/test/resources/golden/test
 ambiguousReferences resolved as hive-1-a8987ff8c7b9ca95bf8b32314694ed1f
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-1-a8987ff8c7b9ca95bf8b32314694ed1f 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-1-a8987ff8c7b9ca95bf8b32314694ed1f
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/e136f477/sql/hive/src/test/resources/golden/test
 ambiguousReferences resolved as hive-2-26f54240cf5b909086fc34a34d7fdb56
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-2-26f54240cf5b909086fc34a34d7fdb56 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-2-26f54240cf5b909086fc34a34d7fdb56
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/e136f477/sql/hive/src/test/resources/golden/test
 ambiguousReferences resolved as hive-3-d08d5280027adea681001ad82a5a6974
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-3-d08d5280027adea681001ad82a5a6974 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-3-d08d5280027adea681001ad82a5a6974
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/e136f477/sql/hive/src/test/resources/golden/test
 ambiguousReferences resolved as hive-4-22eb25b5be6daf72a6649adfe5041749
--
diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences 
resolved as hive-4-22eb25b5be6daf72a6649adfe5041749 
b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-4-22eb25b5be6daf72a6649adfe5041749
new file mode 100644
index 000..d00491f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as 
hive-4-22eb25b5be6daf72a6649adfe5041749   
@@ -0,0 +1 @@
+1

http://git-wip-us.apache.org/repos/asf/spark/blob/e136f477/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index ab5f9cd..029c36a 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -99,6 +99,15 @@ class HiveResolutionSuite extends HiveComparisonTest {
 assert(sql("SELECT nestedArray[0].a FROM 
nestedRepeatedTest").collect().head(0) === 1)
   }
 
+  createQueryTest("test ambiguousReferences resolved as hive",
+"""
+  |CREATE TABLE t1(x INT);
+  |CREATE TABLE t2(a STRUCT, k INT);
+  |INSERT OVERWRITE TABLE t1 SELECT 1 FROM src LIMIT 1;
+  |INSERT OVERWRITE TABLE t2 SELECT named_struct("x",1),1 FROM src LIMIT 1;
+  |SELECT a.x FROM t1 a JOIN t2 b ON a.x = b.k;
+""".stripMargin)
+
   /**
* Negative examples.  Currently only left here for documentation purposes.
* TODO(marmbrus): Test that catalyst fails on these queries.


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



spark git commit: [SQL] Make dataframe more tolerant of being serialized

2015-02-11 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master d931b01dc -> a38e23c30


[SQL] Make dataframe more tolerant of being serialized

Eases use in the spark-shell.

Author: Michael Armbrust 

Closes #4545 from marmbrus/serialization and squashes the following commits:

04748e6 [Michael Armbrust] @scala.annotation.varargs
b36e219 [Michael Armbrust] moreFixes


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

Branch: refs/heads/master
Commit: a38e23c30fb5d12f8f46a119d91a0620036e6800
Parents: d931b01
Author: Michael Armbrust 
Authored: Wed Feb 11 19:05:49 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 11 19:05:49 2015 -0800

--
 .../src/main/scala/org/apache/spark/sql/DataFrame.scala | 2 +-
 .../src/main/scala/org/apache/spark/sql/DataFrameImpl.scala | 4 ++--
 sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala  | 9 +
 .../src/main/scala/org/apache/spark/sql/SQLContext.scala| 4 +++-
 4 files changed, 15 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a38e23c3/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 327cf87..13aff76 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
@@ -76,7 +76,7 @@ private[sql] object DataFrame {
  */
 // TODO: Improve documentation.
 @Experimental
-trait DataFrame extends RDDApi[Row] {
+trait DataFrame extends RDDApi[Row] with Serializable {
 
   val sqlContext: SQLContext
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a38e23c3/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
index 3863df5..4c6e19c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
@@ -44,8 +44,8 @@ import org.apache.spark.sql.types.{NumericType, StructType}
  * Internal implementation of [[DataFrame]]. Users of the API should use 
[[DataFrame]] directly.
  */
 private[sql] class DataFrameImpl protected[sql](
-override val sqlContext: SQLContext,
-val queryExecution: SQLContext#QueryExecution)
+@transient override val sqlContext: SQLContext,
+@transient val queryExecution: SQLContext#QueryExecution)
   extends DataFrame {
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/a38e23c3/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
index 6bf21dd..7bc7683 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
@@ -167,6 +167,15 @@ object Dsl {
   
//
 
   /**
+   * Returns the first column that is not null.
+   * {{{
+   *   df.select(coalesce(df("a"), df("b")))
+   * }}}
+   */
+  @scala.annotation.varargs
+  def coalesce(e: Column*): Column = Coalesce(e.map(_.expr))
+
+  /**
* Unary minus, i.e. negate the expression.
* {{{
*   // Select the amount column and negates all values.

http://git-wip-us.apache.org/repos/asf/spark/blob/a38e23c3/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index ca5e62f..8aae222 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -129,9 +129,11 @@ class SQLContext(@transient val sparkContext: SparkContext)
* A collection of methods that are considered experimental, but can be used 
to hook into
* the query planner for advanced functionalities.
*/
+  @transient
   val experimental: ExperimentalMethods = new ExperimentalMethods(this)
 
   /** Returns a [[DataFrame]] with no rows or columns. */
+  @transient
   lazy val emptyDataFrame = DataFrame(this, NoRelation)
 
   /**
@@ -178,7 +18

spark git commit: [SQL] Make dataframe more tolerant of being serialized

2015-02-11 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 bcb13827c -> 3c1b9bf65


[SQL] Make dataframe more tolerant of being serialized

Eases use in the spark-shell.

Author: Michael Armbrust 

Closes #4545 from marmbrus/serialization and squashes the following commits:

04748e6 [Michael Armbrust] @scala.annotation.varargs
b36e219 [Michael Armbrust] moreFixes

(cherry picked from commit a38e23c30fb5d12f8f46a119d91a0620036e6800)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 3c1b9bf65290cc1fd690a5c5c252667e4576
Parents: bcb1382
Author: Michael Armbrust 
Authored: Wed Feb 11 19:05:49 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 11 19:06:05 2015 -0800

--
 .../src/main/scala/org/apache/spark/sql/DataFrame.scala | 2 +-
 .../src/main/scala/org/apache/spark/sql/DataFrameImpl.scala | 4 ++--
 sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala  | 9 +
 .../src/main/scala/org/apache/spark/sql/SQLContext.scala| 4 +++-
 4 files changed, 15 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3c1b9bf6/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 327cf87..13aff76 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
@@ -76,7 +76,7 @@ private[sql] object DataFrame {
  */
 // TODO: Improve documentation.
 @Experimental
-trait DataFrame extends RDDApi[Row] {
+trait DataFrame extends RDDApi[Row] with Serializable {
 
   val sqlContext: SQLContext
 

http://git-wip-us.apache.org/repos/asf/spark/blob/3c1b9bf6/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
index 3863df5..4c6e19c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
@@ -44,8 +44,8 @@ import org.apache.spark.sql.types.{NumericType, StructType}
  * Internal implementation of [[DataFrame]]. Users of the API should use 
[[DataFrame]] directly.
  */
 private[sql] class DataFrameImpl protected[sql](
-override val sqlContext: SQLContext,
-val queryExecution: SQLContext#QueryExecution)
+@transient override val sqlContext: SQLContext,
+@transient val queryExecution: SQLContext#QueryExecution)
   extends DataFrame {
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/3c1b9bf6/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
index 6bf21dd..7bc7683 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala
@@ -167,6 +167,15 @@ object Dsl {
   
//
 
   /**
+   * Returns the first column that is not null.
+   * {{{
+   *   df.select(coalesce(df("a"), df("b")))
+   * }}}
+   */
+  @scala.annotation.varargs
+  def coalesce(e: Column*): Column = Coalesce(e.map(_.expr))
+
+  /**
* Unary minus, i.e. negate the expression.
* {{{
*   // Select the amount column and negates all values.

http://git-wip-us.apache.org/repos/asf/spark/blob/3c1b9bf6/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index ca5e62f..8aae222 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -129,9 +129,11 @@ class SQLContext(@transient val sparkContext: SparkContext)
* A collection of methods that are considered experimental, but can be used 
to hook into
* the query planner for advanced functionalities.
*/
+  @transient
   val experimental: ExperimentalMethods = new ExperimentalMethods(this)
 
   /** Returns a [[DataFrame]] 

spark git commit: [SQL] Improve error messages

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 6a1be026c -> aa4ca8b87


[SQL] Improve error messages

Author: Michael Armbrust 
Author: wangfei 

Closes #4558 from marmbrus/errorMessages and squashes the following commits:

5e5ab50 [Michael Armbrust] Merge pull request #15 from scwf/errorMessages
fa38881 [wangfei] fix for grouping__id
f279a71 [wangfei] make right references for ScriptTransformation
d29fbde [Michael Armbrust] extra case
1a797b4 [Michael Armbrust] comments
d4e9015 [Michael Armbrust] add comment
af9e668 [Michael Armbrust] no braces
34eb3a4 [Michael Armbrust] more work
6197cd5 [Michael Armbrust] [SQL] Better error messages for analysis failures


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

Branch: refs/heads/master
Commit: aa4ca8b873fd83e64e5faea6f7febcc830e30b02
Parents: 6a1be02
Author: Michael Armbrust 
Authored: Thu Feb 12 13:11:28 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 13:11:28 2015 -0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 123 +++
 .../sql/catalyst/expressions/AttributeSet.scala |   6 +-
 .../catalyst/expressions/namedExpressions.scala |   2 +-
 .../plans/logical/ScriptTransformation.scala|   6 +-
 .../spark/sql/catalyst/trees/TreeNode.scala |   9 ++
 .../sql/catalyst/analysis/AnalysisSuite.scala   |  79 
 .../optimizer/BooleanSimplificationSuite.scala  |   4 +-
 .../optimizer/ConstantFoldingSuite.scala|   4 +-
 .../optimizer/FilterPushdownSuite.scala |  12 +-
 .../catalyst/optimizer/OptimizeInSuite.scala|   4 +-
 .../catalyst/optimizer/UnionPushdownSuite.scala |   4 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala|   6 +-
 .../org/apache/spark/sql/hive/HiveContext.scala |   4 +-
 .../spark/sql/hive/execution/commands.scala |   4 +-
 14 files changed, 164 insertions(+), 103 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/aa4ca8b8/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 2d1fa10..58a7003 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
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.types.{ArrayType, StructField, StructType, 
IntegerType}
+import org.apache.spark.sql.types._
 
 /**
  * A trivial [[Analyzer]] with an [[EmptyCatalog]] and 
[[EmptyFunctionRegistry]]. Used for testing
@@ -66,32 +66,82 @@ class Analyzer(catalog: Catalog,
   typeCoercionRules ++
   extendedRules : _*),
 Batch("Check Analysis", Once,
-  CheckResolution ::
-  CheckAggregation ::
-  Nil: _*),
-Batch("AnalysisOperators", fixedPoint,
-  EliminateAnalysisOperators)
+  CheckResolution),
+Batch("Remove SubQueries", fixedPoint,
+  EliminateSubQueries)
   )
 
   /**
* Makes sure all attributes and logical plans have been resolved.
*/
   object CheckResolution extends Rule[LogicalPlan] {
+def failAnalysis(msg: String) = { throw new AnalysisException(msg) }
+
 def apply(plan: LogicalPlan): LogicalPlan = {
-  plan.transformUp {
-case p if p.expressions.exists(!_.resolved) =>
-  val missing = 
p.expressions.filterNot(_.resolved).map(_.prettyString).mkString(",")
-  val from = p.inputSet.map(_.name).mkString("{", ", ", "}")
-
-  throw new AnalysisException(s"Cannot resolve '$missing' given input 
columns $from")
-case p if !p.resolved && p.childrenResolved =>
-  throw new AnalysisException(s"Unresolved operator in the query plan 
${p.simpleString}")
-  } match {
-// As a backstop, use the root node to check that the entire plan tree 
is resolved.
-case p if !p.resolved =>
-  throw new AnalysisException(s"Unresolved operator in the query plan 
${p.simpleString}")
-case p => p
+  // We transform up and order the rules so as to catch the first possible 
failure instead
+  // of the result of cascading resolution failures.
+  p

spark git commit: [SQL] Improve error messages

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 cbd659e5f -> e3a975d45


[SQL] Improve error messages

Author: Michael Armbrust 
Author: wangfei 

Closes #4558 from marmbrus/errorMessages and squashes the following commits:

5e5ab50 [Michael Armbrust] Merge pull request #15 from scwf/errorMessages
fa38881 [wangfei] fix for grouping__id
f279a71 [wangfei] make right references for ScriptTransformation
d29fbde [Michael Armbrust] extra case
1a797b4 [Michael Armbrust] comments
d4e9015 [Michael Armbrust] add comment
af9e668 [Michael Armbrust] no braces
34eb3a4 [Michael Armbrust] more work
6197cd5 [Michael Armbrust] [SQL] Better error messages for analysis failures

(cherry picked from commit aa4ca8b873fd83e64e5faea6f7febcc830e30b02)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: e3a975d45a960ddbfe03051a6ae8b614e63cde6b
Parents: cbd659e
Author: Michael Armbrust 
Authored: Thu Feb 12 13:11:28 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 13:11:39 2015 -0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 123 +++
 .../sql/catalyst/expressions/AttributeSet.scala |   6 +-
 .../catalyst/expressions/namedExpressions.scala |   2 +-
 .../plans/logical/ScriptTransformation.scala|   6 +-
 .../spark/sql/catalyst/trees/TreeNode.scala |   9 ++
 .../sql/catalyst/analysis/AnalysisSuite.scala   |  79 
 .../optimizer/BooleanSimplificationSuite.scala  |   4 +-
 .../optimizer/ConstantFoldingSuite.scala|   4 +-
 .../optimizer/FilterPushdownSuite.scala |  12 +-
 .../catalyst/optimizer/OptimizeInSuite.scala|   4 +-
 .../catalyst/optimizer/UnionPushdownSuite.scala |   4 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala|   6 +-
 .../org/apache/spark/sql/hive/HiveContext.scala |   4 +-
 .../spark/sql/hive/execution/commands.scala |   4 +-
 14 files changed, 164 insertions(+), 103 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e3a975d4/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 2d1fa10..58a7003 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
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.types.{ArrayType, StructField, StructType, 
IntegerType}
+import org.apache.spark.sql.types._
 
 /**
  * A trivial [[Analyzer]] with an [[EmptyCatalog]] and 
[[EmptyFunctionRegistry]]. Used for testing
@@ -66,32 +66,82 @@ class Analyzer(catalog: Catalog,
   typeCoercionRules ++
   extendedRules : _*),
 Batch("Check Analysis", Once,
-  CheckResolution ::
-  CheckAggregation ::
-  Nil: _*),
-Batch("AnalysisOperators", fixedPoint,
-  EliminateAnalysisOperators)
+  CheckResolution),
+Batch("Remove SubQueries", fixedPoint,
+  EliminateSubQueries)
   )
 
   /**
* Makes sure all attributes and logical plans have been resolved.
*/
   object CheckResolution extends Rule[LogicalPlan] {
+def failAnalysis(msg: String) = { throw new AnalysisException(msg) }
+
 def apply(plan: LogicalPlan): LogicalPlan = {
-  plan.transformUp {
-case p if p.expressions.exists(!_.resolved) =>
-  val missing = 
p.expressions.filterNot(_.resolved).map(_.prettyString).mkString(",")
-  val from = p.inputSet.map(_.name).mkString("{", ", ", "}")
-
-  throw new AnalysisException(s"Cannot resolve '$missing' given input 
columns $from")
-case p if !p.resolved && p.childrenResolved =>
-  throw new AnalysisException(s"Unresolved operator in the query plan 
${p.simpleString}")
-  } match {
-// As a backstop, use the root node to check that the entire plan tree 
is resolved.
-case p if !p.resolved =>
-  throw new AnalysisException(s"Unresolved operator in the query plan 
${p.simpleString}")
-case p => p
+  // We transform up and order the rul

spark git commit: [SPARK-5758][SQL] Use LongType as the default type for integers in JSON schema inference.

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 0bf031582 -> c352ffbdb


[SPARK-5758][SQL] Use LongType as the default type for integers in JSON schema 
inference.

Author: Yin Huai 

Closes #4544 from yhuai/jsonUseLongTypeByDefault and squashes the following 
commits:

6e2ffc2 [Yin Huai] Use LongType as the default type for integers in JSON schema 
inference.


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

Branch: refs/heads/master
Commit: c352ffbdb9112714c176a747edff6115e9369e58
Parents: 0bf0315
Author: Yin Huai 
Authored: Thu Feb 12 15:17:25 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 15:17:25 2015 -0800

--
 .../scala/org/apache/spark/sql/json/JsonRDD.scala   | 12 
 .../spark/sql/api/java/JavaApplySchemaSuite.java|  2 +-
 .../scala/org/apache/spark/sql/json/JsonSuite.scala | 16 
 3 files changed, 17 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c352ffbd/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 1043eef..7dfb304 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -17,13 +17,12 @@
 
 package org.apache.spark.sql.json
 
-import java.io.StringWriter
-import java.sql.{Date, Timestamp}
+import java.sql.Timestamp
 
 import scala.collection.Map
 import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper}
 
-import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException, 
JsonFactory}
+import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException}
 import com.fasterxml.jackson.databind.ObjectMapper
 
 import org.apache.spark.rdd.RDD
@@ -178,7 +177,12 @@ private[sql] object JsonRDD extends Logging {
   }
 
   private def typeOfPrimitiveValue: PartialFunction[Any, DataType] = {
-ScalaReflection.typeOfObject orElse {
+// For Integer values, use LongType by default.
+val useLongType: PartialFunction[Any, DataType] = {
+  case value: IntegerType.JvmType => LongType
+}
+
+useLongType orElse ScalaReflection.typeOfObject orElse {
   // Since we do not have a data type backed by BigInteger,
   // when we see a Java BigInteger, we use DecimalType.
   case value: java.math.BigInteger => DecimalType.Unlimited

http://git-wip-us.apache.org/repos/asf/spark/blob/c352ffbd/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
--
diff --git 
a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
 
b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
index 2e6e977..643b891 100644
--- 
a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
+++ 
b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
@@ -164,7 +164,7 @@ public class JavaApplySchemaSuite implements Serializable {
 fields.add(DataTypes.createStructField("bigInteger", 
DataTypes.createDecimalType(), true));
 fields.add(DataTypes.createStructField("boolean", DataTypes.BooleanType, 
true));
 fields.add(DataTypes.createStructField("double", DataTypes.DoubleType, 
true));
-fields.add(DataTypes.createStructField("integer", DataTypes.IntegerType, 
true));
+fields.add(DataTypes.createStructField("integer", DataTypes.LongType, 
true));
 fields.add(DataTypes.createStructField("long", DataTypes.LongType, true));
 fields.add(DataTypes.createStructField("null", DataTypes.StringType, 
true));
 fields.add(DataTypes.createStructField("string", DataTypes.StringType, 
true));

http://git-wip-us.apache.org/repos/asf/spark/blob/c352ffbd/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
index fde4b47..b5f13f8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
@@ -222,7 +222,7 @@ class JsonSuite extends QueryTest {
   StructField("bigInteger", DecimalType.Unlimited, true) ::
   StructField("boolean", BooleanType, true) ::
   StructField("double", DoubleType, true) ::
-  StructFie

spark git commit: [SPARK-5758][SQL] Use LongType as the default type for integers in JSON schema inference.

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 bf0d15c52 -> b0c79daf4


[SPARK-5758][SQL] Use LongType as the default type for integers in JSON schema 
inference.

Author: Yin Huai 

Closes #4544 from yhuai/jsonUseLongTypeByDefault and squashes the following 
commits:

6e2ffc2 [Yin Huai] Use LongType as the default type for integers in JSON schema 
inference.

(cherry picked from commit c352ffbdb9112714c176a747edff6115e9369e58)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: b0c79daf4a24739963726dfecedff9a4b129f3c0
Parents: bf0d15c
Author: Yin Huai 
Authored: Thu Feb 12 15:17:25 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 15:17:35 2015 -0800

--
 .../scala/org/apache/spark/sql/json/JsonRDD.scala   | 12 
 .../spark/sql/api/java/JavaApplySchemaSuite.java|  2 +-
 .../scala/org/apache/spark/sql/json/JsonSuite.scala | 16 
 3 files changed, 17 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b0c79daf/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 1043eef..7dfb304 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -17,13 +17,12 @@
 
 package org.apache.spark.sql.json
 
-import java.io.StringWriter
-import java.sql.{Date, Timestamp}
+import java.sql.Timestamp
 
 import scala.collection.Map
 import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper}
 
-import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException, 
JsonFactory}
+import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException}
 import com.fasterxml.jackson.databind.ObjectMapper
 
 import org.apache.spark.rdd.RDD
@@ -178,7 +177,12 @@ private[sql] object JsonRDD extends Logging {
   }
 
   private def typeOfPrimitiveValue: PartialFunction[Any, DataType] = {
-ScalaReflection.typeOfObject orElse {
+// For Integer values, use LongType by default.
+val useLongType: PartialFunction[Any, DataType] = {
+  case value: IntegerType.JvmType => LongType
+}
+
+useLongType orElse ScalaReflection.typeOfObject orElse {
   // Since we do not have a data type backed by BigInteger,
   // when we see a Java BigInteger, we use DecimalType.
   case value: java.math.BigInteger => DecimalType.Unlimited

http://git-wip-us.apache.org/repos/asf/spark/blob/b0c79daf/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
--
diff --git 
a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
 
b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
index 2e6e977..643b891 100644
--- 
a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
+++ 
b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java
@@ -164,7 +164,7 @@ public class JavaApplySchemaSuite implements Serializable {
 fields.add(DataTypes.createStructField("bigInteger", 
DataTypes.createDecimalType(), true));
 fields.add(DataTypes.createStructField("boolean", DataTypes.BooleanType, 
true));
 fields.add(DataTypes.createStructField("double", DataTypes.DoubleType, 
true));
-fields.add(DataTypes.createStructField("integer", DataTypes.IntegerType, 
true));
+fields.add(DataTypes.createStructField("integer", DataTypes.LongType, 
true));
 fields.add(DataTypes.createStructField("long", DataTypes.LongType, true));
 fields.add(DataTypes.createStructField("null", DataTypes.StringType, 
true));
 fields.add(DataTypes.createStructField("string", DataTypes.StringType, 
true));

http://git-wip-us.apache.org/repos/asf/spark/blob/b0c79daf/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
index fde4b47..b5f13f8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
@@ -222,7 +222,7 @@ class JsonSuite extends QueryTest {
   StructField("bigInteger", DecimalType.Unlimited, true) ::
   

spark git commit: [SPARK-5573][SQL] Add explode to dataframes

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 b0c79daf4 -> c7eb9ee2c


[SPARK-5573][SQL] Add explode to dataframes

Author: Michael Armbrust 

Closes #4546 from marmbrus/explode and squashes the following commits:

eefd33a [Michael Armbrust] whitespace
a8d496c [Michael Armbrust] Merge remote-tracking branch 'apache/master' into 
explode
4af740e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into 
explode
dc86a5c [Michael Armbrust] simple version
d633d01 [Michael Armbrust] add scala specific
950707a [Michael Armbrust] fix comments
ba8854c [Michael Armbrust] [SPARK-5573][SQL] Add explode to dataframes

(cherry picked from commit ee04a8b19be8330bfc48f470ef365622162c915f)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: c7eb9ee2ccd93211c9ec125fd2baae267b35d3d4
Parents: b0c79da
Author: Michael Armbrust 
Authored: Thu Feb 12 15:19:19 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 15:19:32 2015 -0800

--
 .../sql/catalyst/expressions/generators.scala   | 19 ++
 .../scala/org/apache/spark/sql/DataFrame.scala  | 38 
 .../org/apache/spark/sql/DataFrameImpl.scala| 30 ++--
 .../apache/spark/sql/IncomputableColumn.scala   |  9 +
 .../org/apache/spark/sql/DataFrameSuite.scala   | 25 +
 5 files changed, 119 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c7eb9ee2/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 43b6482..0983d27 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
@@ -74,6 +74,25 @@ abstract class Generator extends Expression {
 }
 
 /**
+ * A generator that produces its output using the provided lambda function.
+ */
+case class UserDefinedGenerator(
+schema: Seq[Attribute],
+function: Row => TraversableOnce[Row],
+children: Seq[Expression])
+  extends Generator{
+
+  override protected def makeOutput(): Seq[Attribute] = schema
+
+  override def eval(input: Row): TraversableOnce[Row] = {
+val inputRow = new InterpretedProjection(children)
+function(inputRow(input))
+  }
+
+  override def toString = s"UserDefinedGenerator(${children.mkString(",")})"
+}
+
+/**
  * Given an input array produces a sequence of rows for each value in the 
array.
  */
 case class Explode(attributeNames: Seq[String], child: Expression)

http://git-wip-us.apache.org/repos/asf/spark/blob/c7eb9ee2/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 13aff76..6525788 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql
 
 import scala.collection.JavaConversions._
 import scala.reflect.ClassTag
+import scala.reflect.runtime.universe.TypeTag
 import scala.util.control.NonFatal
 
 import org.apache.spark.annotation.{DeveloperApi, Experimental}
@@ -441,6 +442,43 @@ trait DataFrame extends RDDApi[Row] with Serializable {
 sample(withReplacement, fraction, Utils.random.nextLong)
   }
 
+  /**
+   * (Scala-specific) Returns a new [[DataFrame]] where each row has been 
expanded to zero or more
+   * rows by the provided function.  This is similar to a `LATERAL VIEW` in 
HiveQL. The columns of
+   * the input row are implicitly joined with each row that is output by the 
function.
+   *
+   * The following example uses this function to count the number of books 
which contain
+   * a given word:
+   *
+   * {{{
+   *   case class Book(title: String, words: String)
+   *   val df: RDD[Book]
+   *
+   *   case class Word(word: String)
+   *   val allWords = df.explode('words) {
+   * case Row(words: String) => words.split(" ").map(Word(_))
+   *   }
+   *
+   *   val bookCountPerWord = 
allWords.groupBy("word").agg(countDistinct("title"))
+   * }}}
+   */
+  def explo

spark git commit: [SPARK-5573][SQL] Add explode to dataframes

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master c352ffbdb -> ee04a8b19


[SPARK-5573][SQL] Add explode to dataframes

Author: Michael Armbrust 

Closes #4546 from marmbrus/explode and squashes the following commits:

eefd33a [Michael Armbrust] whitespace
a8d496c [Michael Armbrust] Merge remote-tracking branch 'apache/master' into 
explode
4af740e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into 
explode
dc86a5c [Michael Armbrust] simple version
d633d01 [Michael Armbrust] add scala specific
950707a [Michael Armbrust] fix comments
ba8854c [Michael Armbrust] [SPARK-5573][SQL] Add explode to dataframes


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

Branch: refs/heads/master
Commit: ee04a8b19be8330bfc48f470ef365622162c915f
Parents: c352ffb
Author: Michael Armbrust 
Authored: Thu Feb 12 15:19:19 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 15:19:19 2015 -0800

--
 .../sql/catalyst/expressions/generators.scala   | 19 ++
 .../scala/org/apache/spark/sql/DataFrame.scala  | 38 
 .../org/apache/spark/sql/DataFrameImpl.scala| 30 ++--
 .../apache/spark/sql/IncomputableColumn.scala   |  9 +
 .../org/apache/spark/sql/DataFrameSuite.scala   | 25 +
 5 files changed, 119 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ee04a8b1/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 43b6482..0983d27 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
@@ -74,6 +74,25 @@ abstract class Generator extends Expression {
 }
 
 /**
+ * A generator that produces its output using the provided lambda function.
+ */
+case class UserDefinedGenerator(
+schema: Seq[Attribute],
+function: Row => TraversableOnce[Row],
+children: Seq[Expression])
+  extends Generator{
+
+  override protected def makeOutput(): Seq[Attribute] = schema
+
+  override def eval(input: Row): TraversableOnce[Row] = {
+val inputRow = new InterpretedProjection(children)
+function(inputRow(input))
+  }
+
+  override def toString = s"UserDefinedGenerator(${children.mkString(",")})"
+}
+
+/**
  * Given an input array produces a sequence of rows for each value in the 
array.
  */
 case class Explode(attributeNames: Seq[String], child: Expression)

http://git-wip-us.apache.org/repos/asf/spark/blob/ee04a8b1/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 13aff76..6525788 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql
 
 import scala.collection.JavaConversions._
 import scala.reflect.ClassTag
+import scala.reflect.runtime.universe.TypeTag
 import scala.util.control.NonFatal
 
 import org.apache.spark.annotation.{DeveloperApi, Experimental}
@@ -441,6 +442,43 @@ trait DataFrame extends RDDApi[Row] with Serializable {
 sample(withReplacement, fraction, Utils.random.nextLong)
   }
 
+  /**
+   * (Scala-specific) Returns a new [[DataFrame]] where each row has been 
expanded to zero or more
+   * rows by the provided function.  This is similar to a `LATERAL VIEW` in 
HiveQL. The columns of
+   * the input row are implicitly joined with each row that is output by the 
function.
+   *
+   * The following example uses this function to count the number of books 
which contain
+   * a given word:
+   *
+   * {{{
+   *   case class Book(title: String, words: String)
+   *   val df: RDD[Book]
+   *
+   *   case class Word(word: String)
+   *   val allWords = df.explode('words) {
+   * case Row(words: String) => words.split(" ").map(Word(_))
+   *   }
+   *
+   *   val bookCountPerWord = 
allWords.groupBy("word").agg(countDistinct("title"))
+   * }}}
+   */
+  def explode[A <: Product : TypeTag](input: Column*)(f: Row => 
TraversableOnce[A]): DataFrame
+
+
+  /**
+   * (Scala-specific) Returns 

spark git commit: [SPARK-5755] [SQL] remove unnecessary Add

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master ee04a8b19 -> d5fc51491


[SPARK-5755] [SQL] remove unnecessary Add

explain extended select +key from src;
before:
== Parsed Logical Plan ==
'Project [(0 + 'key) AS _c0#8]
 'UnresolvedRelation [src], None

== Analyzed Logical Plan ==
Project [(0 + key#10) AS _c0#8]
 MetastoreRelation test, src, None

== Optimized Logical Plan ==
Project [(0 + key#10) AS _c0#8]
 MetastoreRelation test, src, None

== Physical Plan ==
Project [(0 + key#10) AS _c0#8]
 HiveTableScan [key#10], (MetastoreRelation test, src, None), None

after this patch:
== Parsed Logical Plan ==
'Project ['key]
 'UnresolvedRelation [src], None

== Analyzed Logical Plan ==
Project [key#10]
 MetastoreRelation test, src, None

== Optimized Logical Plan ==
Project [key#10]
 MetastoreRelation test, src, None

== Physical Plan ==
HiveTableScan [key#10], (MetastoreRelation test, src, None), None

Author: Daoyuan Wang 

Closes #4551 from adrian-wang/positive and squashes the following commits:

0821ae4 [Daoyuan Wang] remove unnecessary Add


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

Branch: refs/heads/master
Commit: d5fc51491808630d0328a5937dbf349e00de361f
Parents: ee04a8b
Author: Daoyuan Wang 
Authored: Thu Feb 12 15:22:07 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 15:22:07 2015 -0800

--
 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d5fc5149/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
--
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index f3c9e63..5269460 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -1099,7 +1099,7 @@ 
https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
   Cast(nodeToExpr(arg), DateType)
 
 /* Arithmetic */
-case Token("+", child :: Nil) => Add(Literal(0), nodeToExpr(child))
+case Token("+", child :: Nil) => nodeToExpr(child)
 case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child))
 case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child))
 case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), 
nodeToExpr(right))


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



spark git commit: [SPARK-5755] [SQL] remove unnecessary Add

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 c7eb9ee2c -> f7103b343


[SPARK-5755] [SQL] remove unnecessary Add

explain extended select +key from src;
before:
== Parsed Logical Plan ==
'Project [(0 + 'key) AS _c0#8]
 'UnresolvedRelation [src], None

== Analyzed Logical Plan ==
Project [(0 + key#10) AS _c0#8]
 MetastoreRelation test, src, None

== Optimized Logical Plan ==
Project [(0 + key#10) AS _c0#8]
 MetastoreRelation test, src, None

== Physical Plan ==
Project [(0 + key#10) AS _c0#8]
 HiveTableScan [key#10], (MetastoreRelation test, src, None), None

after this patch:
== Parsed Logical Plan ==
'Project ['key]
 'UnresolvedRelation [src], None

== Analyzed Logical Plan ==
Project [key#10]
 MetastoreRelation test, src, None

== Optimized Logical Plan ==
Project [key#10]
 MetastoreRelation test, src, None

== Physical Plan ==
HiveTableScan [key#10], (MetastoreRelation test, src, None), None

Author: Daoyuan Wang 

Closes #4551 from adrian-wang/positive and squashes the following commits:

0821ae4 [Daoyuan Wang] remove unnecessary Add

(cherry picked from commit d5fc51491808630d0328a5937dbf349e00de361f)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: f7103b3437363bd81e4f4cfa282229019fcdcdad
Parents: c7eb9ee
Author: Daoyuan Wang 
Authored: Thu Feb 12 15:22:07 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 15:22:16 2015 -0800

--
 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f7103b34/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
--
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index f3c9e63..5269460 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -1099,7 +1099,7 @@ 
https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
   Cast(nodeToExpr(arg), DateType)
 
 /* Arithmetic */
-case Token("+", child :: Nil) => Add(Literal(0), nodeToExpr(child))
+case Token("+", child :: Nil) => nodeToExpr(child)
 case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child))
 case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child))
 case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), 
nodeToExpr(right))


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



spark git commit: [SQL] Move SaveMode to SQL package.

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 5c9db4e75 -> 925fd84a1


[SQL] Move SaveMode to SQL package.

Author: Yin Huai 

Closes #4542 from yhuai/moveSaveMode and squashes the following commits:

65a4425 [Yin Huai] Move SaveMode to sql package.

(cherry picked from commit c025a468826e9b9f62032e207daa9d42d9dba3ca)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 925fd84a1d25fd453af8ad457569dddb54938388
Parents: 5c9db4e
Author: Yin Huai 
Authored: Thu Feb 12 15:32:17 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 15:32:25 2015 -0800

--
 python/pyspark/sql/dataframe.py |  2 +-
 .../java/org/apache/spark/sql/SaveMode.java | 45 
 .../org/apache/spark/sql/sources/SaveMode.java  | 45 
 .../scala/org/apache/spark/sql/DataFrame.scala  |  1 -
 .../apache/spark/sql/IncomputableColumn.scala   |  1 -
 .../apache/spark/sql/json/JSONRelation.scala|  2 +-
 .../apache/spark/sql/parquet/newParquet.scala   |  3 +-
 .../org/apache/spark/sql/sources/ddl.scala  |  2 +-
 .../apache/spark/sql/sources/interfaces.scala   |  2 +-
 .../spark/sql/sources/SaveLoadSuite.scala   |  2 +-
 .../spark/sql/hive/execution/commands.scala |  2 +-
 .../sql/hive/JavaMetastoreDataSourcesSuite.java |  2 +-
 .../sql/hive/MetastoreDataSourcesSuite.scala|  2 -
 13 files changed, 53 insertions(+), 58 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/925fd84a/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 3eb56ed..b6f052e 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -149,7 +149,7 @@ class DataFrame(object):
 def _java_save_mode(self, mode):
 """Returns the Java save mode based on the Python save mode 
represented by a string.
 """
-jSaveMode = self._sc._jvm.org.apache.spark.sql.sources.SaveMode
+jSaveMode = self._sc._jvm.org.apache.spark.sql.SaveMode
 jmode = jSaveMode.ErrorIfExists
 mode = mode.lower()
 if mode == "append":

http://git-wip-us.apache.org/repos/asf/spark/blob/925fd84a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java
--
diff --git a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java 
b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java
new file mode 100644
index 000..a40be52
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java
@@ -0,0 +1,45 @@
+/*
+ * 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 License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql;
+
+/**
+ * SaveMode is used to specify the expected behavior of saving a DataFrame to 
a data source.
+ */
+public enum SaveMode {
+  /**
+   * Append mode means that when saving a DataFrame to a data source, if 
data/table already exists,
+   * contents of the DataFrame are expected to be appended to existing data.
+   */
+  Append,
+  /**
+   * Overwrite mode means that when saving a DataFrame to a data source,
+   * if data/table already exists, existing data is expected to be overwritten 
by the contents of
+   * the DataFrame.
+   */
+  Overwrite,
+  /**
+   * ErrorIfExists mode means that when saving a DataFrame to a data source, 
if data already exists,
+   * an exception is expected to be thrown.
+   */
+  ErrorIfExists,
+  /**
+   * Ignore mode means that when saving a DataFrame to a data source, if data 
already exists,
+   * the save operation is expected to not save the contents of the DataFrame 
and to not
+   * change the existing data.
+   */
+  Ignore
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/925fd84a/sql/core/src/main/java/org/apache/spark/sql/

spark git commit: [SQL] Move SaveMode to SQL package.

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master ada993e95 -> c025a4688


[SQL] Move SaveMode to SQL package.

Author: Yin Huai 

Closes #4542 from yhuai/moveSaveMode and squashes the following commits:

65a4425 [Yin Huai] Move SaveMode to sql package.


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

Branch: refs/heads/master
Commit: c025a468826e9b9f62032e207daa9d42d9dba3ca
Parents: ada993e
Author: Yin Huai 
Authored: Thu Feb 12 15:32:17 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 15:32:17 2015 -0800

--
 python/pyspark/sql/dataframe.py |  2 +-
 .../java/org/apache/spark/sql/SaveMode.java | 45 
 .../org/apache/spark/sql/sources/SaveMode.java  | 45 
 .../scala/org/apache/spark/sql/DataFrame.scala  |  1 -
 .../apache/spark/sql/IncomputableColumn.scala   |  1 -
 .../apache/spark/sql/json/JSONRelation.scala|  2 +-
 .../apache/spark/sql/parquet/newParquet.scala   |  3 +-
 .../org/apache/spark/sql/sources/ddl.scala  |  2 +-
 .../apache/spark/sql/sources/interfaces.scala   |  2 +-
 .../spark/sql/sources/SaveLoadSuite.scala   |  2 +-
 .../spark/sql/hive/execution/commands.scala |  2 +-
 .../sql/hive/JavaMetastoreDataSourcesSuite.java |  2 +-
 .../sql/hive/MetastoreDataSourcesSuite.scala|  2 -
 13 files changed, 53 insertions(+), 58 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c025a468/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 3eb56ed..b6f052e 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -149,7 +149,7 @@ class DataFrame(object):
 def _java_save_mode(self, mode):
 """Returns the Java save mode based on the Python save mode 
represented by a string.
 """
-jSaveMode = self._sc._jvm.org.apache.spark.sql.sources.SaveMode
+jSaveMode = self._sc._jvm.org.apache.spark.sql.SaveMode
 jmode = jSaveMode.ErrorIfExists
 mode = mode.lower()
 if mode == "append":

http://git-wip-us.apache.org/repos/asf/spark/blob/c025a468/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java
--
diff --git a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java 
b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java
new file mode 100644
index 000..a40be52
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java
@@ -0,0 +1,45 @@
+/*
+ * 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 License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql;
+
+/**
+ * SaveMode is used to specify the expected behavior of saving a DataFrame to 
a data source.
+ */
+public enum SaveMode {
+  /**
+   * Append mode means that when saving a DataFrame to a data source, if 
data/table already exists,
+   * contents of the DataFrame are expected to be appended to existing data.
+   */
+  Append,
+  /**
+   * Overwrite mode means that when saving a DataFrame to a data source,
+   * if data/table already exists, existing data is expected to be overwritten 
by the contents of
+   * the DataFrame.
+   */
+  Overwrite,
+  /**
+   * ErrorIfExists mode means that when saving a DataFrame to a data source, 
if data already exists,
+   * an exception is expected to be thrown.
+   */
+  ErrorIfExists,
+  /**
+   * Ignore mode means that when saving a DataFrame to a data source, if data 
already exists,
+   * the save operation is expected to not save the contents of the DataFrame 
and to not
+   * change the existing data.
+   */
+  Ignore
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/c025a468/sql/core/src/main/java/org/apache/spark/sql/sources/SaveMode.java
--
diff --git a/sql/c

spark git commit: [SPARK-3299][SQL]Public API in SQLContext to list tables

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master c025a4688 -> 1d0596a16


[SPARK-3299][SQL]Public API in SQLContext to list tables

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

Author: Yin Huai 

Closes #4547 from yhuai/tables and squashes the following commits:

6c8f92e [Yin Huai] Add tableNames.
acbb281 [Yin Huai] Update Python test.
7793dcb [Yin Huai] Fix scala test.
572870d [Yin Huai] Address comments.
aba2e88 [Yin Huai] Format.
12c86df [Yin Huai] Add tables() to SQLContext to return a DataFrame containing 
existing tables.


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

Branch: refs/heads/master
Commit: 1d0596a16e1d3add2631f5d8169aeec2876a1362
Parents: c025a46
Author: Yin Huai 
Authored: Thu Feb 12 18:08:01 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 18:08:01 2015 -0800

--
 python/pyspark/sql/context.py   | 34 +
 .../spark/sql/catalyst/analysis/Catalog.scala   | 37 ++
 .../scala/org/apache/spark/sql/SQLContext.scala | 36 +
 .../org/apache/spark/sql/ListTablesSuite.scala  | 76 +++
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  5 ++
 .../apache/spark/sql/hive/ListTablesSuite.scala | 77 
 6 files changed, 265 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1d0596a1/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index db4bcbe..082f1b6 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -621,6 +621,40 @@ class SQLContext(object):
 """
 return DataFrame(self._ssql_ctx.table(tableName), self)
 
+def tables(self, dbName=None):
+"""Returns a DataFrame containing names of tables in the given 
database.
+
+If `dbName` is not specified, the current database will be used.
+
+The returned DataFrame has two columns, tableName and isTemporary
+(a column with BooleanType indicating if a table is a temporary one or 
not).
+
+>>> sqlCtx.registerRDDAsTable(df, "table1")
+>>> df2 = sqlCtx.tables()
+>>> df2.filter("tableName = 'table1'").first()
+Row(tableName=u'table1', isTemporary=True)
+"""
+if dbName is None:
+return DataFrame(self._ssql_ctx.tables(), self)
+else:
+return DataFrame(self._ssql_ctx.tables(dbName), self)
+
+def tableNames(self, dbName=None):
+"""Returns a list of names of tables in the database `dbName`.
+
+If `dbName` is not specified, the current database will be used.
+
+>>> sqlCtx.registerRDDAsTable(df, "table1")
+>>> "table1" in sqlCtx.tableNames()
+True
+>>> "table1" in sqlCtx.tableNames("db")
+True
+"""
+if dbName is None:
+return [name for name in self._ssql_ctx.tableNames()]
+else:
+return [name for name in self._ssql_ctx.tableNames(dbName)]
+
 def cacheTable(self, tableName):
 """Caches the specified table in-memory."""
 self._ssql_ctx.cacheTable(tableName)

http://git-wip-us.apache.org/repos/asf/spark/blob/1d0596a1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
index df8d03b..f57eab2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -34,6 +34,12 @@ trait Catalog {
   tableIdentifier: Seq[String],
   alias: Option[String] = None): LogicalPlan
 
+  /**
+   * Returns tuples of (tableName, isTemporary) for all tables in the given 
database.
+   * isTemporary is a Boolean value indicates if a table is a temporary or not.
+   */
+  def getTables(databaseName: Option[String]): Seq[(String, Boolean)]
+
   def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit
 
   def unregisterTable(tableIdentifier: Seq[String]): Unit
@@ -101,6 +107,12 @@ class SimpleCatalog(val caseSensitive: Boolean) extends 
Catalog {
 // properly qualified with this alias.
 alias.map(a => Subquery(a, 
tableWithQualifiers)).getOrElse(tableWithQualifiers)
   }
+
+  override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] 
= {
+tables.map {
+  case (name, _) => (name, true)
+ 

spark git commit: [SPARK-3299][SQL]Public API in SQLContext to list tables

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 925fd84a1 -> edbac178d


[SPARK-3299][SQL]Public API in SQLContext to list tables

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

Author: Yin Huai 

Closes #4547 from yhuai/tables and squashes the following commits:

6c8f92e [Yin Huai] Add tableNames.
acbb281 [Yin Huai] Update Python test.
7793dcb [Yin Huai] Fix scala test.
572870d [Yin Huai] Address comments.
aba2e88 [Yin Huai] Format.
12c86df [Yin Huai] Add tables() to SQLContext to return a DataFrame containing 
existing tables.

(cherry picked from commit 1d0596a16e1d3add2631f5d8169aeec2876a1362)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: edbac178d186f6936408b211385a5fea9e4f4603
Parents: 925fd84
Author: Yin Huai 
Authored: Thu Feb 12 18:08:01 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 18:08:19 2015 -0800

--
 python/pyspark/sql/context.py   | 34 +
 .../spark/sql/catalyst/analysis/Catalog.scala   | 37 ++
 .../scala/org/apache/spark/sql/SQLContext.scala | 36 +
 .../org/apache/spark/sql/ListTablesSuite.scala  | 76 +++
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  5 ++
 .../apache/spark/sql/hive/ListTablesSuite.scala | 77 
 6 files changed, 265 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/edbac178/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index db4bcbe..082f1b6 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -621,6 +621,40 @@ class SQLContext(object):
 """
 return DataFrame(self._ssql_ctx.table(tableName), self)
 
+def tables(self, dbName=None):
+"""Returns a DataFrame containing names of tables in the given 
database.
+
+If `dbName` is not specified, the current database will be used.
+
+The returned DataFrame has two columns, tableName and isTemporary
+(a column with BooleanType indicating if a table is a temporary one or 
not).
+
+>>> sqlCtx.registerRDDAsTable(df, "table1")
+>>> df2 = sqlCtx.tables()
+>>> df2.filter("tableName = 'table1'").first()
+Row(tableName=u'table1', isTemporary=True)
+"""
+if dbName is None:
+return DataFrame(self._ssql_ctx.tables(), self)
+else:
+return DataFrame(self._ssql_ctx.tables(dbName), self)
+
+def tableNames(self, dbName=None):
+"""Returns a list of names of tables in the database `dbName`.
+
+If `dbName` is not specified, the current database will be used.
+
+>>> sqlCtx.registerRDDAsTable(df, "table1")
+>>> "table1" in sqlCtx.tableNames()
+True
+>>> "table1" in sqlCtx.tableNames("db")
+True
+"""
+if dbName is None:
+return [name for name in self._ssql_ctx.tableNames()]
+else:
+return [name for name in self._ssql_ctx.tableNames(dbName)]
+
 def cacheTable(self, tableName):
 """Caches the specified table in-memory."""
 self._ssql_ctx.cacheTable(tableName)

http://git-wip-us.apache.org/repos/asf/spark/blob/edbac178/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
index df8d03b..f57eab2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -34,6 +34,12 @@ trait Catalog {
   tableIdentifier: Seq[String],
   alias: Option[String] = None): LogicalPlan
 
+  /**
+   * Returns tuples of (tableName, isTemporary) for all tables in the given 
database.
+   * isTemporary is a Boolean value indicates if a table is a temporary or not.
+   */
+  def getTables(databaseName: Option[String]): Seq[(String, Boolean)]
+
   def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit
 
   def unregisterTable(tableIdentifier: Seq[String]): Unit
@@ -101,6 +107,12 @@ class SimpleCatalog(val caseSensitive: Boolean) extends 
Catalog {
 // properly qualified with this alias.
 alias.map(a => Subquery(a, 
tableWithQualifiers)).getOrElse(tableWithQualifiers)
   }
+
+  override def getTables(databa

spark git commit: [SQL] Fix docs of SQLContext.tables

2015-02-12 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 1d0596a16 -> 2aea892eb


[SQL] Fix docs of SQLContext.tables

Author: Yin Huai 

Closes #4579 from yhuai/tablesDoc and squashes the following commits:

7f8964c [Yin Huai] Fix doc.


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

Branch: refs/heads/master
Commit: 2aea892ebd4d6c802defeef35ef7ebfe42c06eba
Parents: 1d0596a
Author: Yin Huai 
Authored: Thu Feb 12 20:37:55 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 12 20:37:55 2015 -0800

--
 .../main/scala/org/apache/spark/sql/SQLContext.scala| 12 ++--
 1 file changed, 6 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2aea892e/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 0f8af75..2165949 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -775,8 +775,8 @@ class SQLContext(@transient val sparkContext: SparkContext)
 DataFrame(this, catalog.lookupRelation(Seq(tableName)))
 
   /**
-   * Returns a [[DataFrame]] containing names of existing tables in the given 
database.
-   * The returned DataFrame has two columns, tableName and isTemporary (a 
column with BooleanType
+   * Returns a [[DataFrame]] containing names of existing tables in the 
current database.
+   * The returned DataFrame has two columns, tableName and isTemporary (a 
Boolean
* indicating if a table is a temporary one or not).
*/
   def tables(): DataFrame = {
@@ -784,8 +784,8 @@ class SQLContext(@transient val sparkContext: SparkContext)
   }
 
   /**
-   * Returns a [[DataFrame]] containing names of existing tables in the 
current database.
-   * The returned DataFrame has two columns, tableName and isTemporary (a 
column with BooleanType
+   * Returns a [[DataFrame]] containing names of existing tables in the given 
database.
+   * The returned DataFrame has two columns, tableName and isTemporary (a 
Boolean
* indicating if a table is a temporary one or not).
*/
   def tables(databaseName: String): DataFrame = {
@@ -793,7 +793,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
   }
 
   /**
-   * Returns an array of names of tables in the current database.
+   * Returns the names of tables in the current database as an array.
*/
   def tableNames(): Array[String] = {
 catalog.getTables(None).map {
@@ -802,7 +802,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
   }
 
   /**
-   * Returns an array of names of tables in the given database.
+   * Returns the names of tables in the given database as an array.
*/
   def tableNames(databaseName: String): Array[String] = {
 catalog.getTables(Some(databaseName)).map {


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



spark git commit: [SPARK-5642] [SQL] Apply column pruning on unused aggregation fields

2015-02-13 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 5d3cc6b3d -> 2cbb3e433


[SPARK-5642] [SQL] Apply column pruning on unused aggregation fields

select k from (select key k, max(value) v from src group by k) t

Author: Daoyuan Wang 
Author: Michael Armbrust 

Closes #4415 from adrian-wang/groupprune and squashes the following commits:

5d2d8a3 [Daoyuan Wang] address Michael's comments
61f8ef7 [Daoyuan Wang] add a unit test
80ddcc6 [Daoyuan Wang] keep project
b69d385 [Daoyuan Wang] add a prune rule for grouping set


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

Branch: refs/heads/master
Commit: 2cbb3e433ae334d5c318f05b987af314c854fbcc
Parents: 5d3cc6b
Author: Daoyuan Wang 
Authored: Fri Feb 13 13:46:50 2015 -0800
Committer: Michael Armbrust 
Committed: Fri Feb 13 13:48:39 2015 -0800

--
 .../sql/catalyst/optimizer/Optimizer.scala  |  9 +
 .../optimizer/FilterPushdownSuite.scala | 37 ++--
 2 files changed, 44 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2cbb3e43/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 0da081e..1a75fcf 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
@@ -119,6 +119,15 @@ object ColumnPruning extends Rule[LogicalPlan] {
 case a @ Aggregate(_, _, child) if (child.outputSet -- 
a.references).nonEmpty =>
   a.copy(child = Project(a.references.toSeq, child))
 
+case p @ Project(projectList, a @ Aggregate(groupingExpressions, 
aggregateExpressions, child))
+if (a.outputSet -- p.references).nonEmpty =>
+  Project(
+projectList,
+Aggregate(
+  groupingExpressions,
+  aggregateExpressions.filter(e => p.references.contains(e)),
+  child))
+
 // Eliminate unneeded attributes from either side of a Join.
 case Project(projectList, Join(left, right, joinType, condition)) =>
   // Collect the list of all references required either above or to 
evaluate the condition.

http://git-wip-us.apache.org/repos/asf/spark/blob/2cbb3e43/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
index 0b74bac..55c6766 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer
 
 import org.apache.spark.sql.catalyst.analysis
 import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries
-import org.apache.spark.sql.catalyst.expressions.Explode
+import org.apache.spark.sql.catalyst.expressions.{Count, Explode}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter}
 import org.apache.spark.sql.catalyst.rules._
@@ -37,7 +37,8 @@ class FilterPushdownSuite extends PlanTest {
 CombineFilters,
 PushPredicateThroughProject,
 PushPredicateThroughJoin,
-PushPredicateThroughGenerate) :: Nil
+PushPredicateThroughGenerate,
+ColumnPruning) :: Nil
   }
 
   val testRelation = LocalRelation('a.int, 'b.int, 'c.int)
@@ -58,6 +59,38 @@ class FilterPushdownSuite extends PlanTest {
 comparePlans(optimized, correctAnswer)
   }
 
+  test("column pruning for group") {
+val originalQuery =
+  testRelation
+.groupBy('a)('a, Count('b))
+.select('a)
+
+val optimized = Optimize(originalQuery.analyze)
+val correctAnswer =
+  testRelation
+.select('a)
+.groupBy('a)('a)
+.select('a).analyze
+
+comparePlans(optimized, correctAnswer)
+  }
+
+  test("column pruning for group with alias") {
+val originalQuery =
+  testRelation
+.groupBy('a)('a as 'c, Count('b))
+.select('c)
+
+val optimized = Optimize(originalQuery.analyze)
+val correctAnswer =
+  testRelation
+.select

spark git commit: [SPARK-5642] [SQL] Apply column pruning on unused aggregation fields

2015-02-13 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 41603717a -> efffc2e42


[SPARK-5642] [SQL] Apply column pruning on unused aggregation fields

select k from (select key k, max(value) v from src group by k) t

Author: Daoyuan Wang 
Author: Michael Armbrust 

Closes #4415 from adrian-wang/groupprune and squashes the following commits:

5d2d8a3 [Daoyuan Wang] address Michael's comments
61f8ef7 [Daoyuan Wang] add a unit test
80ddcc6 [Daoyuan Wang] keep project
b69d385 [Daoyuan Wang] add a prune rule for grouping set

(cherry picked from commit 2cbb3e433ae334d5c318f05b987af314c854fbcc)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: efffc2e428b1e867a586749685da90875f6bcfc4
Parents: 4160371
Author: Daoyuan Wang 
Authored: Fri Feb 13 13:46:50 2015 -0800
Committer: Michael Armbrust 
Committed: Fri Feb 13 13:49:21 2015 -0800

--
 .../sql/catalyst/optimizer/Optimizer.scala  |  9 +
 .../optimizer/FilterPushdownSuite.scala | 37 ++--
 2 files changed, 44 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/efffc2e4/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 0da081e..1a75fcf 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
@@ -119,6 +119,15 @@ object ColumnPruning extends Rule[LogicalPlan] {
 case a @ Aggregate(_, _, child) if (child.outputSet -- 
a.references).nonEmpty =>
   a.copy(child = Project(a.references.toSeq, child))
 
+case p @ Project(projectList, a @ Aggregate(groupingExpressions, 
aggregateExpressions, child))
+if (a.outputSet -- p.references).nonEmpty =>
+  Project(
+projectList,
+Aggregate(
+  groupingExpressions,
+  aggregateExpressions.filter(e => p.references.contains(e)),
+  child))
+
 // Eliminate unneeded attributes from either side of a Join.
 case Project(projectList, Join(left, right, joinType, condition)) =>
   // Collect the list of all references required either above or to 
evaluate the condition.

http://git-wip-us.apache.org/repos/asf/spark/blob/efffc2e4/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
index 0b74bac..55c6766 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer
 
 import org.apache.spark.sql.catalyst.analysis
 import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries
-import org.apache.spark.sql.catalyst.expressions.Explode
+import org.apache.spark.sql.catalyst.expressions.{Count, Explode}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter}
 import org.apache.spark.sql.catalyst.rules._
@@ -37,7 +37,8 @@ class FilterPushdownSuite extends PlanTest {
 CombineFilters,
 PushPredicateThroughProject,
 PushPredicateThroughJoin,
-PushPredicateThroughGenerate) :: Nil
+PushPredicateThroughGenerate,
+ColumnPruning) :: Nil
   }
 
   val testRelation = LocalRelation('a.int, 'b.int, 'c.int)
@@ -58,6 +59,38 @@ class FilterPushdownSuite extends PlanTest {
 comparePlans(optimized, correctAnswer)
   }
 
+  test("column pruning for group") {
+val originalQuery =
+  testRelation
+.groupBy('a)('a, Count('b))
+.select('a)
+
+val optimized = Optimize(originalQuery.analyze)
+val correctAnswer =
+  testRelation
+.select('a)
+.groupBy('a)('a)
+.select('a).analyze
+
+comparePlans(optimized, correctAnswer)
+  }
+
+  test("column pruning for group with alias") {
+val originalQuery =
+  testRelation
+.groupBy('a)('a as 'c, Count('b))
+.select('c)
+
+  

spark git commit: [SPARK-5789][SQL]Throw a better error message if JsonRDD.parseJson encounters unrecoverable parsing errors.

2015-02-13 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 2cbb3e433 -> 2e0c08452


[SPARK-5789][SQL]Throw a better error message if JsonRDD.parseJson encounters 
unrecoverable parsing errors.

Author: Yin Huai 

Closes #4582 from yhuai/jsonErrorMessage and squashes the following commits:

152dbd4 [Yin Huai] Update error message.
1466256 [Yin Huai] Throw a better error message when a JSON object in the input 
dataset span multiple records (lines for files or strings for an RDD of 
strings).


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

Branch: refs/heads/master
Commit: 2e0c084528409e1c565e6945521a33c0835ebbee
Parents: 2cbb3e4
Author: Yin Huai 
Authored: Fri Feb 13 13:51:06 2015 -0800
Committer: Michael Armbrust 
Committed: Fri Feb 13 13:51:06 2015 -0800

--
 sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala | 4 
 1 file changed, 4 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2e0c0845/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 7dfb304..3b8dde1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -306,6 +306,10 @@ private[sql] object JsonRDD extends Logging {
   val parsed = mapper.readValue(record, classOf[Object]) match {
 case map: java.util.Map[_, _] => 
scalafy(map).asInstanceOf[Map[String, Any]] :: Nil
 case list: java.util.List[_] => 
scalafy(list).asInstanceOf[Seq[Map[String, Any]]]
+case _ =>
+  sys.error(
+s"Failed to parse record $record. Please make sure that each 
line of the file " +
+"(or each string in the RDD) is a valid JSON object or an 
array of JSON objects.")
   }
 
   parsed


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



spark git commit: [SPARK-5789][SQL]Throw a better error message if JsonRDD.parseJson encounters unrecoverable parsing errors.

2015-02-13 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 efffc2e42 -> d9d0250fc


[SPARK-5789][SQL]Throw a better error message if JsonRDD.parseJson encounters 
unrecoverable parsing errors.

Author: Yin Huai 

Closes #4582 from yhuai/jsonErrorMessage and squashes the following commits:

152dbd4 [Yin Huai] Update error message.
1466256 [Yin Huai] Throw a better error message when a JSON object in the input 
dataset span multiple records (lines for files or strings for an RDD of 
strings).

(cherry picked from commit 2e0c084528409e1c565e6945521a33c0835ebbee)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: d9d0250fc5dfe529bebd4f67f945f4d7c3fc4106
Parents: efffc2e
Author: Yin Huai 
Authored: Fri Feb 13 13:51:06 2015 -0800
Committer: Michael Armbrust 
Committed: Fri Feb 13 13:51:34 2015 -0800

--
 sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala | 4 
 1 file changed, 4 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d9d0250f/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 7dfb304..3b8dde1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -306,6 +306,10 @@ private[sql] object JsonRDD extends Logging {
   val parsed = mapper.readValue(record, classOf[Object]) match {
 case map: java.util.Map[_, _] => 
scalafy(map).asInstanceOf[Map[String, Any]] :: Nil
 case list: java.util.List[_] => 
scalafy(list).asInstanceOf[Seq[Map[String, Any]]]
+case _ =>
+  sys.error(
+s"Failed to parse record $record. Please make sure that each 
line of the file " +
+"(or each string in the RDD) is a valid JSON object or an 
array of JSON objects.")
   }
 
   parsed


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



spark git commit: [SQL] [Minor] Update the SpecificMutableRow.copy

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 8e25373ce -> cc552e042


[SQL] [Minor] Update the SpecificMutableRow.copy

When profiling the Join / Aggregate queries via VisualVM, I noticed lots of 
`SpecificMutableRow` objects created, as well as the `MutableValue`, since the 
`SpecificMutableRow` are mostly used in data source implementation, but the 
`copy` method could be called multiple times in upper modules (e.g. in Join / 
aggregation etc.), duplicated instances created should be avoid.

Author: Cheng Hao 

Closes #4619 from chenghao-intel/specific_mutable_row and squashes the 
following commits:

9300d23 [Cheng Hao] update the SpecificMutableRow.copy


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

Branch: refs/heads/master
Commit: cc552e042896350e21eec9b78593de25006ecc70
Parents: 8e25373
Author: Cheng Hao 
Authored: Mon Feb 16 12:21:08 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:21:08 2015 -0800

--
 .../spark/sql/catalyst/expressions/SpecificMutableRow.scala   | 7 ---
 1 file changed, 4 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cc552e04/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
index 7434165..21d714c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
@@ -220,13 +220,14 @@ final class SpecificMutableRow(val values: 
Array[MutableValue]) extends MutableR
   override def isNullAt(i: Int): Boolean = values(i).isNull
 
   override def copy(): Row = {
-val newValues = new Array[MutableValue](values.length)
+val newValues = new Array[Any](values.length)
 var i = 0
 while (i < values.length) {
-  newValues(i) = values(i).copy()
+  newValues(i) = values(i).boxed
   i += 1
 }
-new SpecificMutableRow(newValues)
+
+new GenericRow(newValues)
   }
 
   override def update(ordinal: Int, value: Any): Unit = {


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



spark git commit: [SQL] [Minor] Update the SpecificMutableRow.copy

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 fef2267cd -> 1a8895560


[SQL] [Minor] Update the SpecificMutableRow.copy

When profiling the Join / Aggregate queries via VisualVM, I noticed lots of 
`SpecificMutableRow` objects created, as well as the `MutableValue`, since the 
`SpecificMutableRow` are mostly used in data source implementation, but the 
`copy` method could be called multiple times in upper modules (e.g. in Join / 
aggregation etc.), duplicated instances created should be avoid.

Author: Cheng Hao 

Closes #4619 from chenghao-intel/specific_mutable_row and squashes the 
following commits:

9300d23 [Cheng Hao] update the SpecificMutableRow.copy

(cherry picked from commit cc552e042896350e21eec9b78593de25006ecc70)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 1a8895560f668faed33e99bcb88cafefd64fef03
Parents: fef2267
Author: Cheng Hao 
Authored: Mon Feb 16 12:21:08 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:21:29 2015 -0800

--
 .../spark/sql/catalyst/expressions/SpecificMutableRow.scala   | 7 ---
 1 file changed, 4 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1a889556/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
index 7434165..21d714c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
@@ -220,13 +220,14 @@ final class SpecificMutableRow(val values: 
Array[MutableValue]) extends MutableR
   override def isNullAt(i: Int): Boolean = values(i).isNull
 
   override def copy(): Row = {
-val newValues = new Array[MutableValue](values.length)
+val newValues = new Array[Any](values.length)
 var i = 0
 while (i < values.length) {
-  newValues(i) = values(i).copy()
+  newValues(i) = values(i).boxed
   i += 1
 }
-new SpecificMutableRow(newValues)
+
+new GenericRow(newValues)
   }
 
   override def update(ordinal: Int, value: Any): Unit = {


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



spark git commit: [SPARK-5824] [SQL] add null format in ctas and set default col comment to null

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master cc552e042 -> 275a0c081


[SPARK-5824] [SQL] add null format in ctas and set default col comment to null

Author: Daoyuan Wang 

Closes #4609 from adrian-wang/ctas and squashes the following commits:

0a75d5a [Daoyuan Wang] reorder import
93d1863 [Daoyuan Wang] add null format in ctas and set default col comment to 
null


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

Branch: refs/heads/master
Commit: 275a0c08134dea1896eab73a8e017256900fb1db
Parents: cc552e0
Author: Daoyuan Wang 
Authored: Mon Feb 16 12:31:36 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:31:36 2015 -0800

--
 .../hive/execution/HiveCompatibilitySuite.scala   |  1 +
 .../spark/sql/hive/HiveMetastoreCatalog.scala |  3 ++-
 ...llformatCTAS-0-36f9196395758cebfed837a1c391a1e |  0
 ...lformatCTAS-1-b5a31d4cb34218b8de1ac3fed59fa75b |  0
 ...formatCTAS-10-7f4f04b87c7ef9653b4646949b24cf0b | 10 ++
 ...formatCTAS-11-4a4c16b53c612d00012d338c97bf5281 |  0
 ...formatCTAS-12-7f4f04b87c7ef9653b4646949b24cf0b | 10 ++
 ...formatCTAS-13-2e59caa113585495d8684fee69d88bc0 |  0
 ...formatCTAS-14-ad9fe9d68c2cf492259af4f6167c1b12 |  0
 ...lformatCTAS-2-aa2bdbd93668dceae43d1a02f2ede68d |  0
 ...lformatCTAS-3-b0057150f237050f38c1efa1f2d6b273 |  6 ++
 ...lformatCTAS-4-16c7086f39d6458b6c5cf2479f0473bd |  0
 ...lformatCTAS-5-183d77b734ce6a373de5b3ebe1cd04c9 |  0
 ...lformatCTAS-6-159fff36b548e00ee952d1df8ef19833 |  0
 ...lformatCTAS-7-46900b082b02ce3e58087d1f41128f65 |  4 
 ...lformatCTAS-8-7f26cbd6be5631a3acce26f667d1c5d8 | 18 ++
 ...lformatCTAS-9-22e1b3899de7087b39c24d9d8f643b47 |  1 +
 .../scala/org/apache/spark/sql/hive/Shim12.scala  |  2 ++
 .../scala/org/apache/spark/sql/hive/Shim13.scala  |  7 +++
 19 files changed, 61 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/275a0c08/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
--
diff --git 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index e443e5b..133f2d3 100644
--- 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -640,6 +640,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
 "nonblock_op_deduplicate",
 "notable_alias1",
 "notable_alias2",
+"nullformatCTAS",
 "nullgroup",
 "nullgroup2",
 "nullgroup3",

http://git-wip-us.apache.org/repos/asf/spark/blob/275a0c08/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 6d794d0..f82778c 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
@@ -240,7 +240,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
 val hiveSchema: JList[FieldSchema] = if (schema == null || schema.isEmpty) 
{
   crtTbl.getCols
 } else {
-  schema.map(attr => new FieldSchema(attr.name, 
toMetastoreType(attr.dataType), ""))
+  schema.map(attr => new FieldSchema(attr.name, 
toMetastoreType(attr.dataType), null))
 }
 tbl.setFields(hiveSchema)
 
@@ -314,6 +314,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
 if (crtTbl != null && crtTbl.getLineDelim() != null) {
   tbl.setSerdeParam(serdeConstants.LINE_DELIM, crtTbl.getLineDelim())
 }
+HiveShim.setTblNullFormat(crtTbl, tbl)
 
 if (crtTbl != null && crtTbl.getSerdeProps() != null) {
   val iter = crtTbl.getSerdeProps().entrySet().iterator()

http://git-wip-us.apache.org/repos/asf/spark/blob/275a0c08/sql/hive/src/test/resources/golden/nullformatCTAS-0-36f9196395758cebfed837a1c391a1e
--
diff --git 
a/sql/hive/src/test/resources/golden/nullformatCTAS-0-36f9196395758cebfed837a1c391a1e
 
b/sql/hive/src/test/resources/golden/nullformatCTAS-0-36f9196395758cebfed837a1c391a1e
new

spark git commit: [SQL] Initial support for reporting location of error in sql string

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 c2eaaea9f -> 63fa123f1


[SQL] Initial support for reporting location of error in sql string

Author: Michael Armbrust 

Closes #4587 from marmbrus/position and squashes the following commits:

0810052 [Michael Armbrust] fix tests
395c019 [Michael Armbrust] Merge remote-tracking branch 'marmbrus/position' 
into position
e155dce [Michael Armbrust] more errors
f3efa51 [Michael Armbrust] Update AnalysisException.scala
d45ff60 [Michael Armbrust] [SQL] Initial support for reporting location of 
error in sql string

(cherry picked from commit 104b2c45805ce0a9c86e2823f402de6e9f0aee81)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 63fa123f1c2113caea74a7cf9a7293f256441dc7
Parents: c2eaaea
Author: Michael Armbrust 
Authored: Mon Feb 16 12:32:56 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:33:11 2015 -0800

--
 .../apache/spark/sql/AnalysisException.scala|  17 +-
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  19 ++-
 .../spark/sql/catalyst/analysis/Catalog.scala   |   6 +
 .../spark/sql/catalyst/analysis/package.scala   |  10 ++
 .../spark/sql/catalyst/trees/TreeNode.scala |  60 ++-
 .../sql/catalyst/trees/TreeNodeSuite.scala  |  14 ++
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |   9 +-
 .../org/apache/spark/sql/hive/HiveQl.scala  |  47 --
 .../spark/sql/hive/CachedTableSuite.scala   |   4 +-
 .../spark/sql/hive/ErrorPositionSuite.scala | 163 +++
 .../sql/hive/execution/SQLQuerySuite.scala  |   4 +-
 11 files changed, 314 insertions(+), 39 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/63fa123f/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
index 871d560..15add84 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
@@ -17,7 +17,22 @@
 
 package org.apache.spark.sql
 
+import org.apache.spark.annotation.DeveloperApi
+
 /**
+ * :: DeveloperApi ::
  * Thrown when a query fails to analyze, usually because the query itself is 
invalid.
  */
-class AnalysisException(message: String) extends Exception(message) with 
Serializable
+@DeveloperApi
+class AnalysisException protected[sql] (
+val message: String,
+val line: Option[Int] = None,
+val startPosition: Option[Int] = None)
+  extends Exception with Serializable {
+
+  override def getMessage: String = {
+val lineAnnotation = line.map(l => s" line $l").getOrElse("")
+val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("")
+s"$message;$lineAnnotation$positionAnnotation"
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/63fa123f/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 58a7003..aa4320b 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
@@ -85,7 +85,7 @@ class Analyzer(catalog: Catalog,
   operator transformExpressionsUp {
 case a: Attribute if !a.resolved =>
   val from = operator.inputSet.map(_.name).mkString(", ")
-  failAnalysis(s"cannot resolve '${a.prettyString}' given input 
columns $from")
+  a.failAnalysis(s"cannot resolve '${a.prettyString}' given input 
columns $from")
 
 case c: Cast if !c.resolved =>
   failAnalysis(
@@ -246,12 +246,21 @@ class Analyzer(catalog: Catalog,
* Replaces [[UnresolvedRelation]]s with concrete relations from the catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
+def getTable(u: UnresolvedRelation) = {
+  try {
+catalog.lookupRelation(u.tableIdentifier, u.alias)
+  } catch {
+case _: NoSuchTableException =>
+  u.failAnalysis(s"no such table ${u.t

spark git commit: [SPARK-5824] [SQL] add null format in ctas and set default col comment to null

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 1a8895560 -> c2eaaea9f


[SPARK-5824] [SQL] add null format in ctas and set default col comment to null

Author: Daoyuan Wang 

Closes #4609 from adrian-wang/ctas and squashes the following commits:

0a75d5a [Daoyuan Wang] reorder import
93d1863 [Daoyuan Wang] add null format in ctas and set default col comment to 
null

(cherry picked from commit 275a0c08134dea1896eab73a8e017256900fb1db)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: c2eaaea9f9f77662a4c9405b2796aa6bd362466e
Parents: 1a88955
Author: Daoyuan Wang 
Authored: Mon Feb 16 12:31:36 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:31:44 2015 -0800

--
 .../hive/execution/HiveCompatibilitySuite.scala   |  1 +
 .../spark/sql/hive/HiveMetastoreCatalog.scala |  3 ++-
 ...llformatCTAS-0-36f9196395758cebfed837a1c391a1e |  0
 ...lformatCTAS-1-b5a31d4cb34218b8de1ac3fed59fa75b |  0
 ...formatCTAS-10-7f4f04b87c7ef9653b4646949b24cf0b | 10 ++
 ...formatCTAS-11-4a4c16b53c612d00012d338c97bf5281 |  0
 ...formatCTAS-12-7f4f04b87c7ef9653b4646949b24cf0b | 10 ++
 ...formatCTAS-13-2e59caa113585495d8684fee69d88bc0 |  0
 ...formatCTAS-14-ad9fe9d68c2cf492259af4f6167c1b12 |  0
 ...lformatCTAS-2-aa2bdbd93668dceae43d1a02f2ede68d |  0
 ...lformatCTAS-3-b0057150f237050f38c1efa1f2d6b273 |  6 ++
 ...lformatCTAS-4-16c7086f39d6458b6c5cf2479f0473bd |  0
 ...lformatCTAS-5-183d77b734ce6a373de5b3ebe1cd04c9 |  0
 ...lformatCTAS-6-159fff36b548e00ee952d1df8ef19833 |  0
 ...lformatCTAS-7-46900b082b02ce3e58087d1f41128f65 |  4 
 ...lformatCTAS-8-7f26cbd6be5631a3acce26f667d1c5d8 | 18 ++
 ...lformatCTAS-9-22e1b3899de7087b39c24d9d8f643b47 |  1 +
 .../scala/org/apache/spark/sql/hive/Shim12.scala  |  2 ++
 .../scala/org/apache/spark/sql/hive/Shim13.scala  |  7 +++
 19 files changed, 61 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c2eaaea9/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
--
diff --git 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index e443e5b..133f2d3 100644
--- 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -640,6 +640,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
 "nonblock_op_deduplicate",
 "notable_alias1",
 "notable_alias2",
+"nullformatCTAS",
 "nullgroup",
 "nullgroup2",
 "nullgroup3",

http://git-wip-us.apache.org/repos/asf/spark/blob/c2eaaea9/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 6d794d0..f82778c 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
@@ -240,7 +240,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
 val hiveSchema: JList[FieldSchema] = if (schema == null || schema.isEmpty) 
{
   crtTbl.getCols
 } else {
-  schema.map(attr => new FieldSchema(attr.name, 
toMetastoreType(attr.dataType), ""))
+  schema.map(attr => new FieldSchema(attr.name, 
toMetastoreType(attr.dataType), null))
 }
 tbl.setFields(hiveSchema)
 
@@ -314,6 +314,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
 if (crtTbl != null && crtTbl.getLineDelim() != null) {
   tbl.setSerdeParam(serdeConstants.LINE_DELIM, crtTbl.getLineDelim())
 }
+HiveShim.setTblNullFormat(crtTbl, tbl)
 
 if (crtTbl != null && crtTbl.getSerdeProps() != null) {
   val iter = crtTbl.getSerdeProps().entrySet().iterator()

http://git-wip-us.apache.org/repos/asf/spark/blob/c2eaaea9/sql/hive/src/test/resources/golden/nullformatCTAS-0-36f9196395758cebfed837a1c391a1e
--
diff --git 
a/sql/hive/src/test/resources/golden/nullformatCTAS-0-36f919639575

spark git commit: [SQL] Initial support for reporting location of error in sql string

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 275a0c081 -> 104b2c458


[SQL] Initial support for reporting location of error in sql string

Author: Michael Armbrust 

Closes #4587 from marmbrus/position and squashes the following commits:

0810052 [Michael Armbrust] fix tests
395c019 [Michael Armbrust] Merge remote-tracking branch 'marmbrus/position' 
into position
e155dce [Michael Armbrust] more errors
f3efa51 [Michael Armbrust] Update AnalysisException.scala
d45ff60 [Michael Armbrust] [SQL] Initial support for reporting location of 
error in sql string


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

Branch: refs/heads/master
Commit: 104b2c45805ce0a9c86e2823f402de6e9f0aee81
Parents: 275a0c0
Author: Michael Armbrust 
Authored: Mon Feb 16 12:32:56 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:32:56 2015 -0800

--
 .../apache/spark/sql/AnalysisException.scala|  17 +-
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  19 ++-
 .../spark/sql/catalyst/analysis/Catalog.scala   |   6 +
 .../spark/sql/catalyst/analysis/package.scala   |  10 ++
 .../spark/sql/catalyst/trees/TreeNode.scala |  60 ++-
 .../sql/catalyst/trees/TreeNodeSuite.scala  |  14 ++
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |   9 +-
 .../org/apache/spark/sql/hive/HiveQl.scala  |  47 --
 .../spark/sql/hive/CachedTableSuite.scala   |   4 +-
 .../spark/sql/hive/ErrorPositionSuite.scala | 163 +++
 .../sql/hive/execution/SQLQuerySuite.scala  |   4 +-
 11 files changed, 314 insertions(+), 39 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/104b2c45/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
index 871d560..15add84 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
@@ -17,7 +17,22 @@
 
 package org.apache.spark.sql
 
+import org.apache.spark.annotation.DeveloperApi
+
 /**
+ * :: DeveloperApi ::
  * Thrown when a query fails to analyze, usually because the query itself is 
invalid.
  */
-class AnalysisException(message: String) extends Exception(message) with 
Serializable
+@DeveloperApi
+class AnalysisException protected[sql] (
+val message: String,
+val line: Option[Int] = None,
+val startPosition: Option[Int] = None)
+  extends Exception with Serializable {
+
+  override def getMessage: String = {
+val lineAnnotation = line.map(l => s" line $l").getOrElse("")
+val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("")
+s"$message;$lineAnnotation$positionAnnotation"
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/104b2c45/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 58a7003..aa4320b 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
@@ -85,7 +85,7 @@ class Analyzer(catalog: Catalog,
   operator transformExpressionsUp {
 case a: Attribute if !a.resolved =>
   val from = operator.inputSet.map(_.name).mkString(", ")
-  failAnalysis(s"cannot resolve '${a.prettyString}' given input 
columns $from")
+  a.failAnalysis(s"cannot resolve '${a.prettyString}' given input 
columns $from")
 
 case c: Cast if !c.resolved =>
   failAnalysis(
@@ -246,12 +246,21 @@ class Analyzer(catalog: Catalog,
* Replaces [[UnresolvedRelation]]s with concrete relations from the catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
+def getTable(u: UnresolvedRelation) = {
+  try {
+catalog.lookupRelation(u.tableIdentifier, u.alias)
+  } catch {
+case _: NoSuchTableException =>
+  u.failAnalysis(s"no such table ${u.tableIdentifier}")
+  }
+}
+
 def apply(plan: LogicalPlan): LogicalPlan = plan transform {
-   

spark git commit: [SQL] Add fetched row count in SparkSQLCLIDriver

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 63fa123f1 -> 0368494c5


[SQL] Add fetched row count in SparkSQLCLIDriver

before this change:
```scala
Time taken: 0.619 seconds
```

after this change :
```scala
Time taken: 0.619 seconds, Fetched: 4 row(s)
```

Author: OopsOutOfMemory 

Closes #4604 from OopsOutOfMemory/rowcount and squashes the following commits:

7252dea [OopsOutOfMemory] add fetched row count

(cherry picked from commit b4d7c7032d755de42951f92d9535287ef6230b9b)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 0368494c502c33c05f806d106ff2042acad91cee
Parents: 63fa123
Author: OopsOutOfMemory 
Authored: Mon Feb 16 12:34:09 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:34:22 2015 -0800

--
 .../spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala | 12 ++--
 1 file changed, 10 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0368494c/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index bb19ac2..401e97b 100755
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -292,9 +292,13 @@ private[hive] class SparkSQLCLIDriver extends CliDriver 
with Logging {
 }
   }
 
+  var counter = 0
   try {
 while (!out.checkError() && driver.getResults(res)) {
-  res.foreach(out.println)
+  res.foreach{ l =>
+counter += 1
+out.println(l)
+  }
   res.clear()
 }
   } catch {
@@ -311,7 +315,11 @@ private[hive] class SparkSQLCLIDriver extends CliDriver 
with Logging {
 ret = cret
   }
 
-  console.printInfo(s"Time taken: $timeTaken seconds", null)
+  var responseMsg = s"Time taken: $timeTaken seconds"
+  if (counter != 0) {
+responseMsg += s", Fetched $counter row(s)"
+  }
+  console.printInfo(responseMsg , null)
   // Destroy the driver to release all the locks.
   driver.destroy()
 } else {


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



spark git commit: [SQL] Add fetched row count in SparkSQLCLIDriver

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 104b2c458 -> b4d7c7032


[SQL] Add fetched row count in SparkSQLCLIDriver

before this change:
```scala
Time taken: 0.619 seconds
```

after this change :
```scala
Time taken: 0.619 seconds, Fetched: 4 row(s)
```

Author: OopsOutOfMemory 

Closes #4604 from OopsOutOfMemory/rowcount and squashes the following commits:

7252dea [OopsOutOfMemory] add fetched row count


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

Branch: refs/heads/master
Commit: b4d7c7032d755de42951f92d9535287ef6230b9b
Parents: 104b2c4
Author: OopsOutOfMemory 
Authored: Mon Feb 16 12:34:09 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:34:09 2015 -0800

--
 .../spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala | 12 ++--
 1 file changed, 10 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b4d7c703/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index bb19ac2..401e97b 100755
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -292,9 +292,13 @@ private[hive] class SparkSQLCLIDriver extends CliDriver 
with Logging {
 }
   }
 
+  var counter = 0
   try {
 while (!out.checkError() && driver.getResults(res)) {
-  res.foreach(out.println)
+  res.foreach{ l =>
+counter += 1
+out.println(l)
+  }
   res.clear()
 }
   } catch {
@@ -311,7 +315,11 @@ private[hive] class SparkSQLCLIDriver extends CliDriver 
with Logging {
 ret = cret
   }
 
-  console.printInfo(s"Time taken: $timeTaken seconds", null)
+  var responseMsg = s"Time taken: $timeTaken seconds"
+  if (counter != 0) {
+responseMsg += s", Fetched $counter row(s)"
+  }
+  console.printInfo(responseMsg , null)
   // Destroy the driver to release all the locks.
   driver.destroy()
 } else {


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



spark git commit: [SPARK-5296] [SQL] Add more filter types for data sources API

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master b4d7c7032 -> 6f54dee66


[SPARK-5296] [SQL] Add more filter types for data sources API

This PR adds the following filter types for data sources API:

- `IsNull`
- `IsNotNull`
- `Not`
- `And`
- `Or`

The code which converts Catalyst predicate expressions to data sources filters 
is very similar to filter conversion logics in `ParquetFilters` which converts 
Catalyst predicates to Parquet filter predicates. In this way we can support 
nested AND/OR/NOT predicates without changing current `BaseScan` type hierarchy.


[https://reviewable.io/review_button.png"; height=40 alt="Review on 
Reviewable"/>](https://reviewable.io/reviews/apache/spark/4623)


Author: Cheng Lian 

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust 

Closes #4623 from liancheng/more-fiters and squashes the following commits:

1b296f4 [Cheng Lian] Add more filter types for data sources API


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

Branch: refs/heads/master
Commit: 6f54dee66100e5e58f6649158db257eb5009bd6a
Parents: b4d7c70
Author: Cheng Lian 
Authored: Mon Feb 16 12:48:55 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:48:55 2015 -0800

--
 .../scala/org/apache/spark/sql/SQLContext.scala |  9 ++-
 .../apache/spark/sql/parquet/newParquet.scala   |  5 +-
 .../spark/sql/sources/DataSourceStrategy.scala  | 81 ++--
 .../org/apache/spark/sql/sources/filters.scala  |  5 ++
 .../spark/sql/sources/FilteredScanSuite.scala   | 34 +++-
 5 files changed, 103 insertions(+), 31 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6f54dee6/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index b42a52e..1442250 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -28,16 +28,16 @@ import scala.reflect.runtime.universe.TypeTag
 import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental}
 import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer}
 import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan, NoRelation}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
-import org.apache.spark.sql.execution._
+import org.apache.spark.sql.catalyst.{ScalaReflection, expressions}
+import org.apache.spark.sql.execution.{Filter, _}
 import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, 
JDBCRelation}
 import org.apache.spark.sql.json._
-import org.apache.spark.sql.sources.{BaseRelation, DDLParser, 
DataSourceStrategy, LogicalRelation, _}
+import org.apache.spark.sql.sources._
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 import org.apache.spark.{Partition, SparkContext}
@@ -867,7 +867,8 @@ class SQLContext(@transient val sparkContext: SparkContext)
 
   val projectSet = AttributeSet(projectList.flatMap(_.references))
   val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
-  val filterCondition = 
prunePushedDownFilters(filterPredicates).reduceLeftOption(And)
+  val filterCondition =
+
prunePushedDownFilters(filterPredicates).reduceLeftOption(expressions.And)
 
   // Right now we still use a projection even if the only evaluation is 
applying an alias
   // to a column.  Since this is a no-op, it could be avoided. However, 
using this

http://git-wip-us.apache.org/repos/asf/spark/blob/6f54dee6/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
index 9279f5a..9bb34e2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
@@ -32,6 +32,7 @@ import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
 import org.apache.hadoop.mapred

spark git commit: [SPARK-5296] [SQL] Add more filter types for data sources API

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 0368494c5 -> 363a9a7d5


[SPARK-5296] [SQL] Add more filter types for data sources API

This PR adds the following filter types for data sources API:

- `IsNull`
- `IsNotNull`
- `Not`
- `And`
- `Or`

The code which converts Catalyst predicate expressions to data sources filters 
is very similar to filter conversion logics in `ParquetFilters` which converts 
Catalyst predicates to Parquet filter predicates. In this way we can support 
nested AND/OR/NOT predicates without changing current `BaseScan` type hierarchy.


[https://reviewable.io/review_button.png"; height=40 alt="Review on 
Reviewable"/>](https://reviewable.io/reviews/apache/spark/4623)


Author: Cheng Lian 

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust 

Closes #4623 from liancheng/more-fiters and squashes the following commits:

1b296f4 [Cheng Lian] Add more filter types for data sources API


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

Branch: refs/heads/branch-1.3
Commit: 363a9a7d5ad682f828288f792a836c2c0b5e2f89
Parents: 0368494
Author: Cheng Lian 
Authored: Mon Feb 16 12:48:55 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:50:10 2015 -0800

--
 .../scala/org/apache/spark/sql/SQLContext.scala |  9 ++-
 .../apache/spark/sql/parquet/newParquet.scala   |  5 +-
 .../spark/sql/sources/DataSourceStrategy.scala  | 81 ++--
 .../org/apache/spark/sql/sources/filters.scala  |  5 ++
 .../spark/sql/sources/FilteredScanSuite.scala   | 34 +++-
 5 files changed, 103 insertions(+), 31 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/363a9a7d/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index b42a52e..1442250 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -28,16 +28,16 @@ import scala.reflect.runtime.universe.TypeTag
 import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental}
 import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer}
 import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan, NoRelation}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
-import org.apache.spark.sql.execution._
+import org.apache.spark.sql.catalyst.{ScalaReflection, expressions}
+import org.apache.spark.sql.execution.{Filter, _}
 import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, 
JDBCRelation}
 import org.apache.spark.sql.json._
-import org.apache.spark.sql.sources.{BaseRelation, DDLParser, 
DataSourceStrategy, LogicalRelation, _}
+import org.apache.spark.sql.sources._
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 import org.apache.spark.{Partition, SparkContext}
@@ -867,7 +867,8 @@ class SQLContext(@transient val sparkContext: SparkContext)
 
   val projectSet = AttributeSet(projectList.flatMap(_.references))
   val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
-  val filterCondition = 
prunePushedDownFilters(filterPredicates).reduceLeftOption(And)
+  val filterCondition =
+
prunePushedDownFilters(filterPredicates).reduceLeftOption(expressions.And)
 
   // Right now we still use a projection even if the only evaluation is 
applying an alias
   // to a column.  Since this is a no-op, it could be avoided. However, 
using this

http://git-wip-us.apache.org/repos/asf/spark/blob/363a9a7d/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
index 9279f5a..9bb34e2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
@@ -32,6 +32,7 @@ import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
 import org.apache.hadoo

spark git commit: [SPARK-5833] [SQL] Adds REFRESH TABLE command

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 6f54dee66 -> c51ab37fa


[SPARK-5833] [SQL] Adds REFRESH TABLE command

Lifts `HiveMetastoreCatalog.refreshTable` to `Catalog`. Adds `RefreshTable` 
command to refresh (possibly cached) metadata in external data sources tables.


[https://reviewable.io/review_button.png"; height=40 alt="Review on 
Reviewable"/>](https://reviewable.io/reviews/apache/spark/4624)


Author: Cheng Lian 

Closes #4624 from liancheng/refresh-table and squashes the following commits:

8d1aa4c [Cheng Lian] Adds REFRESH TABLE command


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

Branch: refs/heads/master
Commit: c51ab37faddf4ede23243058dfb388e74a192552
Parents: 6f54dee
Author: Cheng Lian 
Authored: Mon Feb 16 12:52:05 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:52:05 2015 -0800

--
 .../spark/sql/catalyst/analysis/Catalog.scala   | 10 
 .../org/apache/spark/sql/sources/ddl.scala  | 52 +++-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  2 +-
 .../sql/hive/MetastoreDataSourcesSuite.scala|  2 +-
 4 files changed, 42 insertions(+), 24 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c51ab37f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
index bf97215..9e6e291 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -46,6 +46,8 @@ trait Catalog {
*/
   def getTables(databaseName: Option[String]): Seq[(String, Boolean)]
 
+  def refreshTable(databaseName: String, tableName: String): Unit
+
   def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit
 
   def unregisterTable(tableIdentifier: Seq[String]): Unit
@@ -119,6 +121,10 @@ class SimpleCatalog(val caseSensitive: Boolean) extends 
Catalog {
   case (name, _) => (name, true)
 }.toSeq
   }
+
+  override def refreshTable(databaseName: String, tableName: String): Unit = {
+throw new UnsupportedOperationException
+  }
 }
 
 /**
@@ -224,4 +230,8 @@ object EmptyCatalog extends Catalog {
   }
 
   override def unregisterAllTables(): Unit = {}
+
+  override def refreshTable(databaseName: String, tableName: String): Unit = {
+throw new UnsupportedOperationException
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/c51ab37f/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
index 8cac9c0..1b5e8c2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference}
 import org.apache.spark.sql.execution.RunnableCommand
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
@@ -66,6 +66,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser 
with Logging {
   protected val EXTENDED = Keyword("EXTENDED")
   protected val AS = Keyword("AS")
   protected val COMMENT = Keyword("COMMENT")
+  protected val REFRESH = Keyword("REFRESH")
 
   // Data types.
   protected val STRING = Keyword("STRING")
@@ -85,7 +86,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser 
with Logging {
   protected val MAP = Keyword("MAP")
   protected val STRUCT = Keyword("STRUCT")
 
-  protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable
+  protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable | 
refreshTable
 
   protected def start: Parser[LogicalPlan] = ddl
 
@@ -104,9 +105,8 @@ private[sql] class DDLParser extends AbstractSparkSQLParser 
with Logging {
* AS SELECT ...
*/
   protected lazy val createTable: Parser[LogicalPlan] =
-  (
-(CREATE ~> TEMPORARY.? <~ TABLE) ~ (IF ~> NOT <~ EXISTS).? ~ ident
-  ~ (tableCo

spark git commit: [SPARK-5833] [SQL] Adds REFRESH TABLE command

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 363a9a7d5 -> 864d77e0d


[SPARK-5833] [SQL] Adds REFRESH TABLE command

Lifts `HiveMetastoreCatalog.refreshTable` to `Catalog`. Adds `RefreshTable` 
command to refresh (possibly cached) metadata in external data sources tables.


[https://reviewable.io/review_button.png"; height=40 alt="Review on 
Reviewable"/>](https://reviewable.io/reviews/apache/spark/4624)


Author: Cheng Lian 

Closes #4624 from liancheng/refresh-table and squashes the following commits:

8d1aa4c [Cheng Lian] Adds REFRESH TABLE command

(cherry picked from commit c51ab37faddf4ede23243058dfb388e74a192552)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 864d77e0d23b974943a1875b7372de05b3595bd5
Parents: 363a9a7
Author: Cheng Lian 
Authored: Mon Feb 16 12:52:05 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 12:52:28 2015 -0800

--
 .../spark/sql/catalyst/analysis/Catalog.scala   | 10 
 .../org/apache/spark/sql/sources/ddl.scala  | 52 +++-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  2 +-
 .../sql/hive/MetastoreDataSourcesSuite.scala|  2 +-
 4 files changed, 42 insertions(+), 24 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/864d77e0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
index bf97215..9e6e291 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -46,6 +46,8 @@ trait Catalog {
*/
   def getTables(databaseName: Option[String]): Seq[(String, Boolean)]
 
+  def refreshTable(databaseName: String, tableName: String): Unit
+
   def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit
 
   def unregisterTable(tableIdentifier: Seq[String]): Unit
@@ -119,6 +121,10 @@ class SimpleCatalog(val caseSensitive: Boolean) extends 
Catalog {
   case (name, _) => (name, true)
 }.toSeq
   }
+
+  override def refreshTable(databaseName: String, tableName: String): Unit = {
+throw new UnsupportedOperationException
+  }
 }
 
 /**
@@ -224,4 +230,8 @@ object EmptyCatalog extends Catalog {
   }
 
   override def unregisterAllTables(): Unit = {}
+
+  override def refreshTable(databaseName: String, tableName: String): Unit = {
+throw new UnsupportedOperationException
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/864d77e0/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
index 16ea952..c762568 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference}
 import org.apache.spark.sql.execution.RunnableCommand
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
@@ -66,6 +66,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser 
with Logging {
   protected val EXTENDED = Keyword("EXTENDED")
   protected val AS = Keyword("AS")
   protected val COMMENT = Keyword("COMMENT")
+  protected val REFRESH = Keyword("REFRESH")
 
   // Data types.
   protected val STRING = Keyword("STRING")
@@ -85,7 +86,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser 
with Logging {
   protected val MAP = Keyword("MAP")
   protected val STRUCT = Keyword("STRUCT")
 
-  protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable
+  protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable | 
refreshTable
 
   protected def start: Parser[LogicalPlan] = ddl
 
@@ -104,9 +105,8 @@ private[sql] class DDLParser extends AbstractSparkSQLParser 
with Logging {
* AS SELECT ...
*/
   protected lazy val createTable: Parse

spark git commit: [SPARK-5746][SQL] Check invalid cases for the write path of data source API

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 04b401da8 -> 5b6cd65cd


[SPARK-5746][SQL] Check invalid cases for the write path of data source API

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

liancheng marmbrus

Author: Yin Huai 

Closes #4617 from yhuai/insertOverwrite and squashes the following commits:

8e3019d [Yin Huai] Fix compilation error.
499e8e7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
insertOverwrite
e76e85a [Yin Huai] Address comments.
ac31b3c [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
insertOverwrite
f30bdad [Yin Huai] Use toDF.
99da57e [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
insertOverwrite
6b7545c [Yin Huai] Add a pre write check to the data source API.
a88c516 [Yin Huai] DDLParser will take a parsering function to take care CTAS 
statements.


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

Branch: refs/heads/master
Commit: 5b6cd65cd611b1a46a7d5eb33139c6224b96264e
Parents: 04b401d
Author: Yin Huai 
Authored: Mon Feb 16 15:51:59 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 15:51:59 2015 -0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  13 +-
 .../org/apache/spark/sql/DataFrameImpl.scala|   8 +-
 .../scala/org/apache/spark/sql/SQLContext.scala |   5 +-
 .../spark/sql/execution/SparkStrategies.scala   |  10 +-
 .../spark/sql/sources/DataSourceStrategy.scala  |   5 +-
 .../org/apache/spark/sql/sources/ddl.scala  |  29 ++-
 .../org/apache/spark/sql/sources/rules.scala|  72 +-
 .../spark/sql/parquet/ParquetQuerySuite.scala   |  13 +-
 .../sql/sources/CreateTableAsSelectSuite.scala  |  28 +++
 .../spark/sql/sources/DataSourceTest.scala  |   3 +-
 .../spark/sql/sources/InsertIntoSuite.scala | 176 ---
 .../apache/spark/sql/sources/InsertSuite.scala  | 218 +++
 .../org/apache/spark/sql/hive/HiveContext.scala |  12 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |   2 +-
 .../apache/spark/sql/hive/HiveStrategies.scala  |   8 +-
 15 files changed, 371 insertions(+), 231 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5b6cd65c/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 aa4320b..fc37b8c 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
@@ -50,7 +50,13 @@ class Analyzer(catalog: Catalog,
   /**
* Override to provide additional rules for the "Resolution" batch.
*/
-  val extendedRules: Seq[Rule[LogicalPlan]] = Nil
+  val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil
+
+  /**
+   * Override to provide additional rules for the "Check Analysis" batch.
+   * These rules will be evaluated after our built-in check rules.
+   */
+  val extendedCheckRules: Seq[Rule[LogicalPlan]] = Nil
 
   lazy val batches: Seq[Batch] = Seq(
 Batch("Resolution", fixedPoint,
@@ -64,9 +70,10 @@ class Analyzer(catalog: Catalog,
   UnresolvedHavingClauseAttributes ::
   TrimGroupingAliases ::
   typeCoercionRules ++
-  extendedRules : _*),
+  extendedResolutionRules : _*),
 Batch("Check Analysis", Once,
-  CheckResolution),
+  CheckResolution +:
+  extendedCheckRules: _*),
 Batch("Remove SubQueries", fixedPoint,
   EliminateSubQueries)
   )

http://git-wip-us.apache.org/repos/asf/spark/blob/5b6cd65c/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
index 500e3c9..3c1cf8d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
@@ -67,7 +67,11 @@ private[sql] class DataFrameImpl protected[sql](
   @transient protected[sql] override val logicalPlan: LogicalPlan = 
queryExecution.logical match {
 // For various commands (like DDL) and queries with side effects, we force 
query optimization to
 // happen right away to let these side effects take place eagerly.
-case _: Command | _: I

spark git commit: [SPARK-5746][SQL] Check invalid cases for the write path of data source API

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 ad8fd4fb3 -> 419865475


[SPARK-5746][SQL] Check invalid cases for the write path of data source API

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

liancheng marmbrus

Author: Yin Huai 

Closes #4617 from yhuai/insertOverwrite and squashes the following commits:

8e3019d [Yin Huai] Fix compilation error.
499e8e7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
insertOverwrite
e76e85a [Yin Huai] Address comments.
ac31b3c [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
insertOverwrite
f30bdad [Yin Huai] Use toDF.
99da57e [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
insertOverwrite
6b7545c [Yin Huai] Add a pre write check to the data source API.
a88c516 [Yin Huai] DDLParser will take a parsering function to take care CTAS 
statements.

(cherry picked from commit 5b6cd65cd611b1a46a7d5eb33139c6224b96264e)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 4198654758ba262177010c4651d8c0782576ff67
Parents: ad8fd4f
Author: Yin Huai 
Authored: Mon Feb 16 15:51:59 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 15:52:07 2015 -0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  13 +-
 .../org/apache/spark/sql/DataFrameImpl.scala|   8 +-
 .../scala/org/apache/spark/sql/SQLContext.scala |   5 +-
 .../spark/sql/execution/SparkStrategies.scala   |  10 +-
 .../spark/sql/sources/DataSourceStrategy.scala  |   5 +-
 .../org/apache/spark/sql/sources/ddl.scala  |  29 ++-
 .../org/apache/spark/sql/sources/rules.scala|  72 +-
 .../spark/sql/parquet/ParquetQuerySuite.scala   |  13 +-
 .../sql/sources/CreateTableAsSelectSuite.scala  |  28 +++
 .../spark/sql/sources/DataSourceTest.scala  |   3 +-
 .../spark/sql/sources/InsertIntoSuite.scala | 176 ---
 .../apache/spark/sql/sources/InsertSuite.scala  | 218 +++
 .../org/apache/spark/sql/hive/HiveContext.scala |  12 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |   2 +-
 .../apache/spark/sql/hive/HiveStrategies.scala  |   8 +-
 15 files changed, 371 insertions(+), 231 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/41986547/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 aa4320b..fc37b8c 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
@@ -50,7 +50,13 @@ class Analyzer(catalog: Catalog,
   /**
* Override to provide additional rules for the "Resolution" batch.
*/
-  val extendedRules: Seq[Rule[LogicalPlan]] = Nil
+  val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil
+
+  /**
+   * Override to provide additional rules for the "Check Analysis" batch.
+   * These rules will be evaluated after our built-in check rules.
+   */
+  val extendedCheckRules: Seq[Rule[LogicalPlan]] = Nil
 
   lazy val batches: Seq[Batch] = Seq(
 Batch("Resolution", fixedPoint,
@@ -64,9 +70,10 @@ class Analyzer(catalog: Catalog,
   UnresolvedHavingClauseAttributes ::
   TrimGroupingAliases ::
   typeCoercionRules ++
-  extendedRules : _*),
+  extendedResolutionRules : _*),
 Batch("Check Analysis", Once,
-  CheckResolution),
+  CheckResolution +:
+  extendedCheckRules: _*),
 Batch("Remove SubQueries", fixedPoint,
   EliminateSubQueries)
   )

http://git-wip-us.apache.org/repos/asf/spark/blob/41986547/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
index 9eb0c13..bc19210 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala
@@ -67,7 +67,11 @@ private[sql] class DataFrameImpl protected[sql](
   @transient protected[sql] override val logicalPlan: LogicalPlan = 
queryExecution.logical match {
 // For various commands (like DDL) and queries with side effects, we force 
query optimization to
 // h

spark git commit: [SPARK-5839][SQL]HiveMetastoreCatalog does not recognize table names and aliases of data source tables.

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 5b6cd65cd -> f3ff1eb29


[SPARK-5839][SQL]HiveMetastoreCatalog does not recognize table names and 
aliases of data source tables.

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

Author: Yin Huai 

Closes #4626 from yhuai/SPARK-5839 and squashes the following commits:

f779d85 [Yin Huai] Use subqeury to wrap replaced ParquetRelation.
2695f13 [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
SPARK-5839
f1ba6ca [Yin Huai] Address comment.
2c7fa08 [Yin Huai] Use Subqueries to wrap a data source table.


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

Branch: refs/heads/master
Commit: f3ff1eb2985ff3e1567645b898f6b42e4b01f237
Parents: 5b6cd65
Author: Yin Huai 
Authored: Mon Feb 16 15:54:01 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 15:54:01 2015 -0800

--
 .../spark/sql/parquet/ParquetQuerySuite.scala   |  5 +--
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 18 +--
 .../sql/hive/MetastoreDataSourcesSuite.scala| 34 
 3 files changed, 53 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f3ff1eb2/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
index 9318c15..8b4d05e 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
@@ -30,9 +30,10 @@ import org.apache.spark.sql.test.TestSQLContext._
 class ParquetQuerySuiteBase extends QueryTest with ParquetTest {
   val sqlContext = TestSQLContext
 
-  test("simple projection") {
+  test("simple select queries") {
 withParquetTable((0 until 10).map(i => (i, i.toString)), "t") {
-  checkAnswer(sql("SELECT _1 FROM t"), (0 until 10).map(Row.apply(_)))
+  checkAnswer(sql("SELECT _1 FROM t where t._1 > 5"), (6 until 
10).map(Row.apply(_)))
+  checkAnswer(sql("SELECT _1 FROM t as tmp where tmp._1 < 5"), (0 until 
5).map(Row.apply(_)))
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f3ff1eb2/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 72211fe..87bc9fe 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
@@ -160,7 +160,15 @@ private[hive] class HiveMetastoreCatalog(hive: 
HiveContext) extends Catalog with
 }
 
 if (table.getProperty("spark.sql.sources.provider") != null) {
-  cachedDataSourceTables(QualifiedTableName(databaseName, 
tblName).toLowerCase)
+  val dataSourceTable =
+cachedDataSourceTables(QualifiedTableName(databaseName, 
tblName).toLowerCase)
+  // Then, if alias is specified, wrap the table with a Subquery using the 
alias.
+  // Othersie, wrap the table with a Subquery using the table name.
+  val withAlias =
+alias.map(a => Subquery(a, dataSourceTable)).getOrElse(
+  Subquery(tableIdent.last, dataSourceTable))
+
+  withAlias
 } else if (table.isView) {
   // if the unresolved relation is from hive view
   // parse the text into logic node.
@@ -433,7 +441,13 @@ private[hive] class HiveMetastoreCatalog(hive: 
HiveContext) extends Catalog with
 val attributedRewrites = 
AttributeMap(relation.output.zip(parquetRelation.output))
 
 lastPlan.transformUp {
-  case r: MetastoreRelation if r == relation => parquetRelation
+  case r: MetastoreRelation if r == relation => {
+val withAlias =
+  r.alias.map(a => Subquery(a, parquetRelation)).getOrElse(
+Subquery(r.tableName, parquetRelation))
+
+withAlias
+  }
   case other => other.transformExpressions {
 case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, 
a)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/f3ff1eb2/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/o

spark git commit: [SPARK-5839][SQL]HiveMetastoreCatalog does not recognize table names and aliases of data source tables.

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 419865475 -> a15a0a02c


[SPARK-5839][SQL]HiveMetastoreCatalog does not recognize table names and 
aliases of data source tables.

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

Author: Yin Huai 

Closes #4626 from yhuai/SPARK-5839 and squashes the following commits:

f779d85 [Yin Huai] Use subqeury to wrap replaced ParquetRelation.
2695f13 [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
SPARK-5839
f1ba6ca [Yin Huai] Address comment.
2c7fa08 [Yin Huai] Use Subqueries to wrap a data source table.

(cherry picked from commit f3ff1eb2985ff3e1567645b898f6b42e4b01f237)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: a15a0a02c654cd70bb84f6d610e1c178c9cb98a6
Parents: 4198654
Author: Yin Huai 
Authored: Mon Feb 16 15:54:01 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 15:54:13 2015 -0800

--
 .../spark/sql/parquet/ParquetQuerySuite.scala   |  5 +--
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 18 +--
 .../sql/hive/MetastoreDataSourcesSuite.scala| 34 
 3 files changed, 53 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a15a0a02/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
index 9318c15..8b4d05e 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
@@ -30,9 +30,10 @@ import org.apache.spark.sql.test.TestSQLContext._
 class ParquetQuerySuiteBase extends QueryTest with ParquetTest {
   val sqlContext = TestSQLContext
 
-  test("simple projection") {
+  test("simple select queries") {
 withParquetTable((0 until 10).map(i => (i, i.toString)), "t") {
-  checkAnswer(sql("SELECT _1 FROM t"), (0 until 10).map(Row.apply(_)))
+  checkAnswer(sql("SELECT _1 FROM t where t._1 > 5"), (6 until 
10).map(Row.apply(_)))
+  checkAnswer(sql("SELECT _1 FROM t as tmp where tmp._1 < 5"), (0 until 
5).map(Row.apply(_)))
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a15a0a02/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 72211fe..87bc9fe 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
@@ -160,7 +160,15 @@ private[hive] class HiveMetastoreCatalog(hive: 
HiveContext) extends Catalog with
 }
 
 if (table.getProperty("spark.sql.sources.provider") != null) {
-  cachedDataSourceTables(QualifiedTableName(databaseName, 
tblName).toLowerCase)
+  val dataSourceTable =
+cachedDataSourceTables(QualifiedTableName(databaseName, 
tblName).toLowerCase)
+  // Then, if alias is specified, wrap the table with a Subquery using the 
alias.
+  // Othersie, wrap the table with a Subquery using the table name.
+  val withAlias =
+alias.map(a => Subquery(a, dataSourceTable)).getOrElse(
+  Subquery(tableIdent.last, dataSourceTable))
+
+  withAlias
 } else if (table.isView) {
   // if the unresolved relation is from hive view
   // parse the text into logic node.
@@ -433,7 +441,13 @@ private[hive] class HiveMetastoreCatalog(hive: 
HiveContext) extends Catalog with
 val attributedRewrites = 
AttributeMap(relation.output.zip(parquetRelation.output))
 
 lastPlan.transformUp {
-  case r: MetastoreRelation if r == relation => parquetRelation
+  case r: MetastoreRelation if r == relation => {
+val withAlias =
+  r.alias.map(a => Subquery(a, parquetRelation)).getOrElse(
+Subquery(r.tableName, parquetRelation))
+
+withAlias
+  }
   case other => other.transformExpressions {
 case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, 
a)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/a15a0a02/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala

spark git commit: [SQL] Optimize arithmetic and predicate operators

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master f3ff1eb29 -> cb6c48c87


[SQL] Optimize arithmetic and predicate operators

Existing implementation of arithmetic operators and BinaryComparison operators 
have redundant type checking codes, e.g.:
Expression.n2 is used by Add/Subtract/Multiply.
(1) n2 always checks left.dataType == right.dataType. However, this checking 
should be done once when we resolve expression types;
(2) n2 requires dataType is a NumericType. This can be done once.

This PR optimizes arithmetic and predicate operators by removing such redundant 
type-checking codes.

Some preliminary benchmarking on 10G TPC-H data over 5 r3.2xlarge EC2 machines 
shows that this PR can reduce the query time by 5.5% to 11%.
The benchmark queries follow the template below, where OP is 
plus/minus/times/divide/remainder/bitwise and/bitwise or/bitwise xor.

SELECT l_returnflag,  l_linestatus, SUM(l_quantity OP cnt1), SUM(l_quantity OP 
cnt2), , SUM(l_quantity OP cnt700)
FROM (
SELECT l_returnflag, l_linestatus, l_quantity, 1 AS cnt1, 2 AS cnt2, ..., 
700 AS cnt700
FROM lineitem
WHERE l_shipdate <= '1998-09-01'
)
GROUP BY l_returnflag,  l_linestatus;

Author: kai 

Closes #4472 from kai-zeng/arithmetic-optimize and squashes the following 
commits:

fef0cf1 [kai] Merge branch 'master' of github.com:apache/spark into 
arithmetic-optimize
4b3a1bb [kai] chmod a-x
5a41e49 [kai] chmod a-x Expression.scala
cb37c94 [kai] rebase onto spark master
7f6e968 [kai] chmod 100755 -> 100644
6cddb46 [kai] format
7490dbc [kai] fix unresolved-expression exception for EqualTo
9c40bc0 [kai] fix bitwisenot
3cbd363 [kai] clean up test code
ca47801 [kai] override evalInternal for bitwise ops
8fa84a1 [kai] add bitwise or and xor
6892fc4 [kai] revert override evalInternal
f8eba24 [kai] override evalInternal
31ccdd4 [kai] rewrite all bitwise op and remove evalInternal
86297e2 [kai] generalized
cb92ae1 [kai] bitwise-and: override eval
97a7d6c [kai] bitwise-and: override evalInternal using and func
0906c39 [kai] add bitwise test
62abbbc [kai] clean up predicate and arithmetic
b34d58d [kai] add caching and benmark option
12c5b32 [kai] override eval
1cd7571 [kai] fix sqrt and maxof
03fd0c3 [kai] fix predicate
16fd84c [kai] optimize + - * / % -(unary) abs < > <= >=
fd95823 [kai] remove unnecessary type checking
24d062f [kai] test suite


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

Branch: refs/heads/master
Commit: cb6c48c874af2bd78ee73c1dc8a44fd28ecc0991
Parents: f3ff1eb
Author: kai 
Authored: Mon Feb 16 15:58:05 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 15:58:05 2015 -0800

--
 .../apache/spark/sql/catalyst/SqlParser.scala   |   0
 .../apache/spark/sql/catalyst/dsl/package.scala |   0
 .../sql/catalyst/expressions/Expression.scala   | 200 ---
 .../sql/catalyst/expressions/aggregates.scala   |   0
 .../sql/catalyst/expressions/arithmetic.scala   | 243 ++-
 .../sql/catalyst/expressions/predicates.scala   | 107 +++-
 .../org/apache/spark/sql/types/Metadata.scala   |   0
 .../spark/sql/catalyst/util/MetadataSuite.scala |   0
 .../apache/spark/sql/execution/Aggregate.scala  |   0
 .../hive/thriftserver/SparkSQLCLIDriver.scala   |   0
 10 files changed, 290 insertions(+), 260 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cb6c48c8/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
old mode 100755
new mode 100644

http://git-wip-us.apache.org/repos/asf/spark/blob/cb6c48c8/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
old mode 100755
new mode 100644

http://git-wip-us.apache.org/repos/asf/spark/blob/cb6c48c8/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
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index c32a4b8..6ad39b8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ 
b/sql/cata

spark git commit: [SQL] Optimize arithmetic and predicate operators

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 a15a0a02c -> 639a3c2fd


[SQL] Optimize arithmetic and predicate operators

Existing implementation of arithmetic operators and BinaryComparison operators 
have redundant type checking codes, e.g.:
Expression.n2 is used by Add/Subtract/Multiply.
(1) n2 always checks left.dataType == right.dataType. However, this checking 
should be done once when we resolve expression types;
(2) n2 requires dataType is a NumericType. This can be done once.

This PR optimizes arithmetic and predicate operators by removing such redundant 
type-checking codes.

Some preliminary benchmarking on 10G TPC-H data over 5 r3.2xlarge EC2 machines 
shows that this PR can reduce the query time by 5.5% to 11%.
The benchmark queries follow the template below, where OP is 
plus/minus/times/divide/remainder/bitwise and/bitwise or/bitwise xor.

SELECT l_returnflag,  l_linestatus, SUM(l_quantity OP cnt1), SUM(l_quantity OP 
cnt2), , SUM(l_quantity OP cnt700)
FROM (
SELECT l_returnflag, l_linestatus, l_quantity, 1 AS cnt1, 2 AS cnt2, ..., 
700 AS cnt700
FROM lineitem
WHERE l_shipdate <= '1998-09-01'
)
GROUP BY l_returnflag,  l_linestatus;

Author: kai 

Closes #4472 from kai-zeng/arithmetic-optimize and squashes the following 
commits:

fef0cf1 [kai] Merge branch 'master' of github.com:apache/spark into 
arithmetic-optimize
4b3a1bb [kai] chmod a-x
5a41e49 [kai] chmod a-x Expression.scala
cb37c94 [kai] rebase onto spark master
7f6e968 [kai] chmod 100755 -> 100644
6cddb46 [kai] format
7490dbc [kai] fix unresolved-expression exception for EqualTo
9c40bc0 [kai] fix bitwisenot
3cbd363 [kai] clean up test code
ca47801 [kai] override evalInternal for bitwise ops
8fa84a1 [kai] add bitwise or and xor
6892fc4 [kai] revert override evalInternal
f8eba24 [kai] override evalInternal
31ccdd4 [kai] rewrite all bitwise op and remove evalInternal
86297e2 [kai] generalized
cb92ae1 [kai] bitwise-and: override eval
97a7d6c [kai] bitwise-and: override evalInternal using and func
0906c39 [kai] add bitwise test
62abbbc [kai] clean up predicate and arithmetic
b34d58d [kai] add caching and benmark option
12c5b32 [kai] override eval
1cd7571 [kai] fix sqrt and maxof
03fd0c3 [kai] fix predicate
16fd84c [kai] optimize + - * / % -(unary) abs < > <= >=
fd95823 [kai] remove unnecessary type checking
24d062f [kai] test suite

(cherry picked from commit cb6c48c874af2bd78ee73c1dc8a44fd28ecc0991)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 639a3c2fdf302a34ad5f62199abee2b96023669b
Parents: a15a0a0
Author: kai 
Authored: Mon Feb 16 15:58:05 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 15:58:18 2015 -0800

--
 .../apache/spark/sql/catalyst/SqlParser.scala   |   0
 .../apache/spark/sql/catalyst/dsl/package.scala |   0
 .../sql/catalyst/expressions/Expression.scala   | 200 ---
 .../sql/catalyst/expressions/aggregates.scala   |   0
 .../sql/catalyst/expressions/arithmetic.scala   | 243 ++-
 .../sql/catalyst/expressions/predicates.scala   | 107 +++-
 .../org/apache/spark/sql/types/Metadata.scala   |   0
 .../spark/sql/catalyst/util/MetadataSuite.scala |   0
 .../apache/spark/sql/execution/Aggregate.scala  |   0
 .../hive/thriftserver/SparkSQLCLIDriver.scala   |   0
 10 files changed, 290 insertions(+), 260 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/639a3c2f/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
old mode 100755
new mode 100644

http://git-wip-us.apache.org/repos/asf/spark/blob/639a3c2f/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
old mode 100755
new mode 100644

http://git-wip-us.apache.org/repos/asf/spark/blob/639a3c2f/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
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index c32a4b8..6ad39b8 10064

spark git commit: [SPARK-4865][SQL]Include temporary tables in SHOW TABLES

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master cb6c48c87 -> e189cbb05


[SPARK-4865][SQL]Include temporary tables in SHOW TABLES

This PR adds a `ShowTablesCommand` to support `SHOW TABLES [IN databaseName]` 
SQL command. The result of `SHOW TABLE` has two columns, `tableName` and 
`isTemporary`. For temporary tables, the value of `isTemporary` column will be 
`false`.

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

Author: Yin Huai 

Closes #4618 from yhuai/showTablesCommand and squashes the following commits:

0c09791 [Yin Huai] Use ShowTablesCommand.
85ee76d [Yin Huai] Since SHOW TABLES is not a Hive native command any more and 
we will not see "OK" (originally generated by Hive's driver), use SHOW 
DATABASES in the test.
94bacac [Yin Huai] Add SHOW TABLES to the list of noExplainCommands.
d71ed09 [Yin Huai] Fix test.
a4a6ec3 [Yin Huai] Add SHOW TABLE command.


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

Branch: refs/heads/master
Commit: e189cbb052d59eb499dd4312403925fdd72f5718
Parents: cb6c48c
Author: Yin Huai 
Authored: Mon Feb 16 15:59:23 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 15:59:23 2015 -0800

--
 .../scala/org/apache/spark/sql/SQLContext.scala |  4 +-
 .../org/apache/spark/sql/SparkSQLParser.scala   | 12 -
 .../apache/spark/sql/execution/commands.scala   | 34 ++-
 .../org/apache/spark/sql/ListTablesSuite.scala  | 37 ++--
 .../spark/sql/hive/thriftserver/CliSuite.scala  |  2 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 10 -
 .../org/apache/spark/sql/hive/HiveQl.scala  |  2 +-
 .../apache/spark/sql/hive/ListTablesSuite.scala | 46 +++-
 .../sql/hive/execution/HiveQuerySuite.scala | 14 +++---
 9 files changed, 111 insertions(+), 50 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e189cbb0/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index d08c2d1..aa0fc3e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -794,7 +794,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
* indicating if a table is a temporary one or not).
*/
   def tables(): DataFrame = {
-createDataFrame(catalog.getTables(None)).toDF("tableName", "isTemporary")
+DataFrame(this, ShowTablesCommand(None))
   }
 
   /**
@@ -803,7 +803,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
* indicating if a table is a temporary one or not).
*/
   def tables(databaseName: String): DataFrame = {
-createDataFrame(catalog.getTables(Some(databaseName))).toDF("tableName", 
"isTemporary")
+DataFrame(this, ShowTablesCommand(Some(databaseName)))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/e189cbb0/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
index f1a4053..00e19da 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
@@ -23,7 +23,7 @@ import scala.util.parsing.combinator.RegexParsers
 import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
 import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.{UncacheTableCommand, CacheTableCommand, 
SetCommand}
+import org.apache.spark.sql.execution.{ShowTablesCommand, UncacheTableCommand, 
CacheTableCommand, SetCommand}
 import org.apache.spark.sql.types.StringType
 
 
@@ -57,12 +57,15 @@ private[sql] class SparkSQLParser(fallback: String => 
LogicalPlan) extends Abstr
 
   protected val AS  = Keyword("AS")
   protected val CACHE   = Keyword("CACHE")
+  protected val IN  = Keyword("IN")
   protected val LAZY= Keyword("LAZY")
   protected val SET = Keyword("SET")
+  protected val SHOW= Keyword("SHOW")
   protected val TABLE   = Keyword("TABLE")
+  protected val TABLES  = Keyword("TABLES")
   protected val UNCACHE = Keyword("UNCACHE")
 
-  override protected lazy val start: Parser[LogicalPlan] = cache | uncache | 
set | others
+  override protected lazy val start: Parser[L

spark git commit: [SPARK-4865][SQL]Include temporary tables in SHOW TABLES

2015-02-16 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 639a3c2fd -> 8a94bf76b


[SPARK-4865][SQL]Include temporary tables in SHOW TABLES

This PR adds a `ShowTablesCommand` to support `SHOW TABLES [IN databaseName]` 
SQL command. The result of `SHOW TABLE` has two columns, `tableName` and 
`isTemporary`. For temporary tables, the value of `isTemporary` column will be 
`false`.

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

Author: Yin Huai 

Closes #4618 from yhuai/showTablesCommand and squashes the following commits:

0c09791 [Yin Huai] Use ShowTablesCommand.
85ee76d [Yin Huai] Since SHOW TABLES is not a Hive native command any more and 
we will not see "OK" (originally generated by Hive's driver), use SHOW 
DATABASES in the test.
94bacac [Yin Huai] Add SHOW TABLES to the list of noExplainCommands.
d71ed09 [Yin Huai] Fix test.
a4a6ec3 [Yin Huai] Add SHOW TABLE command.

(cherry picked from commit e189cbb052d59eb499dd4312403925fdd72f5718)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 8a94bf76b883ec59b6142fab781a05c1ee985262
Parents: 639a3c2
Author: Yin Huai 
Authored: Mon Feb 16 15:59:23 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 16 15:59:41 2015 -0800

--
 .../scala/org/apache/spark/sql/SQLContext.scala |  4 +-
 .../org/apache/spark/sql/SparkSQLParser.scala   | 12 -
 .../apache/spark/sql/execution/commands.scala   | 34 ++-
 .../org/apache/spark/sql/ListTablesSuite.scala  | 37 ++--
 .../spark/sql/hive/thriftserver/CliSuite.scala  |  2 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 10 -
 .../org/apache/spark/sql/hive/HiveQl.scala  |  2 +-
 .../apache/spark/sql/hive/ListTablesSuite.scala | 46 +++-
 .../sql/hive/execution/HiveQuerySuite.scala | 14 +++---
 9 files changed, 111 insertions(+), 50 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8a94bf76/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index d08c2d1..aa0fc3e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -794,7 +794,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
* indicating if a table is a temporary one or not).
*/
   def tables(): DataFrame = {
-createDataFrame(catalog.getTables(None)).toDF("tableName", "isTemporary")
+DataFrame(this, ShowTablesCommand(None))
   }
 
   /**
@@ -803,7 +803,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
* indicating if a table is a temporary one or not).
*/
   def tables(databaseName: String): DataFrame = {
-createDataFrame(catalog.getTables(Some(databaseName))).toDF("tableName", 
"isTemporary")
+DataFrame(this, ShowTablesCommand(Some(databaseName)))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/8a94bf76/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
index f1a4053..00e19da 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
@@ -23,7 +23,7 @@ import scala.util.parsing.combinator.RegexParsers
 import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
 import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.{UncacheTableCommand, CacheTableCommand, 
SetCommand}
+import org.apache.spark.sql.execution.{ShowTablesCommand, UncacheTableCommand, 
CacheTableCommand, SetCommand}
 import org.apache.spark.sql.types.StringType
 
 
@@ -57,12 +57,15 @@ private[sql] class SparkSQLParser(fallback: String => 
LogicalPlan) extends Abstr
 
   protected val AS  = Keyword("AS")
   protected val CACHE   = Keyword("CACHE")
+  protected val IN  = Keyword("IN")
   protected val LAZY= Keyword("LAZY")
   protected val SET = Keyword("SET")
+  protected val SHOW= Keyword("SHOW")
   protected val TABLE   = Keyword("TABLE")
+  protected val TABLES  = Keyword("TABLES")
   protected val UNCACHE = Keyword("UNCACHE")
 
-  override protected lazy 

[2/2] spark git commit: [SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation

2015-02-17 Thread marmbrus
[SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation

Author: Michael Armbrust 

Closes #4642 from marmbrus/docs and squashes the following commits:

d291c34 [Michael Armbrust] python tests
9be66e3 [Michael Armbrust] comments
d56afc2 [Michael Armbrust] fix style
f004747 [Michael Armbrust] fix build
c4a907b [Michael Armbrust] fix tests
42e2b73 [Michael Armbrust] [SQL] Documentation / API Clean-up.


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

Branch: refs/heads/master
Commit: c74b07fa94a8da50437d952ae05cf6ac70fbb93e
Parents: c76da36
Author: Michael Armbrust 
Authored: Tue Feb 17 10:21:17 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 10:21:17 2015 -0800

--
 project/SparkBuild.scala|  12 +-
 python/pyspark/sql/context.py   |  28 +--
 .../org/apache/spark/sql/jdbc/JDBCUtils.java|  59 --
 .../scala/org/apache/spark/sql/DataFrame.scala  | 153 +-
 .../org/apache/spark/sql/DataFrameImpl.scala|  33 ++-
 .../apache/spark/sql/ExperimentalMethods.scala  |   5 +
 .../apache/spark/sql/IncomputableColumn.scala   |   4 +
 .../scala/org/apache/spark/sql/SQLContext.scala | 200 +++
 .../apache/spark/sql/UserDefinedFunction.scala  |   3 +-
 .../org/apache/spark/sql/api/package.scala  |  23 +++
 .../apache/spark/sql/execution/commands.scala   |   2 +-
 .../spark/sql/execution/debug/package.scala |  10 +-
 .../spark/sql/jdbc/JavaJDBCTrampoline.scala |  30 ---
 .../scala/org/apache/spark/sql/jdbc/jdbc.scala  |  74 ++-
 .../sql/parquet/ParquetTableOperations.scala|   4 +-
 .../apache/spark/sql/parquet/ParquetTest.scala  |   4 +-
 .../apache/spark/sql/parquet/newParquet.scala   |   6 +-
 .../spark/sql/parquet/timestamp/NanoTime.scala  |   2 +-
 .../org/apache/spark/sql/sources/ddl.scala  |   4 +-
 .../org/apache/spark/sql/jdbc/JavaJDBCTest.java | 102 --
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala   |   7 +-
 .../apache/spark/sql/jdbc/JDBCWriteSuite.scala  |  20 +-
 .../spark/sql/jdbc/MySQLIntegration.scala   |  14 +-
 .../spark/sql/jdbc/PostgresIntegration.scala|   4 +-
 .../execution/DescribeHiveTableCommand.scala|   4 +-
 .../spark/sql/hive/execution/commands.scala |   8 +
 .../spark/sql/hive/execution/package.scala  |  25 +++
 .../org/apache/spark/sql/hive/package.scala |  10 +
 .../sql/hive/parquet/FakeParquetSerDe.scala |  56 --
 .../org/apache/spark/sql/hive/Shim12.scala  |   9 +-
 .../org/apache/spark/sql/hive/Shim13.scala  |   9 +-
 31 files changed, 501 insertions(+), 423 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c74b07fa/project/SparkBuild.scala
--
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 8fb1239..e4b1b96 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -361,9 +361,16 @@ object Unidoc {
 publish := {},
 
 unidocProjectFilter in(ScalaUnidoc, unidoc) :=
-  inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, 
catalyst, streamingFlumeSink, yarn),
+  inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, 
streamingFlumeSink, yarn),
 unidocProjectFilter in(JavaUnidoc, unidoc) :=
-  inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, 
tools, catalyst, streamingFlumeSink, yarn),
+  inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, 
tools, streamingFlumeSink, yarn),
+
+// Skip actual catalyst, but include the subproject.
+// Catalyst is not public API and contains quasiquotes which break 
scaladoc.
+unidocAllSources in (ScalaUnidoc, unidoc) := {
+  (unidocAllSources in (ScalaUnidoc, unidoc)).value
+.map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst")))
+},
 
 // Skip class names containing $ and some internal packages in Javadocs
 unidocAllSources in (JavaUnidoc, unidoc) := {
@@ -376,6 +383,7 @@ object Unidoc {
 .map(_.filterNot(_.getCanonicalPath.contains("executor")))
 .map(_.filterNot(_.getCanonicalPath.contains("python")))
 .map(_.filterNot(_.getCanonicalPath.contains("collection")))
+.map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst")))
 },
 
 // Javadoc options: create a window title, and group key packages on index 
page

http://git-wip-us.apache.org/repos/asf/spark/blob/c74b07fa/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/contex

spark git commit: [SPARK-5859] [PySpark] [SQL] fix DataFrame Python API

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master c74b07fa9 -> d8adefefc


[SPARK-5859] [PySpark] [SQL] fix DataFrame Python API

1. added explain()
2. add isLocal()
3. do not call show() in __repl__
4. add foreach() and foreachPartition()
5. add distinct()
6. fix functions.col()/column()/lit()
7. fix unit tests in sql/functions.py
8. fix unicode in showString()

Author: Davies Liu 

Closes #4645 from davies/df6 and squashes the following commits:

6b46a2c [Davies Liu] fix DataFrame Python API


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

Branch: refs/heads/master
Commit: d8adefefcc2a4af32295440ed1d4917a6968f017
Parents: c74b07f
Author: Davies Liu 
Authored: Tue Feb 17 10:22:48 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 10:22:48 2015 -0800

--
 python/pyspark/sql/dataframe.py | 65 ++--
 python/pyspark/sql/functions.py | 12 +++
 2 files changed, 59 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d8adefef/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 28a59e7..8417240 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -238,6 +238,22 @@ class DataFrame(object):
 """
 print (self._jdf.schema().treeString())
 
+def explain(self, extended=False):
+"""
+Prints the plans (logical and physical) to the console for
+debugging purpose.
+
+If extended is False, only prints the physical plan.
+"""
+self._jdf.explain(extended)
+
+def isLocal(self):
+"""
+Returns True if the `collect` and `take` methods can be run locally
+(without any Spark executors).
+"""
+return self._jdf.isLocal()
+
 def show(self):
 """
 Print the first 20 rows.
@@ -247,14 +263,12 @@ class DataFrame(object):
 2   Alice
 5   Bob
 >>> df
-age name
-2   Alice
-5   Bob
+DataFrame[age: int, name: string]
 """
-print (self)
+print self._jdf.showString().encode('utf8', 'ignore')
 
 def __repr__(self):
-return self._jdf.showString()
+return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes))
 
 def count(self):
 """Return the number of elements in this RDD.
@@ -336,6 +350,8 @@ class DataFrame(object):
 """
 Return a new RDD by applying a function to each partition.
 
+It's a shorthand for df.rdd.mapPartitions()
+
 >>> rdd = sc.parallelize([1, 2, 3, 4], 4)
 >>> def f(iterator): yield 1
 >>> rdd.mapPartitions(f).sum()
@@ -343,6 +359,31 @@ class DataFrame(object):
 """
 return self.rdd.mapPartitions(f, preservesPartitioning)
 
+def foreach(self, f):
+"""
+Applies a function to all rows of this DataFrame.
+
+It's a shorthand for df.rdd.foreach()
+
+>>> def f(person):
+... print person.name
+>>> df.foreach(f)
+"""
+return self.rdd.foreach(f)
+
+def foreachPartition(self, f):
+"""
+Applies a function to each partition of this DataFrame.
+
+It's a shorthand for df.rdd.foreachPartition()
+
+>>> def f(people):
+... for person in people:
+... print person.name
+>>> df.foreachPartition(f)
+"""
+return self.rdd.foreachPartition(f)
+
 def cache(self):
 """ Persist with the default storage level (C{MEMORY_ONLY_SER}).
 """
@@ -377,8 +418,13 @@ class DataFrame(object):
 """ Return a new :class:`DataFrame` that has exactly `numPartitions`
 partitions.
 """
-rdd = self._jdf.repartition(numPartitions, None)
-return DataFrame(rdd, self.sql_ctx)
+return DataFrame(self._jdf.repartition(numPartitions, None), 
self.sql_ctx)
+
+def distinct(self):
+"""
+Return a new :class:`DataFrame` containing the distinct rows in this 
DataFrame.
+"""
+return DataFrame(self._jdf.distinct(), self.sql_ctx)
 
 def sample(self, withReplacement, fraction, seed=None):
 """
@@ -957,10 +1003,7 @@ class Column(DataFrame):
 return Column(jc, self.sql_ctx)
 
 def __repr__(self):
-if self._jdf.isComputable():
-return self._jdf.samples()
-else:
-return 'Column<%s>' % self._jdf.toString()
+return 'Column<%s>' % self._jdf.toString().encode('utf8')
 
  

[1/2] spark git commit: [SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master c76da36c2 -> c74b07fa9


http://git-wip-us.apache.org/repos/asf/spark/blob/c74b07fa/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala
index 89920f2..4f38110 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala
@@ -143,7 +143,7 @@ class MySQLDatabase {
   }
 
   test("Basic test") {
-val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "tbl")
+val rdd = TestSQLContext.jdbc(url(ip, "foo"), "tbl")
 val rows = rdd.collect
 assert(rows.length == 2)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -153,7 +153,7 @@ class MySQLDatabase {
   }
 
   test("Numeric types") {
-val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers")
+val rdd = TestSQLContext.jdbc(url(ip, "foo"), "numbers")
 val rows = rdd.collect
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -181,7 +181,7 @@ class MySQLDatabase {
   }
 
   test("Date types") {
-val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates")
+val rdd = TestSQLContext.jdbc(url(ip, "foo"), "dates")
 val rows = rdd.collect
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -199,7 +199,7 @@ class MySQLDatabase {
   }
 
   test("String types") {
-val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings")
+val rdd = TestSQLContext.jdbc(url(ip, "foo"), "strings")
 val rows = rdd.collect
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -225,9 +225,9 @@ class MySQLDatabase {
   }
 
   test("Basic write test") {
-val rdd1 = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers")
-val rdd2 = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates")
-val rdd3 = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings")
+val rdd1 = TestSQLContext.jdbc(url(ip, "foo"), "numbers")
+val rdd2 = TestSQLContext.jdbc(url(ip, "foo"), "dates")
+val rdd3 = TestSQLContext.jdbc(url(ip, "foo"), "strings")
 rdd1.createJDBCTable(url(ip, "foo"), "numberscopy", false)
 rdd2.createJDBCTable(url(ip, "foo"), "datescopy", false)
 rdd3.createJDBCTable(url(ip, "foo"), "stringscopy", false)

http://git-wip-us.apache.org/repos/asf/spark/blob/c74b07fa/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala
index c174d7a..7b47fee 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala
@@ -113,7 +113,7 @@ class PostgresDatabase {
   }
 
   test("Type mapping for various types") {
-val rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar")
+val rdd = TestSQLContext.jdbc(url(db.ip), "public.bar")
 val rows = rdd.collect
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -142,7 +142,7 @@ class PostgresDatabase {
   }
 
   test("Basic write test") {
-val rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar")
+val rdd = TestSQLContext.jdbc(url(db.ip), "public.bar")
 rdd.createJDBCTable(url(db.ip), "public.barcopy", false)
 // Test only that it doesn't bomb out.
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/c74b07fa/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
index bfacc51..07b5a84 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
@@ -29,9 +29,9 @@ import org.apache.spark.sql.hive.HiveShim
 import org.apache.spark.sql.SQLContext
 
 /**
- * Implementation for "describe [extended] table".
- *
  * :: DeveloperApi ::
+ *
+ * Implementation for "describe [extended] table".
  */
 @DeveloperApi
 case class DescribeHiveTableCommand(

http://git-wip-us.apache.org/repos/asf/spark/blob/c74b07fa/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/

[2/2] spark git commit: [SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation

2015-02-17 Thread marmbrus
[SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation

Author: Michael Armbrust 

Closes #4642 from marmbrus/docs and squashes the following commits:

d291c34 [Michael Armbrust] python tests
9be66e3 [Michael Armbrust] comments
d56afc2 [Michael Armbrust] fix style
f004747 [Michael Armbrust] fix build
c4a907b [Michael Armbrust] fix tests
42e2b73 [Michael Armbrust] [SQL] Documentation / API Clean-up.

(cherry picked from commit c74b07fa94a8da50437d952ae05cf6ac70fbb93e)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: cd3d4158721b5c3cc35df47675f4f4d9540be6f1
Parents: 97cb568
Author: Michael Armbrust 
Authored: Tue Feb 17 10:21:17 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 10:21:33 2015 -0800

--
 project/SparkBuild.scala|  12 +-
 python/pyspark/sql/context.py   |  28 +--
 .../org/apache/spark/sql/jdbc/JDBCUtils.java|  59 --
 .../scala/org/apache/spark/sql/DataFrame.scala  | 153 +-
 .../org/apache/spark/sql/DataFrameImpl.scala|  33 ++-
 .../apache/spark/sql/ExperimentalMethods.scala  |   5 +
 .../apache/spark/sql/IncomputableColumn.scala   |   4 +
 .../scala/org/apache/spark/sql/SQLContext.scala | 200 +++
 .../apache/spark/sql/UserDefinedFunction.scala  |   3 +-
 .../org/apache/spark/sql/api/package.scala  |  23 +++
 .../apache/spark/sql/execution/commands.scala   |   2 +-
 .../spark/sql/execution/debug/package.scala |  10 +-
 .../spark/sql/jdbc/JavaJDBCTrampoline.scala |  30 ---
 .../scala/org/apache/spark/sql/jdbc/jdbc.scala  |  74 ++-
 .../sql/parquet/ParquetTableOperations.scala|   4 +-
 .../apache/spark/sql/parquet/ParquetTest.scala  |   4 +-
 .../apache/spark/sql/parquet/newParquet.scala   |   6 +-
 .../spark/sql/parquet/timestamp/NanoTime.scala  |   2 +-
 .../org/apache/spark/sql/sources/ddl.scala  |   4 +-
 .../org/apache/spark/sql/jdbc/JavaJDBCTest.java | 102 --
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala   |   7 +-
 .../apache/spark/sql/jdbc/JDBCWriteSuite.scala  |  20 +-
 .../spark/sql/jdbc/MySQLIntegration.scala   |  14 +-
 .../spark/sql/jdbc/PostgresIntegration.scala|   4 +-
 .../execution/DescribeHiveTableCommand.scala|   4 +-
 .../spark/sql/hive/execution/commands.scala |   8 +
 .../spark/sql/hive/execution/package.scala  |  25 +++
 .../org/apache/spark/sql/hive/package.scala |  10 +
 .../sql/hive/parquet/FakeParquetSerDe.scala |  56 --
 .../org/apache/spark/sql/hive/Shim12.scala  |   9 +-
 .../org/apache/spark/sql/hive/Shim13.scala  |   9 +-
 31 files changed, 501 insertions(+), 423 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cd3d4158/project/SparkBuild.scala
--
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 8fb1239..e4b1b96 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -361,9 +361,16 @@ object Unidoc {
 publish := {},
 
 unidocProjectFilter in(ScalaUnidoc, unidoc) :=
-  inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, 
catalyst, streamingFlumeSink, yarn),
+  inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, 
streamingFlumeSink, yarn),
 unidocProjectFilter in(JavaUnidoc, unidoc) :=
-  inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, 
tools, catalyst, streamingFlumeSink, yarn),
+  inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, 
tools, streamingFlumeSink, yarn),
+
+// Skip actual catalyst, but include the subproject.
+// Catalyst is not public API and contains quasiquotes which break 
scaladoc.
+unidocAllSources in (ScalaUnidoc, unidoc) := {
+  (unidocAllSources in (ScalaUnidoc, unidoc)).value
+.map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst")))
+},
 
 // Skip class names containing $ and some internal packages in Javadocs
 unidocAllSources in (JavaUnidoc, unidoc) := {
@@ -376,6 +383,7 @@ object Unidoc {
 .map(_.filterNot(_.getCanonicalPath.contains("executor")))
 .map(_.filterNot(_.getCanonicalPath.contains("python")))
 .map(_.filterNot(_.getCanonicalPath.contains("collection")))
+.map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst")))
 },
 
 // Javadoc options: create a window title, and group key packages on index 
page

http://git-wip-us.apache.org/repos/asf

[1/2] spark git commit: [SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 97cb568a2 -> cd3d41587


http://git-wip-us.apache.org/repos/asf/spark/blob/cd3d4158/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala
index 89920f2..4f38110 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala
@@ -143,7 +143,7 @@ class MySQLDatabase {
   }
 
   test("Basic test") {
-val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "tbl")
+val rdd = TestSQLContext.jdbc(url(ip, "foo"), "tbl")
 val rows = rdd.collect
 assert(rows.length == 2)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -153,7 +153,7 @@ class MySQLDatabase {
   }
 
   test("Numeric types") {
-val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers")
+val rdd = TestSQLContext.jdbc(url(ip, "foo"), "numbers")
 val rows = rdd.collect
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -181,7 +181,7 @@ class MySQLDatabase {
   }
 
   test("Date types") {
-val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates")
+val rdd = TestSQLContext.jdbc(url(ip, "foo"), "dates")
 val rows = rdd.collect
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -199,7 +199,7 @@ class MySQLDatabase {
   }
 
   test("String types") {
-val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings")
+val rdd = TestSQLContext.jdbc(url(ip, "foo"), "strings")
 val rows = rdd.collect
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -225,9 +225,9 @@ class MySQLDatabase {
   }
 
   test("Basic write test") {
-val rdd1 = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers")
-val rdd2 = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates")
-val rdd3 = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings")
+val rdd1 = TestSQLContext.jdbc(url(ip, "foo"), "numbers")
+val rdd2 = TestSQLContext.jdbc(url(ip, "foo"), "dates")
+val rdd3 = TestSQLContext.jdbc(url(ip, "foo"), "strings")
 rdd1.createJDBCTable(url(ip, "foo"), "numberscopy", false)
 rdd2.createJDBCTable(url(ip, "foo"), "datescopy", false)
 rdd3.createJDBCTable(url(ip, "foo"), "stringscopy", false)

http://git-wip-us.apache.org/repos/asf/spark/blob/cd3d4158/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala
index c174d7a..7b47fee 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala
@@ -113,7 +113,7 @@ class PostgresDatabase {
   }
 
   test("Type mapping for various types") {
-val rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar")
+val rdd = TestSQLContext.jdbc(url(db.ip), "public.bar")
 val rows = rdd.collect
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
@@ -142,7 +142,7 @@ class PostgresDatabase {
   }
 
   test("Basic write test") {
-val rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar")
+val rdd = TestSQLContext.jdbc(url(db.ip), "public.bar")
 rdd.createJDBCTable(url(db.ip), "public.barcopy", false)
 // Test only that it doesn't bomb out.
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/cd3d4158/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
index bfacc51..07b5a84 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
@@ -29,9 +29,9 @@ import org.apache.spark.sql.hive.HiveShim
 import org.apache.spark.sql.SQLContext
 
 /**
- * Implementation for "describe [extended] table".
- *
  * :: DeveloperApi ::
+ *
+ * Implementation for "describe [extended] table".
  */
 @DeveloperApi
 case class DescribeHiveTableCommand(

http://git-wip-us.apache.org/repos/asf/spark/blob/cd3d4158/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/

spark git commit: [SPARK-5859] [PySpark] [SQL] fix DataFrame Python API

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 cd3d41587 -> 4a581aa3f


[SPARK-5859] [PySpark] [SQL] fix DataFrame Python API

1. added explain()
2. add isLocal()
3. do not call show() in __repl__
4. add foreach() and foreachPartition()
5. add distinct()
6. fix functions.col()/column()/lit()
7. fix unit tests in sql/functions.py
8. fix unicode in showString()

Author: Davies Liu 

Closes #4645 from davies/df6 and squashes the following commits:

6b46a2c [Davies Liu] fix DataFrame Python API

(cherry picked from commit d8adefefcc2a4af32295440ed1d4917a6968f017)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 4a581aa3f9144732cc0f6dab6e46f72035e072f4
Parents: cd3d415
Author: Davies Liu 
Authored: Tue Feb 17 10:22:48 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 10:22:56 2015 -0800

--
 python/pyspark/sql/dataframe.py | 65 ++--
 python/pyspark/sql/functions.py | 12 +++
 2 files changed, 59 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4a581aa3/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 28a59e7..8417240 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -238,6 +238,22 @@ class DataFrame(object):
 """
 print (self._jdf.schema().treeString())
 
+def explain(self, extended=False):
+"""
+Prints the plans (logical and physical) to the console for
+debugging purpose.
+
+If extended is False, only prints the physical plan.
+"""
+self._jdf.explain(extended)
+
+def isLocal(self):
+"""
+Returns True if the `collect` and `take` methods can be run locally
+(without any Spark executors).
+"""
+return self._jdf.isLocal()
+
 def show(self):
 """
 Print the first 20 rows.
@@ -247,14 +263,12 @@ class DataFrame(object):
 2   Alice
 5   Bob
 >>> df
-age name
-2   Alice
-5   Bob
+DataFrame[age: int, name: string]
 """
-print (self)
+print self._jdf.showString().encode('utf8', 'ignore')
 
 def __repr__(self):
-return self._jdf.showString()
+return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes))
 
 def count(self):
 """Return the number of elements in this RDD.
@@ -336,6 +350,8 @@ class DataFrame(object):
 """
 Return a new RDD by applying a function to each partition.
 
+It's a shorthand for df.rdd.mapPartitions()
+
 >>> rdd = sc.parallelize([1, 2, 3, 4], 4)
 >>> def f(iterator): yield 1
 >>> rdd.mapPartitions(f).sum()
@@ -343,6 +359,31 @@ class DataFrame(object):
 """
 return self.rdd.mapPartitions(f, preservesPartitioning)
 
+def foreach(self, f):
+"""
+Applies a function to all rows of this DataFrame.
+
+It's a shorthand for df.rdd.foreach()
+
+>>> def f(person):
+... print person.name
+>>> df.foreach(f)
+"""
+return self.rdd.foreach(f)
+
+def foreachPartition(self, f):
+"""
+Applies a function to each partition of this DataFrame.
+
+It's a shorthand for df.rdd.foreachPartition()
+
+>>> def f(people):
+... for person in people:
+... print person.name
+>>> df.foreachPartition(f)
+"""
+return self.rdd.foreachPartition(f)
+
 def cache(self):
 """ Persist with the default storage level (C{MEMORY_ONLY_SER}).
 """
@@ -377,8 +418,13 @@ class DataFrame(object):
 """ Return a new :class:`DataFrame` that has exactly `numPartitions`
 partitions.
 """
-rdd = self._jdf.repartition(numPartitions, None)
-return DataFrame(rdd, self.sql_ctx)
+return DataFrame(self._jdf.repartition(numPartitions, None), 
self.sql_ctx)
+
+def distinct(self):
+"""
+Return a new :class:`DataFrame` containing the distinct rows in this 
DataFrame.
+"""
+return DataFrame(self._jdf.distinct(), self.sql_ctx)
 
 def sample(self, withReplacement, fraction, seed=None):
 """
@@ -957,10 +1003,7 @@ class Column(DataFrame):
 return Column(jc, self.sql_ctx)
 
 def __repr__(self):
-if self._jdf.isComputable():
-return self._jdf.samples()
-else:
-ret

spark git commit: [Minor] fix typo in SQL document

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master fc4eb9505 -> 31efb39c1


[Minor] fix typo in SQL document

Author: CodingCat 

Closes #4656 from CodingCat/fix_typo and squashes the following commits:

b41d15c [CodingCat] recover
689fe46 [CodingCat] fix typo


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

Branch: refs/heads/master
Commit: 31efb39c1deb253032b38e8fbafde4b2b1dde1f6
Parents: fc4eb95
Author: CodingCat 
Authored: Tue Feb 17 12:16:52 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 12:16:52 2015 -0800

--
 docs/sql-programming-guide.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/31efb39c/docs/sql-programming-guide.md
--
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 8022c5e..0146a4e 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -159,7 +159,7 @@ you to construct DataFrames when the columns and their 
types are not known until
 
 
 
-The Scala interaface for Spark SQL supports automatically converting an RDD 
containing case classes
+The Scala interface for Spark SQL supports automatically converting an RDD 
containing case classes
 to a DataFrame.  The case class
 defines the schema of the table.  The names of the arguments to the case class 
are read using
 reflection and become the names of the columns. Case classes can also be 
nested or contain complex


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



spark git commit: [Minor] fix typo in SQL document

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 71cf6e295 -> 5636c4a58


[Minor] fix typo in SQL document

Author: CodingCat 

Closes #4656 from CodingCat/fix_typo and squashes the following commits:

b41d15c [CodingCat] recover
689fe46 [CodingCat] fix typo

(cherry picked from commit 31efb39c1deb253032b38e8fbafde4b2b1dde1f6)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 5636c4a583bd28a5b54e14930d72fd9265c301de
Parents: 71cf6e2
Author: CodingCat 
Authored: Tue Feb 17 12:16:52 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 12:17:05 2015 -0800

--
 docs/sql-programming-guide.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5636c4a5/docs/sql-programming-guide.md
--
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 8022c5e..0146a4e 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -159,7 +159,7 @@ you to construct DataFrames when the columns and their 
types are not known until
 
 
 
-The Scala interaface for Spark SQL supports automatically converting an RDD 
containing case classes
+The Scala interface for Spark SQL supports automatically converting an RDD 
containing case classes
 to a DataFrame.  The case class
 defines the schema of the table.  The names of the arguments to the case class 
are read using
 reflection and become the names of the columns. Case classes can also be 
nested or contain complex


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



spark git commit: [SPARK-5862][SQL] Only transformUp the given plan once in HiveMetastoreCatalog

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 31efb39c1 -> 4611de1ce


[SPARK-5862][SQL] Only transformUp the given plan once in HiveMetastoreCatalog

Current `ParquetConversions` in `HiveMetastoreCatalog` will transformUp the 
given plan multiple times if there are many Metastore Parquet tables. Since the 
transformUp operation is recursive, it should be better to only perform it once.

Author: Liang-Chi Hsieh 

Closes #4651 from viirya/parquet_atonce and squashes the following commits:

c1ed29d [Liang-Chi Hsieh] Fix bug.
e0f919b [Liang-Chi Hsieh] Only transformUp the given plan once.


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

Branch: refs/heads/master
Commit: 4611de1cef7363bc71ec608560dfd866ae477747
Parents: 31efb39
Author: Liang-Chi Hsieh 
Authored: Tue Feb 17 12:23:18 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 12:23:18 2015 -0800

--
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 37 +++-
 1 file changed, 20 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4611de1c/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 0e43faa..cfd6f27 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
@@ -430,33 +430,36 @@ private[hive] class HiveMetastoreCatalog(hive: 
HiveContext) extends Catalog with
   hive.convertMetastoreParquet &&
   hive.conf.parquetUseDataSourceApi &&
   
relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") =>
-  relation
+  val parquetRelation = convertToParquetRelation(relation)
+  val attributedRewrites = relation.output.zip(parquetRelation.output)
+  (relation, parquetRelation, attributedRewrites)
 
 // Read path
 case p @ PhysicalOperation(_, _, relation: MetastoreRelation)
 if hive.convertMetastoreParquet &&
   hive.conf.parquetUseDataSourceApi &&
   
relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") =>
-  relation
+  val parquetRelation = convertToParquetRelation(relation)
+  val attributedRewrites = relation.output.zip(parquetRelation.output)
+  (relation, parquetRelation, attributedRewrites)
   }
 
+  val relationMap = toBeReplaced.map(r => (r._1, r._2)).toMap
+  val attributedRewrites = AttributeMap(toBeReplaced.map(_._3).fold(Nil)(_ 
++: _))
+
   // Replaces all `MetastoreRelation`s with corresponding 
`ParquetRelation2`s, and fixes
   // attribute IDs referenced in other nodes.
-  toBeReplaced.distinct.foldLeft(plan) { (lastPlan, relation) =>
-val parquetRelation = convertToParquetRelation(relation)
-val attributedRewrites = 
AttributeMap(relation.output.zip(parquetRelation.output))
-
-lastPlan.transformUp {
-  case r: MetastoreRelation if r == relation => {
-val withAlias =
-  r.alias.map(a => Subquery(a, parquetRelation)).getOrElse(
-Subquery(r.tableName, parquetRelation))
-
-withAlias
-  }
-  case other => other.transformExpressions {
-case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, 
a)
-  }
+  plan.transformUp {
+case r: MetastoreRelation if relationMap.contains(r) => {
+  val parquetRelation = relationMap(r)
+  val withAlias =
+r.alias.map(a => Subquery(a, parquetRelation)).getOrElse(
+  Subquery(r.tableName, parquetRelation))
+
+  withAlias
+}
+case other => other.transformExpressions {
+  case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a)
 }
   }
 }


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



spark git commit: [SPARK-5862][SQL] Only transformUp the given plan once in HiveMetastoreCatalog

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 5636c4a58 -> 62063b7a3


[SPARK-5862][SQL] Only transformUp the given plan once in HiveMetastoreCatalog

Current `ParquetConversions` in `HiveMetastoreCatalog` will transformUp the 
given plan multiple times if there are many Metastore Parquet tables. Since the 
transformUp operation is recursive, it should be better to only perform it once.

Author: Liang-Chi Hsieh 

Closes #4651 from viirya/parquet_atonce and squashes the following commits:

c1ed29d [Liang-Chi Hsieh] Fix bug.
e0f919b [Liang-Chi Hsieh] Only transformUp the given plan once.

(cherry picked from commit 4611de1cef7363bc71ec608560dfd866ae477747)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 62063b7a3e2db9fc7320739d3b900a7840c2dee7
Parents: 5636c4a
Author: Liang-Chi Hsieh 
Authored: Tue Feb 17 12:23:18 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 12:23:26 2015 -0800

--
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 37 +++-
 1 file changed, 20 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/62063b7a/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 0e43faa..cfd6f27 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
@@ -430,33 +430,36 @@ private[hive] class HiveMetastoreCatalog(hive: 
HiveContext) extends Catalog with
   hive.convertMetastoreParquet &&
   hive.conf.parquetUseDataSourceApi &&
   
relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") =>
-  relation
+  val parquetRelation = convertToParquetRelation(relation)
+  val attributedRewrites = relation.output.zip(parquetRelation.output)
+  (relation, parquetRelation, attributedRewrites)
 
 // Read path
 case p @ PhysicalOperation(_, _, relation: MetastoreRelation)
 if hive.convertMetastoreParquet &&
   hive.conf.parquetUseDataSourceApi &&
   
relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") =>
-  relation
+  val parquetRelation = convertToParquetRelation(relation)
+  val attributedRewrites = relation.output.zip(parquetRelation.output)
+  (relation, parquetRelation, attributedRewrites)
   }
 
+  val relationMap = toBeReplaced.map(r => (r._1, r._2)).toMap
+  val attributedRewrites = AttributeMap(toBeReplaced.map(_._3).fold(Nil)(_ 
++: _))
+
   // Replaces all `MetastoreRelation`s with corresponding 
`ParquetRelation2`s, and fixes
   // attribute IDs referenced in other nodes.
-  toBeReplaced.distinct.foldLeft(plan) { (lastPlan, relation) =>
-val parquetRelation = convertToParquetRelation(relation)
-val attributedRewrites = 
AttributeMap(relation.output.zip(parquetRelation.output))
-
-lastPlan.transformUp {
-  case r: MetastoreRelation if r == relation => {
-val withAlias =
-  r.alias.map(a => Subquery(a, parquetRelation)).getOrElse(
-Subquery(r.tableName, parquetRelation))
-
-withAlias
-  }
-  case other => other.transformExpressions {
-case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, 
a)
-  }
+  plan.transformUp {
+case r: MetastoreRelation if relationMap.contains(r) => {
+  val parquetRelation = relationMap(r)
+  val withAlias =
+r.alias.map(a => Subquery(a, parquetRelation)).getOrElse(
+  Subquery(r.tableName, parquetRelation))
+
+  withAlias
+}
+case other => other.transformExpressions {
+  case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a)
 }
   }
 }


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



spark git commit: [Minor][SQL] Use same function to check path parameter in JSONRelation

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 62063b7a3 -> d74d5e86a


[Minor][SQL] Use same function to check path parameter in JSONRelation

Author: Liang-Chi Hsieh 

Closes #4649 from viirya/use_checkpath and squashes the following commits:

0f9a1a1 [Liang-Chi Hsieh] Use same function to check path parameter.

(cherry picked from commit ac506b7c2846f656e03839bbd0e93827c7cc613e)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: d74d5e86abcb2ee4f142bb76f641d66cb4ffeb42
Parents: 62063b7
Author: Liang-Chi Hsieh 
Authored: Tue Feb 17 12:24:13 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 12:24:21 2015 -0800

--
 .../src/main/scala/org/apache/spark/sql/json/JSONRelation.scala  | 4 ++--
 .../org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala| 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d74d5e86/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 2484863..3b68b7c 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
@@ -37,7 +37,7 @@ private[sql] class DefaultSource
   override def createRelation(
   sqlContext: SQLContext,
   parameters: Map[String, String]): BaseRelation = {
-val path = parameters.getOrElse("path", sys.error("Option 'path' not 
specified"))
+val path = checkPath(parameters)
 val samplingRatio = 
parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0)
 
 JSONRelation(path, samplingRatio, None)(sqlContext)
@@ -48,7 +48,7 @@ private[sql] class DefaultSource
   sqlContext: SQLContext,
   parameters: Map[String, String],
   schema: StructType): BaseRelation = {
-val path = parameters.getOrElse("path", sys.error("Option 'path' not 
specified"))
+val path = checkPath(parameters)
 val samplingRatio = 
parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0)
 
 JSONRelation(path, samplingRatio, Some(schema))(sqlContext)

http://git-wip-us.apache.org/repos/asf/spark/blob/d74d5e86/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index 0263e3b..485d5c9 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -547,7 +547,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
BeforeAndAfterEach {
 Map.empty[String, String])
 }.getMessage
 assert(
-  message.contains("Option 'path' not specified"),
+  message.contains("'path' must be specified for json data."),
   "We should complain that path is not specified.")
 
 sql("DROP TABLE savedJsonTable")


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



spark git commit: [SQL] [Minor] Update the HiveContext Unittest

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master ac506b7c2 -> 9d281fa56


[SQL] [Minor] Update the HiveContext Unittest

In unit test, the table src(key INT, value STRING) is not the same as HIVE 
src(key STRING, value STRING)
https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql

And in the reflect.q, test failed for expression `reflect("java.lang.Integer", 
"valueOf", key, 16)`, which expect the argument `key` as STRING not INT.

This PR doesn't aim to change the `src` schema, we can do that after 1.3 
released, however, we probably need to re-generate all the golden files.

Author: Cheng Hao 

Closes #4584 from chenghao-intel/reflect and squashes the following commits:

e5bdc3a [Cheng Hao] Move the test case reflect into blacklist
184abfd [Cheng Hao] revert the change to table src1
d9bcf92 [Cheng Hao] Update the HiveContext Unittest


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

Branch: refs/heads/master
Commit: 9d281fa56022800dc008a3de233fec44379a2bd7
Parents: ac506b7
Author: Cheng Hao 
Authored: Tue Feb 17 12:25:35 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 12:25:35 2015 -0800

--
 .../spark/sql/hive/execution/HiveCompatibilitySuite.scala  | 6 ++
 .../main/scala/org/apache/spark/sql/hive/test/TestHive.scala   | 1 +
 .../golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada | 1 +
 .../golden/udf_reflect2-1-7bec330c7bc6f71cbaf9bf1883d1b184 | 1 +
 .../golden/udf_reflect2-2-c5a05379f482215a5a484bed0299bf19 | 3 +++
 .../golden/udf_reflect2-3-effc057c78c00b0af26a4ac0f5f116ca | 0
 .../golden/udf_reflect2-4-73d466e70e96e9e5f0cd373b37d4e1f4 | 5 +
 7 files changed, 17 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9d281fa5/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
--
diff --git 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 133f2d3..c6ead45 100644
--- 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -225,6 +225,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest 
with BeforeAndAfter {
 // Needs constant object inspectors
 "udf_round",
 
+// the table src(key INT, value STRING) is not the same as HIVE unittest. 
In Hive
+// is src(key STRING, value STRING), and in the reflect.q, it failed in
+// Integer.valueOf, which expect the first argument passed as STRING type 
not INT.
+"udf_reflect",
+
 // Sort with Limit clause causes failure.
 "ctas",
 "ctas_hadoop20",
@@ -886,6 +891,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
 "udf_power",
 "udf_radians",
 "udf_rand",
+"udf_reflect2",
 "udf_regexp",
 "udf_regexp_extract",
 "udf_regexp_replace",

http://git-wip-us.apache.org/repos/asf/spark/blob/9d281fa5/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 840fbc1..a2d99f1 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -196,6 +196,7 @@ class TestHiveContext(sc: SparkContext) extends 
HiveContext(sc) {
 
   // The test tables that are defined in the Hive QTestUtil.
   // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+  // 
https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql
   val hiveQTestUtilTables = Seq(
 TestTable("src",
   "CREATE TABLE src (key INT, value STRING)".cmd,

http://git-wip-us.apache.org/repos/asf/spark/blob/9d281fa5/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada
--
diff --git 
a/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada
 
b/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada
new file mode 100644
index 000..573541a
--- /dev/null
+++ 
b/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a

spark git commit: [SQL] [Minor] Update the HiveContext Unittest

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 d74d5e86a -> 01356514e


[SQL] [Minor] Update the HiveContext Unittest

In unit test, the table src(key INT, value STRING) is not the same as HIVE 
src(key STRING, value STRING)
https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql

And in the reflect.q, test failed for expression `reflect("java.lang.Integer", 
"valueOf", key, 16)`, which expect the argument `key` as STRING not INT.

This PR doesn't aim to change the `src` schema, we can do that after 1.3 
released, however, we probably need to re-generate all the golden files.

Author: Cheng Hao 

Closes #4584 from chenghao-intel/reflect and squashes the following commits:

e5bdc3a [Cheng Hao] Move the test case reflect into blacklist
184abfd [Cheng Hao] revert the change to table src1
d9bcf92 [Cheng Hao] Update the HiveContext Unittest

(cherry picked from commit 9d281fa56022800dc008a3de233fec44379a2bd7)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 01356514ef42321e09e9a67ba08366bb2bd5af4b
Parents: d74d5e8
Author: Cheng Hao 
Authored: Tue Feb 17 12:25:35 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 12:25:43 2015 -0800

--
 .../spark/sql/hive/execution/HiveCompatibilitySuite.scala  | 6 ++
 .../main/scala/org/apache/spark/sql/hive/test/TestHive.scala   | 1 +
 .../golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada | 1 +
 .../golden/udf_reflect2-1-7bec330c7bc6f71cbaf9bf1883d1b184 | 1 +
 .../golden/udf_reflect2-2-c5a05379f482215a5a484bed0299bf19 | 3 +++
 .../golden/udf_reflect2-3-effc057c78c00b0af26a4ac0f5f116ca | 0
 .../golden/udf_reflect2-4-73d466e70e96e9e5f0cd373b37d4e1f4 | 5 +
 7 files changed, 17 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/01356514/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
--
diff --git 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 133f2d3..c6ead45 100644
--- 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -225,6 +225,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest 
with BeforeAndAfter {
 // Needs constant object inspectors
 "udf_round",
 
+// the table src(key INT, value STRING) is not the same as HIVE unittest. 
In Hive
+// is src(key STRING, value STRING), and in the reflect.q, it failed in
+// Integer.valueOf, which expect the first argument passed as STRING type 
not INT.
+"udf_reflect",
+
 // Sort with Limit clause causes failure.
 "ctas",
 "ctas_hadoop20",
@@ -886,6 +891,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
 "udf_power",
 "udf_radians",
 "udf_rand",
+"udf_reflect2",
 "udf_regexp",
 "udf_regexp_extract",
 "udf_regexp_replace",

http://git-wip-us.apache.org/repos/asf/spark/blob/01356514/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 840fbc1..a2d99f1 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -196,6 +196,7 @@ class TestHiveContext(sc: SparkContext) extends 
HiveContext(sc) {
 
   // The test tables that are defined in the Hive QTestUtil.
   // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+  // 
https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql
   val hiveQTestUtilTables = Seq(
 TestTable("src",
   "CREATE TABLE src (key INT, value STRING)".cmd,

http://git-wip-us.apache.org/repos/asf/spark/blob/01356514/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada
--
diff --git 
a/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada
 
b/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada
new file mode 100

spark git commit: [Minor][SQL] Use same function to check path parameter in JSONRelation

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 4611de1ce -> ac506b7c2


[Minor][SQL] Use same function to check path parameter in JSONRelation

Author: Liang-Chi Hsieh 

Closes #4649 from viirya/use_checkpath and squashes the following commits:

0f9a1a1 [Liang-Chi Hsieh] Use same function to check path parameter.


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

Branch: refs/heads/master
Commit: ac506b7c2846f656e03839bbd0e93827c7cc613e
Parents: 4611de1
Author: Liang-Chi Hsieh 
Authored: Tue Feb 17 12:24:13 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 12:24:13 2015 -0800

--
 .../src/main/scala/org/apache/spark/sql/json/JSONRelation.scala  | 4 ++--
 .../org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala| 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ac506b7c/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 2484863..3b68b7c 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
@@ -37,7 +37,7 @@ private[sql] class DefaultSource
   override def createRelation(
   sqlContext: SQLContext,
   parameters: Map[String, String]): BaseRelation = {
-val path = parameters.getOrElse("path", sys.error("Option 'path' not 
specified"))
+val path = checkPath(parameters)
 val samplingRatio = 
parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0)
 
 JSONRelation(path, samplingRatio, None)(sqlContext)
@@ -48,7 +48,7 @@ private[sql] class DefaultSource
   sqlContext: SQLContext,
   parameters: Map[String, String],
   schema: StructType): BaseRelation = {
-val path = parameters.getOrElse("path", sys.error("Option 'path' not 
specified"))
+val path = checkPath(parameters)
 val samplingRatio = 
parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0)
 
 JSONRelation(path, samplingRatio, Some(schema))(sqlContext)

http://git-wip-us.apache.org/repos/asf/spark/blob/ac506b7c/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index 0263e3b..485d5c9 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -547,7 +547,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
BeforeAndAfterEach {
 Map.empty[String, String])
 }.getMessage
 assert(
-  message.contains("Option 'path' not specified"),
+  message.contains("'path' must be specified for json data."),
   "We should complain that path is not specified.")
 
 sql("DROP TABLE savedJsonTable")


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



spark git commit: [SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 01356514e -> e65dc1fd5


[SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext

Author: Michael Armbrust 

Closes #4657 from marmbrus/pythonUdfs and squashes the following commits:

a7823a8 [Michael Armbrust] [SPARK-5868][SQL] Fix python UDFs in HiveContext and 
checks in SQLContext

(cherry picked from commit de4836f8f12c36c1b350cef288a75b5e59155735)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: e65dc1fd58e4f9445247c9fd4e94b34550e992fb
Parents: 0135651
Author: Michael Armbrust 
Authored: Tue Feb 17 13:23:45 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 13:23:56 2015 -0800

--
 sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 3 ++-
 .../main/scala/org/apache/spark/sql/execution/pythonUdfs.scala| 3 +++
 2 files changed, 5 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e65dc1fd/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 31afa0e..709b350 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -113,6 +113,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
   protected[sql] lazy val analyzer: Analyzer =
 new Analyzer(catalog, functionRegistry, caseSensitive = true) {
   override val extendedResolutionRules =
+ExtractPythonUdfs ::
 sources.PreWriteCheck(catalog) ::
 sources.PreInsertCastAndRename ::
 Nil
@@ -1059,7 +1060,7 @@ class SQLContext(@transient val sparkContext: 
SparkContext)
   @DeveloperApi
   protected[sql] class QueryExecution(val logical: LogicalPlan) {
 
-lazy val analyzed: LogicalPlan = ExtractPythonUdfs(analyzer(logical))
+lazy val analyzed: LogicalPlan = analyzer(logical)
 lazy val withCachedData: LogicalPlan = cacheManager.useCachedData(analyzed)
 lazy val optimizedPlan: LogicalPlan = optimizer(withCachedData)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e65dc1fd/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
index 3a2f8d7..69de4d1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
@@ -205,6 +205,9 @@ case class EvaluatePython(
   extends logical.UnaryNode {
 
   def output = child.output :+ resultAttribute
+
+  // References should not include the produced attribute.
+  override def references = udf.references
 }
 
 /**


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



spark git commit: [SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 9d281fa56 -> de4836f8f


[SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext

Author: Michael Armbrust 

Closes #4657 from marmbrus/pythonUdfs and squashes the following commits:

a7823a8 [Michael Armbrust] [SPARK-5868][SQL] Fix python UDFs in HiveContext and 
checks in SQLContext


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

Branch: refs/heads/master
Commit: de4836f8f12c36c1b350cef288a75b5e59155735
Parents: 9d281fa
Author: Michael Armbrust 
Authored: Tue Feb 17 13:23:45 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 13:23:45 2015 -0800

--
 sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 3 ++-
 .../main/scala/org/apache/spark/sql/execution/pythonUdfs.scala| 3 +++
 2 files changed, 5 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/de4836f8/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 31afa0e..709b350 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -113,6 +113,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
   protected[sql] lazy val analyzer: Analyzer =
 new Analyzer(catalog, functionRegistry, caseSensitive = true) {
   override val extendedResolutionRules =
+ExtractPythonUdfs ::
 sources.PreWriteCheck(catalog) ::
 sources.PreInsertCastAndRename ::
 Nil
@@ -1059,7 +1060,7 @@ class SQLContext(@transient val sparkContext: 
SparkContext)
   @DeveloperApi
   protected[sql] class QueryExecution(val logical: LogicalPlan) {
 
-lazy val analyzed: LogicalPlan = ExtractPythonUdfs(analyzer(logical))
+lazy val analyzed: LogicalPlan = analyzer(logical)
 lazy val withCachedData: LogicalPlan = cacheManager.useCachedData(analyzed)
 lazy val optimizedPlan: LogicalPlan = optimizer(withCachedData)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/de4836f8/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
index 3a2f8d7..69de4d1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
@@ -205,6 +205,9 @@ case class EvaluatePython(
   extends logical.UnaryNode {
 
   def output = child.output :+ resultAttribute
+
+  // References should not include the produced attribute.
+  override def references = udf.references
 }
 
 /**


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



spark git commit: [SPARK-5871] output explain in Python

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 445a755b8 -> 3df85dccb


[SPARK-5871] output explain in Python

Author: Davies Liu 

Closes #4658 from davies/explain and squashes the following commits:

db87ea2 [Davies Liu] output explain in Python


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

Branch: refs/heads/master
Commit: 3df85dccbc8fd1ba19bbcdb8d359c073b1494d98
Parents: 445a755
Author: Davies Liu 
Authored: Tue Feb 17 13:48:38 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 13:48:38 2015 -0800

--
 python/pyspark/sql/dataframe.py | 23 ---
 1 file changed, 20 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3df85dcc/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 8417240..388033d 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -244,8 +244,25 @@ class DataFrame(object):
 debugging purpose.
 
 If extended is False, only prints the physical plan.
-"""
-self._jdf.explain(extended)
+
+>>> df.explain()
+PhysicalRDD [age#0,name#1], MapPartitionsRDD[...] at mapPartitions at 
SQLContext.scala:...
+
+>>> df.explain(True)
+== Parsed Logical Plan ==
+...
+== Analyzed Logical Plan ==
+...
+== Optimized Logical Plan ==
+...
+== Physical Plan ==
+...
+== RDD ==
+"""
+if extended:
+print self._jdf.queryExecution().toString()
+else:
+print self._jdf.queryExecution().executedPlan().toString()
 
 def isLocal(self):
 """
@@ -1034,7 +1051,7 @@ def _test():
   Row(name='Bob', age=5, height=85)]).toDF()
 (failure_count, test_count) = doctest.testmod(
 pyspark.sql.dataframe, globs=globs,
-optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE)
+optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | 
doctest.REPORT_NDIFF)
 globs['sc'].stop()
 if failure_count:
 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-5871] output explain in Python

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 35e23ff14 -> cb061603c


[SPARK-5871] output explain in Python

Author: Davies Liu 

Closes #4658 from davies/explain and squashes the following commits:

db87ea2 [Davies Liu] output explain in Python

(cherry picked from commit 3df85dccbc8fd1ba19bbcdb8d359c073b1494d98)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: cb061603c3ca4cd5162a36fc32de15779614e854
Parents: 35e23ff
Author: Davies Liu 
Authored: Tue Feb 17 13:48:38 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 13:51:00 2015 -0800

--
 python/pyspark/sql/dataframe.py | 23 ---
 1 file changed, 20 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cb061603/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 8417240..388033d 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -244,8 +244,25 @@ class DataFrame(object):
 debugging purpose.
 
 If extended is False, only prints the physical plan.
-"""
-self._jdf.explain(extended)
+
+>>> df.explain()
+PhysicalRDD [age#0,name#1], MapPartitionsRDD[...] at mapPartitions at 
SQLContext.scala:...
+
+>>> df.explain(True)
+== Parsed Logical Plan ==
+...
+== Analyzed Logical Plan ==
+...
+== Optimized Logical Plan ==
+...
+== Physical Plan ==
+...
+== RDD ==
+"""
+if extended:
+print self._jdf.queryExecution().toString()
+else:
+print self._jdf.queryExecution().executedPlan().toString()
 
 def isLocal(self):
 """
@@ -1034,7 +1051,7 @@ def _test():
   Row(name='Bob', age=5, height=85)]).toDF()
 (failure_count, test_count) = doctest.testmod(
 pyspark.sql.dataframe, globs=globs,
-optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE)
+optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | 
doctest.REPORT_NDIFF)
 globs['sc'].stop()
 if failure_count:
 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-5872] [SQL] create a sqlCtx in pyspark shell

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 3df85dccb -> 4d4cc760f


[SPARK-5872] [SQL] create a sqlCtx in pyspark shell

The sqlCtx will be HiveContext if hive is built in assembly jar, or SQLContext 
if not.

It also skip the Hive tests in pyspark.sql.tests if no hive is available.

Author: Davies Liu 

Closes #4659 from davies/sqlctx and squashes the following commits:

0e6629a [Davies Liu] sqlCtx in pyspark


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

Branch: refs/heads/master
Commit: 4d4cc760fa9687ce563320094557ef9144488676
Parents: 3df85dc
Author: Davies Liu 
Authored: Tue Feb 17 15:44:37 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 15:44:37 2015 -0800

--
 python/pyspark/shell.py | 13 -
 python/pyspark/sql/tests.py | 12 ++--
 2 files changed, 22 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4d4cc760/python/pyspark/shell.py
--
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index 4cf4b89..1a02fec 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -31,8 +31,12 @@ if sys.version_info[0] != 2:
 import atexit
 import os
 import platform
+
+import py4j
+
 import pyspark
 from pyspark.context import SparkContext
+from pyspark.sql import SQLContext, HiveContext
 from pyspark.storagelevel import StorageLevel
 
 # this is the deprecated equivalent of ADD_JARS
@@ -46,6 +50,13 @@ if os.environ.get("SPARK_EXECUTOR_URI"):
 sc = SparkContext(appName="PySparkShell", pyFiles=add_files)
 atexit.register(lambda: sc.stop())
 
+try:
+# Try to access HiveConf, it will raise exception if Hive is not added
+sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
+sqlCtx = HiveContext(sc)
+except py4j.protocol.Py4JError:
+sqlCtx = SQLContext(sc)
+
 print("""Welcome to
     __
  / __/__  ___ _/ /__
@@ -57,7 +68,7 @@ print("Using Python version %s (%s, %s)" % (
 platform.python_version(),
 platform.python_build()[0],
 platform.python_build()[1]))
-print("SparkContext available as sc.")
+print("SparkContext available as sc, %s available as sqlCtx." % 
sqlCtx.__class__.__name__)
 
 if add_files is not None:
 print("Warning: ADD_FILES environment variable is deprecated, use 
--py-files argument instead")

http://git-wip-us.apache.org/repos/asf/spark/blob/4d4cc760/python/pyspark/sql/tests.py
--
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index aa80bca..52f7e65 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -25,6 +25,8 @@ import pydoc
 import shutil
 import tempfile
 
+import py4j
+
 if sys.version_info[:2] <= (2, 6):
 try:
 import unittest2 as unittest
@@ -329,9 +331,12 @@ class HiveContextSQLTests(ReusedPySparkTestCase):
 def setUpClass(cls):
 ReusedPySparkTestCase.setUpClass()
 cls.tempdir = tempfile.NamedTemporaryFile(delete=False)
+try:
+cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
+except py4j.protocol.Py4JError:
+cls.sqlCtx = None
+return
 os.unlink(cls.tempdir.name)
-print "type", type(cls.sc)
-print "type", type(cls.sc._jsc)
 _scala_HiveContext =\
 
cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc())
 cls.sqlCtx = HiveContext(cls.sc, _scala_HiveContext)
@@ -344,6 +349,9 @@ class HiveContextSQLTests(ReusedPySparkTestCase):
 shutil.rmtree(cls.tempdir.name, ignore_errors=True)
 
 def test_save_and_load_table(self):
+if self.sqlCtx is None:
+return  # no hive available, skipped
+
 df = self.df
 tmpPath = tempfile.mkdtemp()
 shutil.rmtree(tmpPath)


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



spark git commit: [SPARK-5872] [SQL] create a sqlCtx in pyspark shell

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 cb061603c -> 0dba382ee


[SPARK-5872] [SQL] create a sqlCtx in pyspark shell

The sqlCtx will be HiveContext if hive is built in assembly jar, or SQLContext 
if not.

It also skip the Hive tests in pyspark.sql.tests if no hive is available.

Author: Davies Liu 

Closes #4659 from davies/sqlctx and squashes the following commits:

0e6629a [Davies Liu] sqlCtx in pyspark

(cherry picked from commit 4d4cc760fa9687ce563320094557ef9144488676)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 0dba382ee65694969704384c4968e3a656b3c833
Parents: cb06160
Author: Davies Liu 
Authored: Tue Feb 17 15:44:37 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 15:44:45 2015 -0800

--
 python/pyspark/shell.py | 13 -
 python/pyspark/sql/tests.py | 12 ++--
 2 files changed, 22 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0dba382e/python/pyspark/shell.py
--
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index 4cf4b89..1a02fec 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -31,8 +31,12 @@ if sys.version_info[0] != 2:
 import atexit
 import os
 import platform
+
+import py4j
+
 import pyspark
 from pyspark.context import SparkContext
+from pyspark.sql import SQLContext, HiveContext
 from pyspark.storagelevel import StorageLevel
 
 # this is the deprecated equivalent of ADD_JARS
@@ -46,6 +50,13 @@ if os.environ.get("SPARK_EXECUTOR_URI"):
 sc = SparkContext(appName="PySparkShell", pyFiles=add_files)
 atexit.register(lambda: sc.stop())
 
+try:
+# Try to access HiveConf, it will raise exception if Hive is not added
+sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
+sqlCtx = HiveContext(sc)
+except py4j.protocol.Py4JError:
+sqlCtx = SQLContext(sc)
+
 print("""Welcome to
     __
  / __/__  ___ _/ /__
@@ -57,7 +68,7 @@ print("Using Python version %s (%s, %s)" % (
 platform.python_version(),
 platform.python_build()[0],
 platform.python_build()[1]))
-print("SparkContext available as sc.")
+print("SparkContext available as sc, %s available as sqlCtx." % 
sqlCtx.__class__.__name__)
 
 if add_files is not None:
 print("Warning: ADD_FILES environment variable is deprecated, use 
--py-files argument instead")

http://git-wip-us.apache.org/repos/asf/spark/blob/0dba382e/python/pyspark/sql/tests.py
--
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index aa80bca..52f7e65 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -25,6 +25,8 @@ import pydoc
 import shutil
 import tempfile
 
+import py4j
+
 if sys.version_info[:2] <= (2, 6):
 try:
 import unittest2 as unittest
@@ -329,9 +331,12 @@ class HiveContextSQLTests(ReusedPySparkTestCase):
 def setUpClass(cls):
 ReusedPySparkTestCase.setUpClass()
 cls.tempdir = tempfile.NamedTemporaryFile(delete=False)
+try:
+cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
+except py4j.protocol.Py4JError:
+cls.sqlCtx = None
+return
 os.unlink(cls.tempdir.name)
-print "type", type(cls.sc)
-print "type", type(cls.sc._jsc)
 _scala_HiveContext =\
 
cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc())
 cls.sqlCtx = HiveContext(cls.sc, _scala_HiveContext)
@@ -344,6 +349,9 @@ class HiveContextSQLTests(ReusedPySparkTestCase):
 shutil.rmtree(cls.tempdir.name, ignore_errors=True)
 
 def test_save_and_load_table(self):
+if self.sqlCtx is None:
+return  # no hive available, skipped
+
 df = self.df
 tmpPath = tempfile.mkdtemp()
 shutil.rmtree(tmpPath)


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



spark git commit: [SPARK-5852][SQL]Fail to convert a newly created empty metastore parquet table to a data source parquet table.

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 4d4cc760f -> 117121a4e


[SPARK-5852][SQL]Fail to convert a newly created empty metastore parquet table 
to a data source parquet table.

The problem is that after we create an empty hive metastore parquet table (e.g. 
`CREATE TABLE test (a int) STORED AS PARQUET`), Hive will create an empty dir 
for us, which cause our data source `ParquetRelation2` fail to get the schema 
of the table. See JIRA for the case to reproduce the bug and the exception.

This PR is based on #4562 from chenghao-intel.

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

Author: Yin Huai 
Author: Cheng Hao 

Closes #4655 from yhuai/CTASParquet and squashes the following commits:

b8b3450 [Yin Huai] Update tests.
2ac94f7 [Yin Huai] Update tests.
3db3d20 [Yin Huai] Minor update.
d7e2308 [Yin Huai] Revert changes in HiveMetastoreCatalog.scala.
36978d1 [Cheng Hao] Update the code as feedback
a04930b [Cheng Hao] fix bug of scan an empty parquet based table
442ffe0 [Cheng Hao] passdown the schema for Parquet File in HiveContext


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

Branch: refs/heads/master
Commit: 117121a4ecaadda156a82255333670775e7727db
Parents: 4d4cc76
Author: Yin Huai 
Authored: Tue Feb 17 15:47:59 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 15:47:59 2015 -0800

--
 .../apache/spark/sql/parquet/newParquet.scala   |  18 ++-
 .../sql/hive/MetastoreDataSourcesSuite.scala|  38 +++
 .../spark/sql/parquet/parquetSuites.scala   | 114 ++-
 3 files changed, 164 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/117121a4/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
index 95bea92..16b7713 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
@@ -287,7 +287,16 @@ private[sql] case class ParquetRelation2(
 }
   }
 
-  parquetSchema = maybeSchema.getOrElse(readSchema())
+  // To get the schema. We first try to get the schema defined in 
maybeSchema.
+  // If maybeSchema is not defined, we will try to get the schema from 
existing parquet data
+  // (through readSchema). If data does not exist, we will try to get the 
schema defined in
+  // maybeMetastoreSchema (defined in the options of the data source).
+  // Finally, if we still could not get the schema. We throw an error.
+  parquetSchema =
+maybeSchema
+  .orElse(readSchema())
+  .orElse(maybeMetastoreSchema)
+  .getOrElse(sys.error("Failed to get the schema."))
 
   partitionKeysIncludedInParquetSchema =
 isPartitioned &&
@@ -308,7 +317,7 @@ private[sql] case class ParquetRelation2(
   }
 }
 
-private def readSchema(): StructType = {
+private def readSchema(): Option[StructType] = {
   // Sees which file(s) we need to touch in order to figure out the schema.
   val filesToTouch =
   // Always tries the summary files first if users don't require a merged 
schema.  In this case,
@@ -611,7 +620,8 @@ private[sql] object ParquetRelation2 {
   // internally.
   private[sql] val METASTORE_SCHEMA = "metastoreSchema"
 
-  private[parquet] def readSchema(footers: Seq[Footer], sqlContext: 
SQLContext): StructType = {
+  private[parquet] def readSchema(
+  footers: Seq[Footer], sqlContext: SQLContext): Option[StructType] = {
 footers.map { footer =>
   val metadata = footer.getParquetMetadata.getFileMetaData
   val parquetSchema = metadata.getSchema
@@ -630,7 +640,7 @@ private[sql] object ParquetRelation2 {
 sqlContext.conf.isParquetBinaryAsString,
 sqlContext.conf.isParquetINT96AsTimestamp))
   }
-}.reduce { (left, right) =>
+}.reduceOption { (left, right) =>
   try left.merge(right) catch { case e: Throwable =>
 throw new SparkException(s"Failed to merge incompatible schemas $left 
and $right", e)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/117121a4/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuit

spark git commit: [SPARK-5852][SQL]Fail to convert a newly created empty metastore parquet table to a data source parquet table.

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 0dba382ee -> 07d8ef9e7


[SPARK-5852][SQL]Fail to convert a newly created empty metastore parquet table 
to a data source parquet table.

The problem is that after we create an empty hive metastore parquet table (e.g. 
`CREATE TABLE test (a int) STORED AS PARQUET`), Hive will create an empty dir 
for us, which cause our data source `ParquetRelation2` fail to get the schema 
of the table. See JIRA for the case to reproduce the bug and the exception.

This PR is based on #4562 from chenghao-intel.

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

Author: Yin Huai 
Author: Cheng Hao 

Closes #4655 from yhuai/CTASParquet and squashes the following commits:

b8b3450 [Yin Huai] Update tests.
2ac94f7 [Yin Huai] Update tests.
3db3d20 [Yin Huai] Minor update.
d7e2308 [Yin Huai] Revert changes in HiveMetastoreCatalog.scala.
36978d1 [Cheng Hao] Update the code as feedback
a04930b [Cheng Hao] fix bug of scan an empty parquet based table
442ffe0 [Cheng Hao] passdown the schema for Parquet File in HiveContext

(cherry picked from commit 117121a4ecaadda156a82255333670775e7727db)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 07d8ef9e730251b66f962fa7acdaf7d7eacc62a1
Parents: 0dba382
Author: Yin Huai 
Authored: Tue Feb 17 15:47:59 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 15:48:23 2015 -0800

--
 .../apache/spark/sql/parquet/newParquet.scala   |  18 ++-
 .../sql/hive/MetastoreDataSourcesSuite.scala|  38 +++
 .../spark/sql/parquet/parquetSuites.scala   | 114 ++-
 3 files changed, 164 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/07d8ef9e/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
index 95bea92..16b7713 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
@@ -287,7 +287,16 @@ private[sql] case class ParquetRelation2(
 }
   }
 
-  parquetSchema = maybeSchema.getOrElse(readSchema())
+  // To get the schema. We first try to get the schema defined in 
maybeSchema.
+  // If maybeSchema is not defined, we will try to get the schema from 
existing parquet data
+  // (through readSchema). If data does not exist, we will try to get the 
schema defined in
+  // maybeMetastoreSchema (defined in the options of the data source).
+  // Finally, if we still could not get the schema. We throw an error.
+  parquetSchema =
+maybeSchema
+  .orElse(readSchema())
+  .orElse(maybeMetastoreSchema)
+  .getOrElse(sys.error("Failed to get the schema."))
 
   partitionKeysIncludedInParquetSchema =
 isPartitioned &&
@@ -308,7 +317,7 @@ private[sql] case class ParquetRelation2(
   }
 }
 
-private def readSchema(): StructType = {
+private def readSchema(): Option[StructType] = {
   // Sees which file(s) we need to touch in order to figure out the schema.
   val filesToTouch =
   // Always tries the summary files first if users don't require a merged 
schema.  In this case,
@@ -611,7 +620,8 @@ private[sql] object ParquetRelation2 {
   // internally.
   private[sql] val METASTORE_SCHEMA = "metastoreSchema"
 
-  private[parquet] def readSchema(footers: Seq[Footer], sqlContext: 
SQLContext): StructType = {
+  private[parquet] def readSchema(
+  footers: Seq[Footer], sqlContext: SQLContext): Option[StructType] = {
 footers.map { footer =>
   val metadata = footer.getParquetMetadata.getFileMetaData
   val parquetSchema = metadata.getSchema
@@ -630,7 +640,7 @@ private[sql] object ParquetRelation2 {
 sqlContext.conf.isParquetBinaryAsString,
 sqlContext.conf.isParquetINT96AsTimestamp))
   }
-}.reduce { (left, right) =>
+}.reduceOption { (left, right) =>
   try left.merge(right) catch { case e: Throwable =>
 throw new SparkException(s"Failed to merge incompatible schemas $left 
and $right", e)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/07d8ef9e/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive

spark git commit: [SPARK-5875][SQL]logical.Project should not be resolved if it contains aggregates or generators

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master a51fc7ef9 -> d5f12bfe8


[SPARK-5875][SQL]logical.Project should not be resolved if it contains 
aggregates or generators

https://issues.apache.org/jira/browse/SPARK-5875 has a case to reproduce the 
bug and explain the root cause.

Author: Yin Huai 

Closes #4663 from yhuai/projectResolved and squashes the following commits:

472f7b6 [Yin Huai] If a logical.Project has any AggregateExpression or 
Generator, it's resolved field should be false.


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

Branch: refs/heads/master
Commit: d5f12bfe8f0a98d6fee114bb24376668ebe2898e
Parents: a51fc7e
Author: Yin Huai 
Authored: Tue Feb 17 17:50:39 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 17:50:39 2015 -0800

--
 .../catalyst/plans/logical/basicOperators.scala | 10 ++
 .../sql/catalyst/analysis/AnalysisSuite.scala   | 13 +++-
 .../sql/hive/execution/SQLQuerySuite.scala  | 32 +++-
 3 files changed, 53 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d5f12bfe/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
index 9628e93..89544ad 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
@@ -23,6 +23,16 @@ import org.apache.spark.sql.types._
 
 case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) 
extends UnaryNode {
   def output = projectList.map(_.toAttribute)
+
+  override lazy val resolved: Boolean = {
+val containsAggregatesOrGenerators = projectList.exists ( _.collect {
+case agg: AggregateExpression => agg
+case generator: Generator => generator
+  }.nonEmpty
+)
+
+!expressions.exists(!_.resolved) && childrenResolved && 
!containsAggregatesOrGenerators
+  }
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/d5f12bfe/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index e70c651..aec7847 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis
 import org.scalatest.{BeforeAndAfter, FunSuite}
 
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.expressions.{Literal, Alias, 
AttributeReference}
+import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.types._
 
@@ -58,6 +58,17 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
 assert(caseInsensitiveAnalyze(plan).resolved)
   }
 
+  test("check project's resolved") {
+assert(Project(testRelation.output, testRelation).resolved)
+
+assert(!Project(Seq(UnresolvedAttribute("a")), testRelation).resolved)
+
+val explode = Explode(Nil, AttributeReference("a", IntegerType, nullable = 
true)())
+assert(!Project(Seq(Alias(explode, "explode")()), testRelation).resolved)
+
+assert(!Project(Seq(Alias(Count(Literal(1)), "count")()), 
testRelation).resolved)
+  }
+
   test("analyze project") {
 assert(
   caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("a")), 
testRelation)) ===

http://git-wip-us.apache.org/repos/asf/spark/blob/d5f12bfe/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 e8d9eec..ae03bc5 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
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.hive.execution
 
-import or

spark git commit: [SPARK-5875][SQL]logical.Project should not be resolved if it contains aggregates or generators

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 7320605ad -> e8284b29d


[SPARK-5875][SQL]logical.Project should not be resolved if it contains 
aggregates or generators

https://issues.apache.org/jira/browse/SPARK-5875 has a case to reproduce the 
bug and explain the root cause.

Author: Yin Huai 

Closes #4663 from yhuai/projectResolved and squashes the following commits:

472f7b6 [Yin Huai] If a logical.Project has any AggregateExpression or 
Generator, it's resolved field should be false.

(cherry picked from commit d5f12bfe8f0a98d6fee114bb24376668ebe2898e)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: e8284b29df0445bdf92f5ea2c74402a111718792
Parents: 7320605
Author: Yin Huai 
Authored: Tue Feb 17 17:50:39 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 17:50:51 2015 -0800

--
 .../catalyst/plans/logical/basicOperators.scala | 10 ++
 .../sql/catalyst/analysis/AnalysisSuite.scala   | 13 +++-
 .../sql/hive/execution/SQLQuerySuite.scala  | 32 +++-
 3 files changed, 53 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e8284b29/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
index 9628e93..89544ad 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
@@ -23,6 +23,16 @@ import org.apache.spark.sql.types._
 
 case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) 
extends UnaryNode {
   def output = projectList.map(_.toAttribute)
+
+  override lazy val resolved: Boolean = {
+val containsAggregatesOrGenerators = projectList.exists ( _.collect {
+case agg: AggregateExpression => agg
+case generator: Generator => generator
+  }.nonEmpty
+)
+
+!expressions.exists(!_.resolved) && childrenResolved && 
!containsAggregatesOrGenerators
+  }
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/e8284b29/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index e70c651..aec7847 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis
 import org.scalatest.{BeforeAndAfter, FunSuite}
 
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.expressions.{Literal, Alias, 
AttributeReference}
+import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.types._
 
@@ -58,6 +58,17 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter {
 assert(caseInsensitiveAnalyze(plan).resolved)
   }
 
+  test("check project's resolved") {
+assert(Project(testRelation.output, testRelation).resolved)
+
+assert(!Project(Seq(UnresolvedAttribute("a")), testRelation).resolved)
+
+val explode = Explode(Nil, AttributeReference("a", IntegerType, nullable = 
true)())
+assert(!Project(Seq(Alias(explode, "explode")()), testRelation).resolved)
+
+assert(!Project(Seq(Alias(Count(Literal(1)), "count")()), 
testRelation).resolved)
+  }
+
   test("analyze project") {
 assert(
   caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("a")), 
testRelation)) ===

http://git-wip-us.apache.org/repos/asf/spark/blob/e8284b29/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 e8d9eec..ae03bc5 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/h

spark git commit: [SPARK-5723][SQL]Change the default file format to Parquet for CTAS statements.

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master d5f12bfe8 -> e50934f11


[SPARK-5723][SQL]Change the default file format to Parquet for CTAS statements.

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

Author: Yin Huai 

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust 

Closes #4639 from yhuai/defaultCTASFileFormat and squashes the following 
commits:

a568137 [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
defaultCTASFileFormat
ad2b07d [Yin Huai] Update tests and error messages.
8af5b2a [Yin Huai] Update conf key and unit test.
5a67903 [Yin Huai] Use data source write path for Hive's CTAS statements when 
no storage format/handler is specified.


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

Branch: refs/heads/master
Commit: e50934f11e1e3ded21a631e5ab69db3c79467137
Parents: d5f12bf
Author: Yin Huai 
Authored: Tue Feb 17 18:14:33 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 18:14:33 2015 -0800

--
 .../org/apache/spark/sql/hive/HiveContext.scala | 15 
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 75 
 .../spark/sql/hive/execution/commands.scala | 17 +++--
 .../sql/hive/MetastoreDataSourcesSuite.scala|  6 +-
 .../sql/hive/execution/SQLQuerySuite.scala  | 70 ++
 5 files changed, 158 insertions(+), 25 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e50934f1/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 6c55bc6..d3365b1 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
@@ -61,6 +61,21 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
   protected[sql] def convertMetastoreParquet: Boolean =
 getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true"
 
+  /**
+   * When true, a table created by a Hive CTAS statement (no USING clause) 
will be
+   * converted to a data source table, using the data source set by 
spark.sql.sources.default.
+   * The table in CTAS statement will be converted when it meets any of the 
following conditions:
+   *   - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File 
Format (STORED AS), or
+   * a Storage Hanlder (STORED BY), and the value of 
hive.default.fileformat in hive-site.xml
+   * is either TextFile or SequenceFile.
+   *   - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the 
file format and no SerDe
+   * is specified (no ROW FORMAT SERDE clause).
+   *   - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as 
the file format
+   * and no SerDe is specified (no ROW FORMAT SERDE clause).
+   */
+  protected[sql] def convertCTAS: Boolean =
+getConf("spark.sql.hive.convertCTAS", "false").toBoolean
+
   override protected[sql] def executePlan(plan: LogicalPlan): 
this.QueryExecution =
 new this.QueryExecution(plan)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e50934f1/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 cfd6f27..f7ad2ef 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
@@ -31,7 +31,7 @@ import org.apache.hadoop.hive.serde2.{Deserializer, 
SerDeException}
 import org.apache.hadoop.util.ReflectionUtils
 
 import org.apache.spark.Logging
-import org.apache.spark.sql.{AnalysisException, SQLContext}
+import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext}
 import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Catalog, 
OverrideCatalog}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.planning.PhysicalOperation
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
 import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => 
ParquetPartition, PartitionSpec}
-import org.apache.spark.sql.sources.{DDLParser, LogicalRelation, 
ResolvedData

spark git commit: [SPARK-5723][SQL]Change the default file format to Parquet for CTAS statements.

2015-02-17 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 2ab0ba04f -> 6e82c46bf


[SPARK-5723][SQL]Change the default file format to Parquet for CTAS statements.

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

Author: Yin Huai 

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust 

Closes #4639 from yhuai/defaultCTASFileFormat and squashes the following 
commits:

a568137 [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
defaultCTASFileFormat
ad2b07d [Yin Huai] Update tests and error messages.
8af5b2a [Yin Huai] Update conf key and unit test.
5a67903 [Yin Huai] Use data source write path for Hive's CTAS statements when 
no storage format/handler is specified.

(cherry picked from commit e50934f11e1e3ded21a631e5ab69db3c79467137)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 6e82c46bf6cdfd227b5680f114e029219204e641
Parents: 2ab0ba0
Author: Yin Huai 
Authored: Tue Feb 17 18:14:33 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 17 18:14:45 2015 -0800

--
 .../org/apache/spark/sql/hive/HiveContext.scala | 15 
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 75 
 .../spark/sql/hive/execution/commands.scala | 17 +++--
 .../sql/hive/MetastoreDataSourcesSuite.scala|  6 +-
 .../sql/hive/execution/SQLQuerySuite.scala  | 70 ++
 5 files changed, 158 insertions(+), 25 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6e82c46b/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 6c55bc6..d3365b1 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
@@ -61,6 +61,21 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
   protected[sql] def convertMetastoreParquet: Boolean =
 getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true"
 
+  /**
+   * When true, a table created by a Hive CTAS statement (no USING clause) 
will be
+   * converted to a data source table, using the data source set by 
spark.sql.sources.default.
+   * The table in CTAS statement will be converted when it meets any of the 
following conditions:
+   *   - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File 
Format (STORED AS), or
+   * a Storage Hanlder (STORED BY), and the value of 
hive.default.fileformat in hive-site.xml
+   * is either TextFile or SequenceFile.
+   *   - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the 
file format and no SerDe
+   * is specified (no ROW FORMAT SERDE clause).
+   *   - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as 
the file format
+   * and no SerDe is specified (no ROW FORMAT SERDE clause).
+   */
+  protected[sql] def convertCTAS: Boolean =
+getConf("spark.sql.hive.convertCTAS", "false").toBoolean
+
   override protected[sql] def executePlan(plan: LogicalPlan): 
this.QueryExecution =
 new this.QueryExecution(plan)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/6e82c46b/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 cfd6f27..f7ad2ef 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
@@ -31,7 +31,7 @@ import org.apache.hadoop.hive.serde2.{Deserializer, 
SerDeException}
 import org.apache.hadoop.util.ReflectionUtils
 
 import org.apache.spark.Logging
-import org.apache.spark.sql.{AnalysisException, SQLContext}
+import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext}
 import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Catalog, 
OverrideCatalog}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.planning.PhysicalOperation
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
 import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => 
P

spark git commit: [SPARK-4903][SQL]Backport the bug fix for SPARK-4903

2015-02-18 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 068ba45cf -> 36e15b48e


[SPARK-4903][SQL]Backport the bug fix for SPARK-4903

The original fix was a part of https://issues.apache.org/jira/browse/SPARK-4912 
(commit 
https://github.com/apache/spark/commit/6463e0b9e8067cce70602c5c9006a2546856a9d6).

Author: Yin Huai 

Closes #4671 from yhuai/SPARK-4903-1.2 and squashes the following commits:

3168b4b [Yin Huai] Uncache table in drop table command.


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

Branch: refs/heads/branch-1.2
Commit: 36e15b48ead360f8b870606add5cf0f7d639f126
Parents: 068ba45
Author: Yin Huai 
Authored: Wed Feb 18 13:59:55 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 18 13:59:55 2015 -0800

--
 .../org/apache/spark/sql/hive/execution/commands.scala  | 9 +
 1 file changed, 9 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/36e15b48/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 a42a7c8..9d52a57 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
@@ -55,6 +55,15 @@ case class DropTable(tableName: String, ifExists: Boolean) 
extends LeafNode with
 
   override protected lazy val sideEffectResult: Seq[Row] = {
 val ifExistsClause = if (ifExists) "IF EXISTS " else ""
+try {
+  hiveContext.tryUncacheQuery(hiveContext.table(tableName))
+} catch {
+  // This table's metadata is not in
+  case _: org.apache.hadoop.hive.ql.metadata.InvalidTableException =>
+  // Got an error during table lookup or uncache the query. We log the 
exception message.
+  // Users should be able to drop such kinds of tables regardless if there 
is an exception.
+  case e: Exception => log.warn(s"${e.getMessage}")
+}
 hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName")
 hiveContext.catalog.unregisterTable(Seq(tableName))
 Seq.empty[Row]


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



spark git commit: [SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction

2015-02-18 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master a8eb92dcb -> f0e3b7107


[SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction

Also added test cases for checking the serializability of HiveContext and 
SQLContext.

Author: Reynold Xin 

Closes #4628 from rxin/SPARK-5840 and squashes the following commits:

ecb3bcd [Reynold Xin] test cases and reviews.
55eb822 [Reynold Xin] [SPARK-5840][SQL] HiveContext cannot be serialized due to 
tuple extraction.


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

Branch: refs/heads/master
Commit: f0e3b71077a6c28aba29a7a75e901a9e0911b9f0
Parents: a8eb92d
Author: Reynold Xin 
Authored: Wed Feb 18 14:02:32 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 18 14:02:32 2015 -0800

--
 .../apache/spark/sql/SerializationSuite.scala   | 32 ++
 .../org/apache/spark/sql/hive/HiveContext.scala | 35 +++-
 .../spark/sql/hive/SerializationSuite.scala | 33 ++
 3 files changed, 84 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f0e3b710/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
new file mode 100644
index 000..6f6d3c9
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
@@ -0,0 +1,32 @@
+/*
+ * 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 License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.SparkConf
+import org.apache.spark.serializer.JavaSerializer
+import org.apache.spark.sql.test.TestSQLContext
+
+class SerializationSuite extends FunSuite {
+
+  test("[SPARK-5235] SQLContext should be serializable") {
+val sqlContext = new SQLContext(TestSQLContext.sparkContext)
+new JavaSerializer(new SparkConf()).newInstance().serialize(sqlContext)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f0e3b710/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 d3365b1..2e205e6 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
@@ -222,22 +222,25 @@ class HiveContext(sc: SparkContext) extends 
SQLContext(sc) {
*SQLConf.  Additionally, any properties set by set() or a SET command 
inside sql() will be
*set in the SQLConf *as well as* in the HiveConf.
*/
-  @transient protected[hive] lazy val (hiveconf, sessionState) =
-Option(SessionState.get())
-  .orElse {
-val newState = new SessionState(new HiveConf(classOf[SessionState]))
-// Only starts newly created `SessionState` instance.  Any existing 
`SessionState` instance
-// returned by `SessionState.get()` must be the most recently started 
one.
-SessionState.start(newState)
-Some(newState)
-  }
-  .map { state =>
-setConf(state.getConf.getAllProperties)
-if (state.out == null) state.out = new PrintStream(outputBuffer, true, 
"UTF-8")
-if (state.err == null) state.err = new PrintStream(outputBuffer, true, 
"UTF-8")
-(state.getConf, state)
-  }
-  .get
+  @transient protected[hive] lazy val sessionState: SessionState = {
+var state = SessionState.get()
+if (state == null) {
+  state = new SessionState(new HiveConf(classOf[SessionState]))
+  SessionState.start(state)
+}
+if (state.out == null) {
+  state.out = 

spark git commit: [SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction

2015-02-18 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 56f8f295c -> b86e44cd9


[SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction

Also added test cases for checking the serializability of HiveContext and 
SQLContext.

Author: Reynold Xin 

Closes #4628 from rxin/SPARK-5840 and squashes the following commits:

ecb3bcd [Reynold Xin] test cases and reviews.
55eb822 [Reynold Xin] [SPARK-5840][SQL] HiveContext cannot be serialized due to 
tuple extraction.

(cherry picked from commit f0e3b71077a6c28aba29a7a75e901a9e0911b9f0)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: b86e44cd9e7658abd2f8021120cb927bdc6ba693
Parents: 56f8f29
Author: Reynold Xin 
Authored: Wed Feb 18 14:02:32 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 18 14:02:42 2015 -0800

--
 .../apache/spark/sql/SerializationSuite.scala   | 32 ++
 .../org/apache/spark/sql/hive/HiveContext.scala | 35 +++-
 .../spark/sql/hive/SerializationSuite.scala | 33 ++
 3 files changed, 84 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b86e44cd/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
new file mode 100644
index 000..6f6d3c9
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
@@ -0,0 +1,32 @@
+/*
+ * 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 License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.SparkConf
+import org.apache.spark.serializer.JavaSerializer
+import org.apache.spark.sql.test.TestSQLContext
+
+class SerializationSuite extends FunSuite {
+
+  test("[SPARK-5235] SQLContext should be serializable") {
+val sqlContext = new SQLContext(TestSQLContext.sparkContext)
+new JavaSerializer(new SparkConf()).newInstance().serialize(sqlContext)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/b86e44cd/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 d3365b1..2e205e6 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
@@ -222,22 +222,25 @@ class HiveContext(sc: SparkContext) extends 
SQLContext(sc) {
*SQLConf.  Additionally, any properties set by set() or a SET command 
inside sql() will be
*set in the SQLConf *as well as* in the HiveConf.
*/
-  @transient protected[hive] lazy val (hiveconf, sessionState) =
-Option(SessionState.get())
-  .orElse {
-val newState = new SessionState(new HiveConf(classOf[SessionState]))
-// Only starts newly created `SessionState` instance.  Any existing 
`SessionState` instance
-// returned by `SessionState.get()` must be the most recently started 
one.
-SessionState.start(newState)
-Some(newState)
-  }
-  .map { state =>
-setConf(state.getConf.getAllProperties)
-if (state.out == null) state.out = new PrintStream(outputBuffer, true, 
"UTF-8")
-if (state.err == null) state.err = new PrintStream(outputBuffer, true, 
"UTF-8")
-(state.getConf, state)
-  }
-  .get
+  @transient protected[hive] lazy val sessionState: SessionState = {
+var state = SessionState.get()
+if (state == null) {
+  state = new SessionState(new HiveConf(cl

spark git commit: [SPARK-5722] [SQL] [PySpark] infer int as LongType

2015-02-18 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master f0e3b7107 -> aa8f10e82


[SPARK-5722] [SQL] [PySpark] infer int as LongType

The `int` is 64-bit on 64-bit machine (very common now), we should infer it as 
LongType for it in Spark SQL.

Also, LongType in SQL will come back as `int`.

Author: Davies Liu 

Closes #4666 from davies/long and squashes the following commits:

6bc6cc4 [Davies Liu] infer int as LongType


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

Branch: refs/heads/master
Commit: aa8f10e82a743d59ce87348af19c0177eb618a66
Parents: f0e3b71
Author: Davies Liu 
Authored: Wed Feb 18 14:17:04 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 18 14:17:04 2015 -0800

--
 python/pyspark/sql/dataframe.py | 14 +++--
 python/pyspark/sql/tests.py | 22 +++-
 python/pyspark/sql/types.py |  8 +++
 .../scala/org/apache/spark/sql/SQLContext.scala |  1 +
 .../apache/spark/sql/execution/pythonUdfs.scala |  1 +
 5 files changed, 35 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/aa8f10e8/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 52bd75b..c68c97e 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -803,7 +803,7 @@ class GroupedData(object):
 >>> df.groupBy().mean('age').collect()
 [Row(AVG(age#0)=3.5)]
 >>> df3.groupBy().mean('age', 'height').collect()
-[Row(AVG(age#4)=3.5, AVG(height#5)=82.5)]
+[Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)]
 """
 
 @df_varargs_api
@@ -814,7 +814,7 @@ class GroupedData(object):
 >>> df.groupBy().avg('age').collect()
 [Row(AVG(age#0)=3.5)]
 >>> df3.groupBy().avg('age', 'height').collect()
-[Row(AVG(age#4)=3.5, AVG(height#5)=82.5)]
+[Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)]
 """
 
 @df_varargs_api
@@ -825,7 +825,7 @@ class GroupedData(object):
 >>> df.groupBy().max('age').collect()
 [Row(MAX(age#0)=5)]
 >>> df3.groupBy().max('age', 'height').collect()
-[Row(MAX(age#4)=5, MAX(height#5)=85)]
+[Row(MAX(age#4L)=5, MAX(height#5L)=85)]
 """
 
 @df_varargs_api
@@ -836,7 +836,7 @@ class GroupedData(object):
 >>> df.groupBy().min('age').collect()
 [Row(MIN(age#0)=2)]
 >>> df3.groupBy().min('age', 'height').collect()
-[Row(MIN(age#4)=2, MIN(height#5)=80)]
+[Row(MIN(age#4L)=2, MIN(height#5L)=80)]
 """
 
 @df_varargs_api
@@ -847,7 +847,7 @@ class GroupedData(object):
 >>> df.groupBy().sum('age').collect()
 [Row(SUM(age#0)=7)]
 >>> df3.groupBy().sum('age', 'height').collect()
-[Row(SUM(age#4)=7, SUM(height#5)=165)]
+[Row(SUM(age#4L)=7, SUM(height#5L)=165)]
 """
 
 
@@ -1051,7 +1051,9 @@ def _test():
 sc = SparkContext('local[4]', 'PythonTest')
 globs['sc'] = sc
 globs['sqlCtx'] = SQLContext(sc)
-globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', 
age=5)]).toDF()
+globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\
+.toDF(StructType([StructField('age', IntegerType()),
+  StructField('name', StringType())]))
 globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', 
height=85)]).toDF()
 globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80),
   Row(name='Bob', age=5, height=85)]).toDF()

http://git-wip-us.apache.org/repos/asf/spark/blob/aa8f10e8/python/pyspark/sql/tests.py
--
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index 52f7e65..8e1bb36 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -38,7 +38,7 @@ else:
 
 from pyspark.sql import SQLContext, HiveContext, Column
 from pyspark.sql.types import IntegerType, Row, ArrayType, StructType, 
StructField, \
-UserDefinedType, DoubleType, LongType, StringType
+UserDefinedType, DoubleType, LongType, StringType, _infer_type
 from pyspark.tests import ReusedPySparkTestCase
 
 
@@ -324,6 +324,26 @@ class SQLTests(ReusedPySparkTestCase):
 pydoc.render_doc(df.foo)
 pydoc.render_doc(df.take(1))
 
+def test_infer_long_type(self):
+longrow = [Row(f1='a', f2=100)]
+df = self.sc.parallelize(longrow).toDF()
+self.assertEqual(df.sch

spark git commit: [SPARK-5722] [SQL] [PySpark] infer int as LongType

2015-02-18 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 b86e44cd9 -> 470cba82c


[SPARK-5722] [SQL] [PySpark] infer int as LongType

The `int` is 64-bit on 64-bit machine (very common now), we should infer it as 
LongType for it in Spark SQL.

Also, LongType in SQL will come back as `int`.

Author: Davies Liu 

Closes #4666 from davies/long and squashes the following commits:

6bc6cc4 [Davies Liu] infer int as LongType

(cherry picked from commit aa8f10e82a743d59ce87348af19c0177eb618a66)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 470cba82c021f1ae78ae8a597e2518d9e4045e77
Parents: b86e44c
Author: Davies Liu 
Authored: Wed Feb 18 14:17:04 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 18 14:17:14 2015 -0800

--
 python/pyspark/sql/dataframe.py | 14 +++--
 python/pyspark/sql/tests.py | 22 +++-
 python/pyspark/sql/types.py |  8 +++
 .../scala/org/apache/spark/sql/SQLContext.scala |  1 +
 .../apache/spark/sql/execution/pythonUdfs.scala |  1 +
 5 files changed, 35 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/470cba82/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 52bd75b..c68c97e 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -803,7 +803,7 @@ class GroupedData(object):
 >>> df.groupBy().mean('age').collect()
 [Row(AVG(age#0)=3.5)]
 >>> df3.groupBy().mean('age', 'height').collect()
-[Row(AVG(age#4)=3.5, AVG(height#5)=82.5)]
+[Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)]
 """
 
 @df_varargs_api
@@ -814,7 +814,7 @@ class GroupedData(object):
 >>> df.groupBy().avg('age').collect()
 [Row(AVG(age#0)=3.5)]
 >>> df3.groupBy().avg('age', 'height').collect()
-[Row(AVG(age#4)=3.5, AVG(height#5)=82.5)]
+[Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)]
 """
 
 @df_varargs_api
@@ -825,7 +825,7 @@ class GroupedData(object):
 >>> df.groupBy().max('age').collect()
 [Row(MAX(age#0)=5)]
 >>> df3.groupBy().max('age', 'height').collect()
-[Row(MAX(age#4)=5, MAX(height#5)=85)]
+[Row(MAX(age#4L)=5, MAX(height#5L)=85)]
 """
 
 @df_varargs_api
@@ -836,7 +836,7 @@ class GroupedData(object):
 >>> df.groupBy().min('age').collect()
 [Row(MIN(age#0)=2)]
 >>> df3.groupBy().min('age', 'height').collect()
-[Row(MIN(age#4)=2, MIN(height#5)=80)]
+[Row(MIN(age#4L)=2, MIN(height#5L)=80)]
 """
 
 @df_varargs_api
@@ -847,7 +847,7 @@ class GroupedData(object):
 >>> df.groupBy().sum('age').collect()
 [Row(SUM(age#0)=7)]
 >>> df3.groupBy().sum('age', 'height').collect()
-[Row(SUM(age#4)=7, SUM(height#5)=165)]
+[Row(SUM(age#4L)=7, SUM(height#5L)=165)]
 """
 
 
@@ -1051,7 +1051,9 @@ def _test():
 sc = SparkContext('local[4]', 'PythonTest')
 globs['sc'] = sc
 globs['sqlCtx'] = SQLContext(sc)
-globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', 
age=5)]).toDF()
+globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\
+.toDF(StructType([StructField('age', IntegerType()),
+  StructField('name', StringType())]))
 globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', 
height=85)]).toDF()
 globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80),
   Row(name='Bob', age=5, height=85)]).toDF()

http://git-wip-us.apache.org/repos/asf/spark/blob/470cba82/python/pyspark/sql/tests.py
--
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index 52f7e65..8e1bb36 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -38,7 +38,7 @@ else:
 
 from pyspark.sql import SQLContext, HiveContext, Column
 from pyspark.sql.types import IntegerType, Row, ArrayType, StructType, 
StructField, \
-UserDefinedType, DoubleType, LongType, StringType
+UserDefinedType, DoubleType, LongType, StringType, _infer_type
 from pyspark.tests import ReusedPySparkTestCase
 
 
@@ -324,6 +324,26 @@ class SQLTests(ReusedPySparkTestCase):
 pydoc.render_doc(df.foo)
 pydoc.render_doc(df.take(1))
 
+def test_infer_long_type(self):
+longrow = [Row(

spark git commit: [SPARK-5904][SQL] DataFrame API fixes.

2015-02-19 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 94cdb05ff -> 8ca3418e1


[SPARK-5904][SQL] DataFrame API fixes.

1. Column is no longer a DataFrame to simplify class hierarchy.
2. Don't use varargs on abstract methods (see Scala compiler bug SI-9013).

Author: Reynold Xin 

Closes #4686 from rxin/SPARK-5904 and squashes the following commits:

fd9b199 [Reynold Xin] Fixed Python tests.
df25cef [Reynold Xin] Non final.
5221530 [Reynold Xin] [SPARK-5904][SQL] DataFrame API fixes.


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

Branch: refs/heads/master
Commit: 8ca3418e1b3e2687e75a08c185d17045a97279fb
Parents: 94cdb05
Author: Reynold Xin 
Authored: Thu Feb 19 12:09:44 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 19 12:09:44 2015 -0800

--
 python/pyspark/sql/dataframe.py |  56 +--
 .../scala/org/apache/spark/sql/Column.scala | 223 +++--
 .../org/apache/spark/sql/ComputableColumn.scala |  33 --
 .../scala/org/apache/spark/sql/DataFrame.scala  | 420 
 .../org/apache/spark/sql/DataFrameImpl.scala| 483 ---
 .../org/apache/spark/sql/GroupedData.scala  |   2 +-
 .../apache/spark/sql/IncomputableColumn.scala   | 183 ---
 .../spark/sql/ColumnExpressionSuite.scala   |  44 +-
 .../org/apache/spark/sql/DataFrameSuite.scala   |   7 +-
 9 files changed, 427 insertions(+), 1024 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8ca3418e/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index c68c97e..010c38f 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -546,7 +546,7 @@ class DataFrame(object):
 def __getitem__(self, item):
 """ Return the column by given name
 
->>> df['age'].collect()
+>>> df.select(df['age']).collect()
 [Row(age=2), Row(age=5)]
 >>> df[ ["name", "age"]].collect()
 [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)]
@@ -555,7 +555,7 @@ class DataFrame(object):
 """
 if isinstance(item, basestring):
 jc = self._jdf.apply(item)
-return Column(jc, self.sql_ctx)
+return Column(jc)
 elif isinstance(item, Column):
 return self.filter(item)
 elif isinstance(item, list):
@@ -566,13 +566,13 @@ class DataFrame(object):
 def __getattr__(self, name):
 """ Return the column by given name
 
->>> df.age.collect()
+>>> df.select(df.age).collect()
 [Row(age=2), Row(age=5)]
 """
 if name.startswith("__"):
 raise AttributeError(name)
 jc = self._jdf.apply(name)
-return Column(jc, self.sql_ctx)
+return Column(jc)
 
 def select(self, *cols):
 """ Selecting a set of expressions.
@@ -698,7 +698,7 @@ class DataFrame(object):
 >>> df.withColumnRenamed('age', 'age2').collect()
 [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')]
 """
-cols = [Column(_to_java_column(c), self.sql_ctx).alias(new)
+cols = [Column(_to_java_column(c)).alias(new)
 if c == existing else c
 for c in self.columns]
 return self.select(*cols)
@@ -873,15 +873,16 @@ def _unary_op(name, doc="unary operator"):
 """ Create a method for given unary operator """
 def _(self):
 jc = getattr(self._jc, name)()
-return Column(jc, self.sql_ctx)
+return Column(jc)
 _.__doc__ = doc
 return _
 
 
 def _func_op(name, doc=''):
 def _(self):
-jc = getattr(self._sc._jvm.functions, name)(self._jc)
-return Column(jc, self.sql_ctx)
+sc = SparkContext._active_spark_context
+jc = getattr(sc._jvm.functions, name)(self._jc)
+return Column(jc)
 _.__doc__ = doc
 return _
 
@@ -892,7 +893,7 @@ def _bin_op(name, doc="binary operator"):
 def _(self, other):
 jc = other._jc if isinstance(other, Column) else other
 njc = getattr(self._jc, name)(jc)
-return Column(njc, self.sql_ctx)
+return Column(njc)
 _.__doc__ = doc
 return _
 
@@ -903,12 +904,12 @@ def _reverse_op(name, doc="binary operator"):
 def _(self, other):
 jother = _create_column_from_literal(other)
 jc = getattr(jother, name)(self._jc)
-return Column(jc, self.sql_ctx)
+return Column(jc)
 _.__doc__ = doc
 return _
 
 
-class Column(DataFrame):
+class Column(object):
 
 """
 A column in a DataFrame.
@@ -924,9 +925,8 

spark git commit: [SPARK-5904][SQL] DataFrame API fixes.

2015-02-19 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 fe00eb66e -> 55d91d92b


[SPARK-5904][SQL] DataFrame API fixes.

1. Column is no longer a DataFrame to simplify class hierarchy.
2. Don't use varargs on abstract methods (see Scala compiler bug SI-9013).

Author: Reynold Xin 

Closes #4686 from rxin/SPARK-5904 and squashes the following commits:

fd9b199 [Reynold Xin] Fixed Python tests.
df25cef [Reynold Xin] Non final.
5221530 [Reynold Xin] [SPARK-5904][SQL] DataFrame API fixes.

Conflicts:
sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala


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

Branch: refs/heads/branch-1.3
Commit: 55d91d92bcdd56c479138cbdd9d2354bb8db52a7
Parents: fe00eb6
Author: Reynold Xin 
Authored: Thu Feb 19 12:09:44 2015 -0800
Committer: Michael Armbrust 
Committed: Thu Feb 19 12:15:18 2015 -0800

--
 python/pyspark/sql/dataframe.py |  56 +--
 .../scala/org/apache/spark/sql/Column.scala | 223 +++--
 .../org/apache/spark/sql/ComputableColumn.scala |  33 --
 .../scala/org/apache/spark/sql/DataFrame.scala  | 420 
 .../org/apache/spark/sql/DataFrameImpl.scala| 483 ---
 .../org/apache/spark/sql/GroupedData.scala  |   2 +-
 .../apache/spark/sql/IncomputableColumn.scala   | 183 ---
 .../spark/sql/ColumnExpressionSuite.scala   |  44 +-
 .../org/apache/spark/sql/DataFrameSuite.scala   |   7 +-
 9 files changed, 427 insertions(+), 1024 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/55d91d92/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index c68c97e..010c38f 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -546,7 +546,7 @@ class DataFrame(object):
 def __getitem__(self, item):
 """ Return the column by given name
 
->>> df['age'].collect()
+>>> df.select(df['age']).collect()
 [Row(age=2), Row(age=5)]
 >>> df[ ["name", "age"]].collect()
 [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)]
@@ -555,7 +555,7 @@ class DataFrame(object):
 """
 if isinstance(item, basestring):
 jc = self._jdf.apply(item)
-return Column(jc, self.sql_ctx)
+return Column(jc)
 elif isinstance(item, Column):
 return self.filter(item)
 elif isinstance(item, list):
@@ -566,13 +566,13 @@ class DataFrame(object):
 def __getattr__(self, name):
 """ Return the column by given name
 
->>> df.age.collect()
+>>> df.select(df.age).collect()
 [Row(age=2), Row(age=5)]
 """
 if name.startswith("__"):
 raise AttributeError(name)
 jc = self._jdf.apply(name)
-return Column(jc, self.sql_ctx)
+return Column(jc)
 
 def select(self, *cols):
 """ Selecting a set of expressions.
@@ -698,7 +698,7 @@ class DataFrame(object):
 >>> df.withColumnRenamed('age', 'age2').collect()
 [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')]
 """
-cols = [Column(_to_java_column(c), self.sql_ctx).alias(new)
+cols = [Column(_to_java_column(c)).alias(new)
 if c == existing else c
 for c in self.columns]
 return self.select(*cols)
@@ -873,15 +873,16 @@ def _unary_op(name, doc="unary operator"):
 """ Create a method for given unary operator """
 def _(self):
 jc = getattr(self._jc, name)()
-return Column(jc, self.sql_ctx)
+return Column(jc)
 _.__doc__ = doc
 return _
 
 
 def _func_op(name, doc=''):
 def _(self):
-jc = getattr(self._sc._jvm.functions, name)(self._jc)
-return Column(jc, self.sql_ctx)
+sc = SparkContext._active_spark_context
+jc = getattr(sc._jvm.functions, name)(self._jc)
+return Column(jc)
 _.__doc__ = doc
 return _
 
@@ -892,7 +893,7 @@ def _bin_op(name, doc="binary operator"):
 def _(self, other):
 jc = other._jc if isinstance(other, Column) else other
 njc = getattr(self._jc, name)(jc)
-return Column(njc, self.sql_ctx)
+return Column(njc)
 _.__doc__ = doc
 return _
 
@@ -903,12 +904,12 @@ def _reverse_op(name, doc="binary operator"):
 def _(self, other):
 jother = _create_column_from_literal(other)
 jc = getattr(jother, name)(self._jc)
-return Column(jc, self.sql_ctx)
+return Column(jc)
 _.__doc__ = doc
 return _
 
 
-class Column(

spark git commit: [SPARK-5898] [SPARK-5896] [SQL] [PySpark] create DataFrame from pandas and tuple/list

2015-02-20 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 4a17eedb1 -> 5b0a42cb1


[SPARK-5898] [SPARK-5896] [SQL]  [PySpark] create DataFrame from pandas and 
tuple/list

Fix createDataFrame() from pandas DataFrame (not tested by jenkins, depends on 
SPARK-5693).

It also support to create DataFrame from plain tuple/list without column names, 
`_1`, `_2` will be used as column names.

Author: Davies Liu 

Closes #4679 from davies/pandas and squashes the following commits:

c0cbe0b [Davies Liu] fix tests
8466d1d [Davies Liu] fix create DataFrame from pandas


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

Branch: refs/heads/master
Commit: 5b0a42cb17b840c82d3f8a5ad061d99e261ceadf
Parents: 4a17eed
Author: Davies Liu 
Authored: Fri Feb 20 15:35:05 2015 -0800
Committer: Michael Armbrust 
Committed: Fri Feb 20 15:35:05 2015 -0800

--
 python/pyspark/sql/context.py | 12 ++--
 python/pyspark/sql/tests.py   |  2 +-
 python/pyspark/sql/types.py   | 26 +-
 3 files changed, 20 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5b0a42cb/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 3f168f7..313f15e 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -351,6 +351,8 @@ class SQLContext(object):
 :return: a DataFrame
 
 >>> l = [('Alice', 1)]
+>>> sqlCtx.createDataFrame(l).collect()
+[Row(_1=u'Alice', _2=1)]
 >>> sqlCtx.createDataFrame(l, ['name', 'age']).collect()
 [Row(name=u'Alice', age=1)]
 
@@ -359,6 +361,8 @@ class SQLContext(object):
 [Row(age=1, name=u'Alice')]
 
 >>> rdd = sc.parallelize(l)
+>>> sqlCtx.createDataFrame(rdd).collect()
+[Row(_1=u'Alice', _2=1)]
 >>> df = sqlCtx.createDataFrame(rdd, ['name', 'age'])
 >>> df.collect()
 [Row(name=u'Alice', age=1)]
@@ -377,14 +381,17 @@ class SQLContext(object):
 >>> df3 = sqlCtx.createDataFrame(rdd, schema)
 >>> df3.collect()
 [Row(name=u'Alice', age=1)]
+
+>>> sqlCtx.createDataFrame(df.toPandas()).collect()  # doctest: +SKIP
+[Row(name=u'Alice', age=1)]
 """
 if isinstance(data, DataFrame):
 raise TypeError("data is already a DataFrame")
 
 if has_pandas and isinstance(data, pandas.DataFrame):
-data = self._sc.parallelize(data.to_records(index=False))
 if schema is None:
 schema = list(data.columns)
+data = [r.tolist() for r in data.to_records(index=False)]
 
 if not isinstance(data, RDD):
 try:
@@ -399,7 +406,8 @@ class SQLContext(object):
 if isinstance(schema, (list, tuple)):
 first = data.first()
 if not isinstance(first, (list, tuple)):
-raise ValueError("each row in `rdd` should be list or tuple")
+raise ValueError("each row in `rdd` should be list or tuple, "
+ "but got %r" % type(first))
 row_cls = Row(*schema)
 schema = self._inferSchema(data.map(lambda r: row_cls(*r)), 
samplingRatio)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/5b0a42cb/python/pyspark/sql/tests.py
--
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index 8e1bb36..39071e7 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -186,7 +186,7 @@ class SQLTests(ReusedPySparkTestCase):
 self.assertEqual("2", row.d)
 
 def test_infer_schema(self):
-d = [Row(l=[], d={}),
+d = [Row(l=[], d={}, s=None),
  Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}, s="")]
 rdd = self.sc.parallelize(d)
 df = self.sqlCtx.createDataFrame(rdd)

http://git-wip-us.apache.org/repos/asf/spark/blob/5b0a42cb/python/pyspark/sql/types.py
--
diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py
index 9409c6f..b6e41cf 100644
--- a/python/pyspark/sql/types.py
+++ b/python/pyspark/sql/types.py
@@ -604,7 +604,7 @@ def _infer_type(obj):
 ExamplePointUDT
 """
 if obj is None:
-raise ValueError("Can not infer type for None")
+return NullType()
 
 if hasattr(obj, '__UDT__'):
 return obj.__UDT__
@@ -637,15 +637,14 @@ def _infer_schema(row):
 if isinstance(row, dict):
 items = sorted(row.

spark git commit: [SPARK-5898] [SPARK-5896] [SQL] [PySpark] create DataFrame from pandas and tuple/list

2015-02-20 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 8c12f3114 -> 913562ae7


[SPARK-5898] [SPARK-5896] [SQL]  [PySpark] create DataFrame from pandas and 
tuple/list

Fix createDataFrame() from pandas DataFrame (not tested by jenkins, depends on 
SPARK-5693).

It also support to create DataFrame from plain tuple/list without column names, 
`_1`, `_2` will be used as column names.

Author: Davies Liu 

Closes #4679 from davies/pandas and squashes the following commits:

c0cbe0b [Davies Liu] fix tests
8466d1d [Davies Liu] fix create DataFrame from pandas

(cherry picked from commit 5b0a42cb17b840c82d3f8a5ad061d99e261ceadf)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 913562ae7c3141b2d02419828b9e364867e85d85
Parents: 8c12f31
Author: Davies Liu 
Authored: Fri Feb 20 15:35:05 2015 -0800
Committer: Michael Armbrust 
Committed: Fri Feb 20 15:35:21 2015 -0800

--
 python/pyspark/sql/context.py | 12 ++--
 python/pyspark/sql/tests.py   |  2 +-
 python/pyspark/sql/types.py   | 26 +-
 3 files changed, 20 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/913562ae/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 2e2309f..c5b361a 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -351,6 +351,8 @@ class SQLContext(object):
 :return: a DataFrame
 
 >>> l = [('Alice', 1)]
+>>> sqlCtx.createDataFrame(l).collect()
+[Row(_1=u'Alice', _2=1)]
 >>> sqlCtx.createDataFrame(l, ['name', 'age']).collect()
 [Row(name=u'Alice', age=1)]
 
@@ -359,6 +361,8 @@ class SQLContext(object):
 [Row(age=1, name=u'Alice')]
 
 >>> rdd = sc.parallelize(l)
+>>> sqlCtx.createDataFrame(rdd).collect()
+[Row(_1=u'Alice', _2=1)]
 >>> df = sqlCtx.createDataFrame(rdd, ['name', 'age'])
 >>> df.collect()
 [Row(name=u'Alice', age=1)]
@@ -377,14 +381,17 @@ class SQLContext(object):
 >>> df3 = sqlCtx.createDataFrame(rdd, schema)
 >>> df3.collect()
 [Row(name=u'Alice', age=1)]
+
+>>> sqlCtx.createDataFrame(df.toPandas()).collect()  # doctest: +SKIP
+[Row(name=u'Alice', age=1)]
 """
 if isinstance(data, DataFrame):
 raise TypeError("data is already a DataFrame")
 
 if has_pandas and isinstance(data, pandas.DataFrame):
-data = self._sc.parallelize(data.to_records(index=False))
 if schema is None:
 schema = list(data.columns)
+data = [r.tolist() for r in data.to_records(index=False)]
 
 if not isinstance(data, RDD):
 try:
@@ -399,7 +406,8 @@ class SQLContext(object):
 if isinstance(schema, (list, tuple)):
 first = data.first()
 if not isinstance(first, (list, tuple)):
-raise ValueError("each row in `rdd` should be list or tuple")
+raise ValueError("each row in `rdd` should be list or tuple, "
+ "but got %r" % type(first))
 row_cls = Row(*schema)
 schema = self._inferSchema(data.map(lambda r: row_cls(*r)), 
samplingRatio)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/913562ae/python/pyspark/sql/tests.py
--
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index 8e1bb36..39071e7 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -186,7 +186,7 @@ class SQLTests(ReusedPySparkTestCase):
 self.assertEqual("2", row.d)
 
 def test_infer_schema(self):
-d = [Row(l=[], d={}),
+d = [Row(l=[], d={}, s=None),
  Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}, s="")]
 rdd = self.sc.parallelize(d)
 df = self.sqlCtx.createDataFrame(rdd)

http://git-wip-us.apache.org/repos/asf/spark/blob/913562ae/python/pyspark/sql/types.py
--
diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py
index 9409c6f..b6e41cf 100644
--- a/python/pyspark/sql/types.py
+++ b/python/pyspark/sql/types.py
@@ -604,7 +604,7 @@ def _infer_type(obj):
 ExamplePointUDT
 """
 if obj is None:
-raise ValueError("Can not infer type for None")
+return NullType()
 
 if hasattr(obj, '__UDT__'):
 return obj.__

spark git commit: [SPARK-5909][SQL] Add a clearCache command to Spark SQL's cache manager

2015-02-20 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 913562ae7 -> b9a6c5c84


[SPARK-5909][SQL] Add a clearCache command to Spark SQL's cache manager

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

Author: Yin Huai 

Closes #4694 from yhuai/clearCache and squashes the following commits:

397ecc4 [Yin Huai] Address comments.
a2702fc [Yin Huai] Update parser.
3a54506 [Yin Huai] add isEmpty to CacheManager.
6d14460 [Yin Huai] Python clearCache.
f7b8dbd [Yin Huai] Add clear cache command.


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

Branch: refs/heads/branch-1.3
Commit: b9a6c5c840be1cb4ec4c256920424afbe09c9b37
Parents: 913562a
Author: Yin Huai 
Authored: Fri Feb 20 16:20:02 2015 +0800
Committer: Michael Armbrust 
Committed: Fri Feb 20 15:37:38 2015 -0800

--
 python/pyspark/sql/context.py   |  4 
 .../scala/org/apache/spark/sql/CacheManager.scala   |  6 ++
 .../scala/org/apache/spark/sql/SQLContext.scala |  5 +
 .../scala/org/apache/spark/sql/SparkSQLParser.scala | 11 +++
 .../org/apache/spark/sql/execution/commands.scala   | 15 +++
 .../org/apache/spark/sql/CachedTableSuite.scala | 16 
 6 files changed, 53 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b9a6c5c8/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index c5b361a..313f15e 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -695,6 +695,10 @@ class SQLContext(object):
 """Removes the specified table from the in-memory cache."""
 self._ssql_ctx.uncacheTable(tableName)
 
+def clearCache(self):
+"""Removes all cached tables from the in-memory cache. """
+self._ssql_ctx.clearCache()
+
 
 class HiveContext(SQLContext):
 

http://git-wip-us.apache.org/repos/asf/spark/blob/b9a6c5c8/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala
index f1949aa..ca4a127 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala
@@ -71,11 +71,17 @@ private[sql] class CacheManager(sqlContext: SQLContext) 
extends Logging {
 }
   }
 
+  /** Clears all cached tables. */
   private[sql] def clearCache(): Unit = writeLock {
 cachedData.foreach(_.cachedRepresentation.cachedColumnBuffers.unpersist())
 cachedData.clear()
   }
 
+  /** Checks if the cache is empty. */
+  private[sql] def isEmpty: Boolean = readLock {
+cachedData.isEmpty
+  }
+
   /**
* Caches the data produced by the logical representation of the given 
schema rdd.  Unlike
* `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` 
because recomputing

http://git-wip-us.apache.org/repos/asf/spark/blob/b9a6c5c8/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index a6cf3cd..4bdaa02 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -217,6 +217,11 @@ class SQLContext(@transient val sparkContext: SparkContext)
*/
   def uncacheTable(tableName: String): Unit = 
cacheManager.uncacheTable(tableName)
 
+  /**
+   * Removes all cached tables from the in-memory cache.
+   */
+  def clearCache(): Unit = cacheManager.clearCache()
+
   // scalastyle:off
   // Disable style checker so "implicits" object can start with lowercase i
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/b9a6c5c8/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
index 00e19da..5921eaf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
@@ -23,7 +23,7 @@ import scala.util.parsing.combinator.RegexParsers
 import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
 import org.apache.spark.sql.ca

spark git commit: [SPARK-5935][SQL] Accept MapType in the schema provided to a JSON dataset.

2015-02-23 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 ae9704010 -> 33ccad20e


[SPARK-5935][SQL] Accept MapType in the schema provided to a JSON dataset.

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

Author: Yin Huai 
Author: Yin Huai 

Closes #4710 from yhuai/jsonMapType and squashes the following commits:

3e40390 [Yin Huai] Remove unnecessary changes.
f8e6267 [Yin Huai] Fix test.
baa36e3 [Yin Huai] Accept MapType in the schema provided to jsonFile/jsonRDD.

(cherry picked from commit 48376bfe9c97bf31279918def6c6615849c88f4d)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 33ccad20ef2a4501ea8a8c5983f88f256c3ed478
Parents: ae97040
Author: Yin Huai 
Authored: Mon Feb 23 17:16:34 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 23 17:16:42 2015 -0800

--
 .../org/apache/spark/sql/json/JsonRDD.scala |  3 ++
 .../org/apache/spark/sql/json/JsonSuite.scala   | 56 
 .../apache/spark/sql/json/TestJsonData.scala| 17 ++
 3 files changed, 76 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/33ccad20/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 3b8dde1..d83bdc2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -416,6 +416,9 @@ private[sql] object JsonRDD extends Logging {
 case NullType => null
 case ArrayType(elementType, _) =>
   value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType))
+case MapType(StringType, valueType, _) =>
+  val map = value.asInstanceOf[Map[String, Any]]
+  map.mapValues(enforceCorrectType(_, valueType)).map(identity)
 case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], 
struct)
 case DateType => toDate(value)
 case TimestampType => toTimestamp(value)

http://git-wip-us.apache.org/repos/asf/spark/blob/33ccad20/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
index c94e44b..005f20b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
@@ -657,6 +657,62 @@ class JsonSuite extends QueryTest {
 )
   }
 
+  test("Applying schemas with MapType") {
+val schemaWithSimpleMap = StructType(
+  StructField("map", MapType(StringType, IntegerType, true), false) :: Nil)
+val jsonWithSimpleMap = jsonRDD(mapType1, schemaWithSimpleMap)
+
+jsonWithSimpleMap.registerTempTable("jsonWithSimpleMap")
+
+checkAnswer(
+  sql("select map from jsonWithSimpleMap"),
+  Row(Map("a" -> 1)) ::
+  Row(Map("b" -> 2)) ::
+  Row(Map("c" -> 3)) ::
+  Row(Map("c" -> 1, "d" -> 4)) ::
+  Row(Map("e" -> null)) :: Nil
+)
+
+checkAnswer(
+  sql("select map['c'] from jsonWithSimpleMap"),
+  Row(null) ::
+  Row(null) ::
+  Row(3) ::
+  Row(1) ::
+  Row(null) :: Nil
+)
+
+val innerStruct = StructType(
+  StructField("field1", ArrayType(IntegerType, true), true) ::
+  StructField("field2", IntegerType, true) :: Nil)
+val schemaWithComplexMap = StructType(
+  StructField("map", MapType(StringType, innerStruct, true), false) :: Nil)
+
+val jsonWithComplexMap = jsonRDD(mapType2, schemaWithComplexMap)
+
+jsonWithComplexMap.registerTempTable("jsonWithComplexMap")
+
+checkAnswer(
+  sql("select map from jsonWithComplexMap"),
+  Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) ::
+  Row(Map("b" -> Row(null, 2))) ::
+  Row(Map("c" -> Row(Seq(), 4))) ::
+  Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) ::
+  Row(Map("e" -> null)) ::
+  Row(Map("f" -> Row(null, null))) :: Nil
+)
+
+checkAnswer(
+  sql("select map['a'].field1, map['c'].field2 from jsonWithComplexMap"),
+  Row(Seq(1, 2, 3, null), null) ::
+  Row(null, null) ::
+  Row(null, 4) ::
+  Row(null, 3) ::
+  Row(null, null) ::
+  Row(null, null) :: Nil
+)
+  }
+
   test("SPARK-2096 Correctly parse dot notations") {
 val jsonDF = jsonRDD(complexField

spark git commit: [SPARK-5935][SQL] Accept MapType in the schema provided to a JSON dataset.

2015-02-23 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 59536cc87 -> 48376bfe9


[SPARK-5935][SQL] Accept MapType in the schema provided to a JSON dataset.

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

Author: Yin Huai 
Author: Yin Huai 

Closes #4710 from yhuai/jsonMapType and squashes the following commits:

3e40390 [Yin Huai] Remove unnecessary changes.
f8e6267 [Yin Huai] Fix test.
baa36e3 [Yin Huai] Accept MapType in the schema provided to jsonFile/jsonRDD.


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

Branch: refs/heads/master
Commit: 48376bfe9c97bf31279918def6c6615849c88f4d
Parents: 59536cc
Author: Yin Huai 
Authored: Mon Feb 23 17:16:34 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 23 17:16:34 2015 -0800

--
 .../org/apache/spark/sql/json/JsonRDD.scala |  3 ++
 .../org/apache/spark/sql/json/JsonSuite.scala   | 56 
 .../apache/spark/sql/json/TestJsonData.scala| 17 ++
 3 files changed, 76 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/48376bfe/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 3b8dde1..d83bdc2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -416,6 +416,9 @@ private[sql] object JsonRDD extends Logging {
 case NullType => null
 case ArrayType(elementType, _) =>
   value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType))
+case MapType(StringType, valueType, _) =>
+  val map = value.asInstanceOf[Map[String, Any]]
+  map.mapValues(enforceCorrectType(_, valueType)).map(identity)
 case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], 
struct)
 case DateType => toDate(value)
 case TimestampType => toTimestamp(value)

http://git-wip-us.apache.org/repos/asf/spark/blob/48376bfe/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
index c94e44b..005f20b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
@@ -657,6 +657,62 @@ class JsonSuite extends QueryTest {
 )
   }
 
+  test("Applying schemas with MapType") {
+val schemaWithSimpleMap = StructType(
+  StructField("map", MapType(StringType, IntegerType, true), false) :: Nil)
+val jsonWithSimpleMap = jsonRDD(mapType1, schemaWithSimpleMap)
+
+jsonWithSimpleMap.registerTempTable("jsonWithSimpleMap")
+
+checkAnswer(
+  sql("select map from jsonWithSimpleMap"),
+  Row(Map("a" -> 1)) ::
+  Row(Map("b" -> 2)) ::
+  Row(Map("c" -> 3)) ::
+  Row(Map("c" -> 1, "d" -> 4)) ::
+  Row(Map("e" -> null)) :: Nil
+)
+
+checkAnswer(
+  sql("select map['c'] from jsonWithSimpleMap"),
+  Row(null) ::
+  Row(null) ::
+  Row(3) ::
+  Row(1) ::
+  Row(null) :: Nil
+)
+
+val innerStruct = StructType(
+  StructField("field1", ArrayType(IntegerType, true), true) ::
+  StructField("field2", IntegerType, true) :: Nil)
+val schemaWithComplexMap = StructType(
+  StructField("map", MapType(StringType, innerStruct, true), false) :: Nil)
+
+val jsonWithComplexMap = jsonRDD(mapType2, schemaWithComplexMap)
+
+jsonWithComplexMap.registerTempTable("jsonWithComplexMap")
+
+checkAnswer(
+  sql("select map from jsonWithComplexMap"),
+  Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) ::
+  Row(Map("b" -> Row(null, 2))) ::
+  Row(Map("c" -> Row(Seq(), 4))) ::
+  Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) ::
+  Row(Map("e" -> null)) ::
+  Row(Map("f" -> Row(null, null))) :: Nil
+)
+
+checkAnswer(
+  sql("select map['a'].field1, map['c'].field2 from jsonWithComplexMap"),
+  Row(Seq(1, 2, 3, null), null) ::
+  Row(null, null) ::
+  Row(null, 4) ::
+  Row(null, 3) ::
+  Row(null, null) ::
+  Row(null, null) :: Nil
+)
+  }
+
   test("SPARK-2096 Correctly parse dot notations") {
 val jsonDF = jsonRDD(complexFieldAndType2)
 jsonDF.registerTempTable("jsonTable")

http://git-wip-us.apache.org/repos/asf/spark/blob/48376bf

spark git commit: [SPARK-5722] [SQL] [PySpark] infer int as LongType in Python (for 1.2 branch)

2015-02-23 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 5cea859fd -> 71173de7a


[SPARK-5722] [SQL] [PySpark] infer int as LongType in Python (for 1.2 branch)

This PR change to use LongType for int in Python, when inferSchema(), because 
IntegerType in SQL is not enough for int in Python (which is 64-bit on 64-bit 
machines).

Closes #4521

cc dondrake marmbrus

Author: Davies Liu 

Closes #4681 from davies/long2 and squashes the following commits:

05ef1c8 [Davies Liu] infer LongType for int in Python


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

Branch: refs/heads/branch-1.2
Commit: 71173de7ac1bde60afb4e8473817ec766eda013a
Parents: 5cea859
Author: Davies Liu 
Authored: Mon Feb 23 17:29:25 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 23 17:29:25 2015 -0800

--
 python/pyspark/sql.py   |  8 +++
 python/pyspark/tests.py | 23 +++-
 .../scala/org/apache/spark/sql/SQLContext.scala |  1 +
 .../apache/spark/sql/execution/pythonUdfs.scala |  1 +
 4 files changed, 28 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/71173de7/python/pyspark/sql.py
--
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index ae28847..aa5af1b 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -577,7 +577,7 @@ def _parse_datatype_json_value(json_value):
 _type_mappings = {
 type(None): NullType,
 bool: BooleanType,
-int: IntegerType,
+int: LongType,
 long: LongType,
 float: DoubleType,
 str: StringType,
@@ -926,11 +926,11 @@ def _infer_schema_type(obj, dataType):
 >>> schema = _parse_schema_abstract("a b c d")
 >>> row = (1, 1.0, "str", datetime.date(2014, 10, 10))
 >>> _infer_schema_type(row, schema)
-StructType...IntegerType...DoubleType...StringType...DateType...
+StructType...LongType...DoubleType...StringType...DateType...
 >>> row = [[1], {"key": (1, 2.0)}]
 >>> schema = _parse_schema_abstract("a[] b{c d}")
 >>> _infer_schema_type(row, schema)
-StructType...a,ArrayType...b,MapType(StringType,...c,IntegerType...
+StructType...a,ArrayType...b,MapType(StringType,...c,LongType...
 """
 if dataType is None:
 return _infer_type(obj)
@@ -985,7 +985,7 @@ def _verify_type(obj, dataType):
 
 >>> _verify_type(None, StructType([]))
 >>> _verify_type("", StringType())
->>> _verify_type(0, IntegerType())
+>>> _verify_type(0, LongType())
 >>> _verify_type(range(3), ArrayType(ShortType()))
 >>> _verify_type(set(), ArrayType(StringType())) # doctest: 
+IGNORE_EXCEPTION_DETAIL
 Traceback (most recent call last):

http://git-wip-us.apache.org/repos/asf/spark/blob/71173de7/python/pyspark/tests.py
--
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 1349384..1fc690a 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -51,7 +51,7 @@ from pyspark.serializers import read_int, BatchedSerializer, 
MarshalSerializer,
 CloudPickleSerializer, CompressedSerializer
 from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, 
ExternalSorter
 from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, 
StructField, \
-UserDefinedType, DoubleType
+UserDefinedType, DoubleType, LongType, _infer_type
 from pyspark import shuffle
 
 _have_scipy = False
@@ -985,6 +985,27 @@ class SQLTests(ReusedPySparkTestCase):
 point = srdd1.first().point
 self.assertEquals(point, ExamplePoint(1.0, 2.0))
 
+def test_infer_long_type(self):
+longrow = [Row(f1='a', f2=100)]
+rdd = self.sc.parallelize(longrow)
+srdd = self.sqlCtx.inferSchema(rdd)
+self.assertEqual(srdd.schema().fields[1].dataType, LongType())
+
+# this saving as Parquet caused issues as well.
+output_dir = os.path.join(self.tempdir.name, "infer_long_type")
+srdd.saveAsParquetFile(output_dir)
+df1 = self.sqlCtx.parquetFile(output_dir)
+self.assertEquals('a', df1.first().f1)
+self.assertEquals(100, df1.first().f2)
+
+self.assertEqual(_infer_type(1), LongType())
+self.assertEqual(_infer_type(2**10), LongType())
+self.assertEqual(_infer_type(2**20), LongType(

spark git commit: [SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecution

2015-02-23 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 33ccad20e -> 2d7786ed1


[SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecution

Author: Michael Armbrust 

Closes #4684 from marmbrus/explainAnalysis and squashes the following commits:

afbaa19 [Michael Armbrust] fix python
d93278c [Michael Armbrust] fix hive
e5fa0a4 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into 
explainAnalysis
52119f2 [Michael Armbrust] more tests
82a5431 [Michael Armbrust] fix tests
25753d2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into 
explainAnalysis
aee1e6a [Michael Armbrust] fix hive
b23a844 [Michael Armbrust] newline
de8dc51 [Michael Armbrust] more comments
acf620a [Michael Armbrust] [SPARK-5873][SQL] Show partially analyzed plans in 
query execution

(cherry picked from commit 1ed57086d402c38d95cda6c3d9d7aea806609bf9)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 2d7786ed1e008b33e8b171a8f2ea30e19426ba1f
Parents: 33ccad2
Author: Michael Armbrust 
Authored: Mon Feb 23 17:34:54 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 23 17:35:04 2015 -0800

--
 python/pyspark/sql/context.py   |  30 +++---
 .../apache/spark/sql/catalyst/SqlParser.scala   |   2 +
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  83 ---
 .../sql/catalyst/analysis/CheckAnalysis.scala   | 105 +++
 .../sql/catalyst/analysis/AnalysisSuite.scala   |  35 ---
 .../scala/org/apache/spark/sql/DataFrame.scala  |   2 +-
 .../scala/org/apache/spark/sql/SQLConf.scala|   5 +-
 .../scala/org/apache/spark/sql/SQLContext.scala |  14 ++-
 .../org/apache/spark/sql/sources/rules.scala|  10 +-
 .../spark/sql/sources/DataSourceTest.scala  |   1 -
 .../apache/spark/sql/sources/InsertSuite.scala  |   2 +-
 .../org/apache/spark/sql/hive/HiveContext.scala |   1 -
 12 files changed, 164 insertions(+), 126 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2d7786ed/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 313f15e..125933c 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -267,20 +267,20 @@ class SQLContext(object):
 ... StructField("byte2", ByteType(), False),
 ... StructField("short1", ShortType(), False),
 ... StructField("short2", ShortType(), False),
-... StructField("int", IntegerType(), False),
-... StructField("float", FloatType(), False),
-... StructField("date", DateType(), False),
-... StructField("time", TimestampType(), False),
-... StructField("map",
+... StructField("int1", IntegerType(), False),
+... StructField("float1", FloatType(), False),
+... StructField("date1", DateType(), False),
+... StructField("time1", TimestampType(), False),
+... StructField("map1",
 ... MapType(StringType(), IntegerType(), False), False),
-... StructField("struct",
+... StructField("struct1",
 ... StructType([StructField("b", ShortType(), False)]), False),
-... StructField("list", ArrayType(ByteType(), False), False),
-... StructField("null", DoubleType(), True)])
+... StructField("list1", ArrayType(ByteType(), False), False),
+... StructField("null1", DoubleType(), True)])
 >>> df = sqlCtx.applySchema(rdd, schema)
 >>> results = df.map(
-... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, 
x.float, x.date,
-... x.time, x.map["a"], x.struct.b, x.list, x.null))
+... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int1, 
x.float1, x.date1,
+... x.time1, x.map1["a"], x.struct1.b, x.list1, x.null1))
 >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE
 (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1),
  datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None)
@@ -288,20 +288,20 @@ class SQLContext(object):
 >>> df.registerTempTable("table2")
 >

spark git commit: [SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecution

2015-02-23 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 48376bfe9 -> 1ed57086d


[SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecution

Author: Michael Armbrust 

Closes #4684 from marmbrus/explainAnalysis and squashes the following commits:

afbaa19 [Michael Armbrust] fix python
d93278c [Michael Armbrust] fix hive
e5fa0a4 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into 
explainAnalysis
52119f2 [Michael Armbrust] more tests
82a5431 [Michael Armbrust] fix tests
25753d2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into 
explainAnalysis
aee1e6a [Michael Armbrust] fix hive
b23a844 [Michael Armbrust] newline
de8dc51 [Michael Armbrust] more comments
acf620a [Michael Armbrust] [SPARK-5873][SQL] Show partially analyzed plans in 
query execution


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

Branch: refs/heads/master
Commit: 1ed57086d402c38d95cda6c3d9d7aea806609bf9
Parents: 48376bf
Author: Michael Armbrust 
Authored: Mon Feb 23 17:34:54 2015 -0800
Committer: Michael Armbrust 
Committed: Mon Feb 23 17:34:54 2015 -0800

--
 python/pyspark/sql/context.py   |  30 +++---
 .../apache/spark/sql/catalyst/SqlParser.scala   |   2 +
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  83 ---
 .../sql/catalyst/analysis/CheckAnalysis.scala   | 105 +++
 .../sql/catalyst/analysis/AnalysisSuite.scala   |  35 ---
 .../scala/org/apache/spark/sql/DataFrame.scala  |   2 +-
 .../scala/org/apache/spark/sql/SQLConf.scala|   5 +-
 .../scala/org/apache/spark/sql/SQLContext.scala |  14 ++-
 .../org/apache/spark/sql/sources/rules.scala|  10 +-
 .../spark/sql/sources/DataSourceTest.scala  |   1 -
 .../apache/spark/sql/sources/InsertSuite.scala  |   2 +-
 .../org/apache/spark/sql/hive/HiveContext.scala |   1 -
 12 files changed, 164 insertions(+), 126 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1ed57086/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 313f15e..125933c 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -267,20 +267,20 @@ class SQLContext(object):
 ... StructField("byte2", ByteType(), False),
 ... StructField("short1", ShortType(), False),
 ... StructField("short2", ShortType(), False),
-... StructField("int", IntegerType(), False),
-... StructField("float", FloatType(), False),
-... StructField("date", DateType(), False),
-... StructField("time", TimestampType(), False),
-... StructField("map",
+... StructField("int1", IntegerType(), False),
+... StructField("float1", FloatType(), False),
+... StructField("date1", DateType(), False),
+... StructField("time1", TimestampType(), False),
+... StructField("map1",
 ... MapType(StringType(), IntegerType(), False), False),
-... StructField("struct",
+... StructField("struct1",
 ... StructType([StructField("b", ShortType(), False)]), False),
-... StructField("list", ArrayType(ByteType(), False), False),
-... StructField("null", DoubleType(), True)])
+... StructField("list1", ArrayType(ByteType(), False), False),
+... StructField("null1", DoubleType(), True)])
 >>> df = sqlCtx.applySchema(rdd, schema)
 >>> results = df.map(
-... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, 
x.float, x.date,
-... x.time, x.map["a"], x.struct.b, x.list, x.null))
+... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int1, 
x.float1, x.date1,
+... x.time1, x.map1["a"], x.struct1.b, x.list1, x.null1))
 >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE
 (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1),
  datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None)
@@ -288,20 +288,20 @@ class SQLContext(object):
 >>> df.registerTempTable("table2")
 >>> sqlCtx.sql(
 ...   "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " +
-..

spark git commit: [SPARK-5968] [SQL] Suppresses ParquetOutputCommitter WARN logs

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master cf2e41653 -> 840333133


[SPARK-5968] [SQL] Suppresses ParquetOutputCommitter WARN logs

Please refer to the [JIRA ticket] [1] for the motivation.

[1]: https://issues.apache.org/jira/browse/SPARK-5968


[https://reviewable.io/review_button.png"; height=40 alt="Review on 
Reviewable"/>](https://reviewable.io/reviews/apache/spark/4744)


Author: Cheng Lian 

Closes #4744 from liancheng/spark-5968 and squashes the following commits:

caac6a8 [Cheng Lian] Suppresses ParquetOutputCommitter WARN logs


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

Branch: refs/heads/master
Commit: 840333133396d443e747f62fce9967f7681fb276
Parents: cf2e416
Author: Cheng Lian 
Authored: Tue Feb 24 10:45:38 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 10:45:38 2015 -0800

--
 .../org/apache/spark/sql/parquet/ParquetRelation.scala  | 12 +---
 1 file changed, 9 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/84033313/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
index b0db994..a0d1005 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -18,11 +18,12 @@
 package org.apache.spark.sql.parquet
 
 import java.io.IOException
+import java.util.logging.Level
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.fs.permission.FsAction
-import parquet.hadoop.ParquetOutputFormat
+import parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat}
 import parquet.hadoop.metadata.CompressionCodecName
 import parquet.schema.MessageType
 
@@ -91,7 +92,7 @@ private[sql] object ParquetRelation {
 // checks first to see if there's any handlers already set
 // and if not it creates them. If this method executes prior
 // to that class being loaded then:
-//  1) there's no handlers installed so there's none to 
+//  1) there's no handlers installed so there's none to
 // remove. But when it IS finally loaded the desired affect
 // of removing them is circumvented.
 //  2) The parquet.Log static initializer calls setUseParentHanders(false)
@@ -99,7 +100,7 @@ private[sql] object ParquetRelation {
 //
 // Therefore we need to force the class to be loaded.
 // This should really be resolved by Parquet.
-Class.forName(classOf[parquet.Log].getName())
+Class.forName(classOf[parquet.Log].getName)
 
 // Note: Logger.getLogger("parquet") has a default logger
 // that appends to Console which needs to be cleared.
@@ -108,6 +109,11 @@ private[sql] object ParquetRelation {
 // TODO(witgo): Need to set the log level ?
 // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null)
 if (!parquetLogger.getUseParentHandlers) 
parquetLogger.setUseParentHandlers(true)
+
+// Disables WARN log message in ParquetOutputCommitter.
+// See https://issues.apache.org/jira/browse/SPARK-5968 for details
+Class.forName(classOf[ParquetOutputCommitter].getName)
+
java.util.logging.Logger.getLogger(classOf[ParquetOutputCommitter].getName).setLevel(Level.OFF)
   }
 
   // The element type for the RDDs that this relation maps to.


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



spark git commit: [SPARK-5968] [SQL] Suppresses ParquetOutputCommitter WARN logs

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 dd4255850 -> 2b562b043


[SPARK-5968] [SQL] Suppresses ParquetOutputCommitter WARN logs

Please refer to the [JIRA ticket] [1] for the motivation.

[1]: https://issues.apache.org/jira/browse/SPARK-5968


[https://reviewable.io/review_button.png"; height=40 alt="Review on 
Reviewable"/>](https://reviewable.io/reviews/apache/spark/4744)


Author: Cheng Lian 

Closes #4744 from liancheng/spark-5968 and squashes the following commits:

caac6a8 [Cheng Lian] Suppresses ParquetOutputCommitter WARN logs

(cherry picked from commit 840333133396d443e747f62fce9967f7681fb276)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 2b562b04324229e43a48803fe9aff65ee4af21e8
Parents: dd42558
Author: Cheng Lian 
Authored: Tue Feb 24 10:45:38 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 10:47:02 2015 -0800

--
 .../org/apache/spark/sql/parquet/ParquetRelation.scala  | 12 +---
 1 file changed, 9 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2b562b04/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
index b0db994..a0d1005 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -18,11 +18,12 @@
 package org.apache.spark.sql.parquet
 
 import java.io.IOException
+import java.util.logging.Level
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.fs.permission.FsAction
-import parquet.hadoop.ParquetOutputFormat
+import parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat}
 import parquet.hadoop.metadata.CompressionCodecName
 import parquet.schema.MessageType
 
@@ -91,7 +92,7 @@ private[sql] object ParquetRelation {
 // checks first to see if there's any handlers already set
 // and if not it creates them. If this method executes prior
 // to that class being loaded then:
-//  1) there's no handlers installed so there's none to 
+//  1) there's no handlers installed so there's none to
 // remove. But when it IS finally loaded the desired affect
 // of removing them is circumvented.
 //  2) The parquet.Log static initializer calls setUseParentHanders(false)
@@ -99,7 +100,7 @@ private[sql] object ParquetRelation {
 //
 // Therefore we need to force the class to be loaded.
 // This should really be resolved by Parquet.
-Class.forName(classOf[parquet.Log].getName())
+Class.forName(classOf[parquet.Log].getName)
 
 // Note: Logger.getLogger("parquet") has a default logger
 // that appends to Console which needs to be cleared.
@@ -108,6 +109,11 @@ private[sql] object ParquetRelation {
 // TODO(witgo): Need to set the log level ?
 // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null)
 if (!parquetLogger.getUseParentHandlers) 
parquetLogger.setUseParentHandlers(true)
+
+// Disables WARN log message in ParquetOutputCommitter.
+// See https://issues.apache.org/jira/browse/SPARK-5968 for details
+Class.forName(classOf[ParquetOutputCommitter].getName)
+
java.util.logging.Logger.getLogger(classOf[ParquetOutputCommitter].getName).setLevel(Level.OFF)
   }
 
   // The element type for the RDDs that this relation maps to.


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



spark git commit: [SPARK-5910][SQL] Support for as in selectExpr

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 840333133 -> 0a59e45e2


[SPARK-5910][SQL] Support for as in selectExpr

Author: Michael Armbrust 

Closes #4736 from marmbrus/asExprs and squashes the following commits:

5ba97e4 [Michael Armbrust] [SPARK-5910][SQL] Support for as in selectExpr


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

Branch: refs/heads/master
Commit: 0a59e45e2f2e6f00ccd5f10c79f629fb796fd8d0
Parents: 8403331
Author: Michael Armbrust 
Authored: Tue Feb 24 10:49:51 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 10:49:51 2015 -0800

--
 .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala   | 2 +-
 .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala   | 6 ++
 2 files changed, 7 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0a59e45e/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 b16aff9..c363a5e 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
@@ -40,7 +40,7 @@ class SqlParser extends AbstractSparkSQLParser {
   def parseExpression(input: String): Expression = {
 // Initialize the Keywords.
 lexical.initialize(reservedWords)
-phrase(expression)(new lexical.Scanner(input)) match {
+phrase(projection)(new lexical.Scanner(input)) match {
   case Success(plan, _) => plan
   case failureOrError => sys.error(failureOrError.toString)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0a59e45e/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 6b9b3a8..e71e9be 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
@@ -130,6 +130,12 @@ class DataFrameSuite extends QueryTest {
   testData.collect().map(row => Row(math.abs(row.getInt(0)), 
row.getString(1))).toSeq)
   }
 
+  test("selectExpr with alias") {
+checkAnswer(
+  testData.selectExpr("key as k").select("k"),
+  testData.select("key").collect().toSeq)
+  }
+
   test("filterExpr") {
 checkAnswer(
   testData.filter("key > 90"),


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



spark git commit: [SPARK-5910][SQL] Support for as in selectExpr

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 2b562b043 -> ba5d60dda


[SPARK-5910][SQL] Support for as in selectExpr

Author: Michael Armbrust 

Closes #4736 from marmbrus/asExprs and squashes the following commits:

5ba97e4 [Michael Armbrust] [SPARK-5910][SQL] Support for as in selectExpr

(cherry picked from commit 0a59e45e2f2e6f00ccd5f10c79f629fb796fd8d0)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: ba5d60dda770fe1c1f8034149495009ef65749e2
Parents: 2b562b0
Author: Michael Armbrust 
Authored: Tue Feb 24 10:49:51 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 10:49:58 2015 -0800

--
 .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala   | 2 +-
 .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala   | 6 ++
 2 files changed, 7 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ba5d60dd/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 b16aff9..c363a5e 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
@@ -40,7 +40,7 @@ class SqlParser extends AbstractSparkSQLParser {
   def parseExpression(input: String): Expression = {
 // Initialize the Keywords.
 lexical.initialize(reservedWords)
-phrase(expression)(new lexical.Scanner(input)) match {
+phrase(projection)(new lexical.Scanner(input)) match {
   case Success(plan, _) => plan
   case failureOrError => sys.error(failureOrError.toString)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/ba5d60dd/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 6b9b3a8..e71e9be 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
@@ -130,6 +130,12 @@ class DataFrameSuite extends QueryTest {
   testData.collect().map(row => Row(math.abs(row.getInt(0)), 
row.getString(1))).toSeq)
   }
 
+  test("selectExpr with alias") {
+checkAnswer(
+  testData.selectExpr("key as k").select("k"),
+  testData.select("key").collect().toSeq)
+  }
+
   test("filterExpr") {
 checkAnswer(
   testData.filter("key > 90"),


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



spark git commit: [SPARK-5532][SQL] Repartition should not use external rdd representation

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 0a59e45e2 -> 201236628


[SPARK-5532][SQL] Repartition should not use external rdd representation

Author: Michael Armbrust 

Closes #4738 from marmbrus/udtRepart and squashes the following commits:

c06d7b5 [Michael Armbrust] fix compilation
91c8829 [Michael Armbrust] [SQL][SPARK-5532] Repartition should not use 
external rdd representation


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

Branch: refs/heads/master
Commit: 201236628a344194f7c20ba8e9afeeaefbe9318c
Parents: 0a59e45
Author: Michael Armbrust 
Authored: Tue Feb 24 10:52:18 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 10:52:18 2015 -0800

--
 .../main/scala/org/apache/spark/sql/DataFrame.scala |  5 +++--
 .../apache/spark/sql/execution/debug/package.scala  |  1 +
 .../org/apache/spark/sql/UserDefinedTypeSuite.scala | 16 +++-
 3 files changed, 19 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/20123662/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 27ac398..04bf5d9 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
@@ -799,14 +799,15 @@ class DataFrame protected[sql](
* Returns the number of rows in the [[DataFrame]].
* @group action
*/
-  override def count(): Long = groupBy().count().rdd.collect().head.getLong(0)
+  override def count(): Long = groupBy().count().collect().head.getLong(0)
 
   /**
* Returns a new [[DataFrame]] that has exactly `numPartitions` partitions.
* @group rdd
*/
   override def repartition(numPartitions: Int): DataFrame = {
-sqlContext.createDataFrame(rdd.repartition(numPartitions), schema)
+sqlContext.createDataFrame(
+  queryExecution.toRdd.map(_.copy()).repartition(numPartitions), schema)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/20123662/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
index 73162b2..ffe388c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
@@ -168,6 +168,7 @@ package object debug {
   case (_: Short, ShortType) =>
   case (_: Boolean, BooleanType) =>
   case (_: Double, DoubleType) =>
+  case (v, udt: UserDefinedType[_]) => typeCheck(v, udt.sqlType)
 
   case (d, t) => sys.error(s"Invalid data found: got $d (${d.getClass}) 
expected $t")
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/20123662/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index 5f21d99..9c098df 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -17,10 +17,11 @@
 
 package org.apache.spark.sql
 
+import java.io.File
+
 import scala.beans.{BeanInfo, BeanProperty}
 
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.functions._
 import org.apache.spark.sql.test.TestSQLContext
 import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql}
 import org.apache.spark.sql.test.TestSQLContext.implicits._
@@ -91,4 +92,17 @@ class UserDefinedTypeSuite extends QueryTest {
   sql("SELECT testType(features) from points"),
   Seq(Row(true), Row(true)))
   }
+
+
+  test("UDTs with Parquet") {
+val tempDir = File.createTempFile("parquet", "test")
+tempDir.delete()
+pointsRDD.saveAsParquetFile(tempDir.getCanonicalPath)
+  }
+
+  test("Repartition UDTs with Parquet") {
+val tempDir = File.createTempFile("parquet", "test")
+tempDir.delete()
+pointsRDD.repartition(1).saveAsParquetFile(tempDir.getCanonicalPath)
+  }
 }


-
To un

spark git commit: [SPARK-5532][SQL] Repartition should not use external rdd representation

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 ba5d60dda -> e46096b1e


[SPARK-5532][SQL] Repartition should not use external rdd representation

Author: Michael Armbrust 

Closes #4738 from marmbrus/udtRepart and squashes the following commits:

c06d7b5 [Michael Armbrust] fix compilation
91c8829 [Michael Armbrust] [SQL][SPARK-5532] Repartition should not use 
external rdd representation

(cherry picked from commit 201236628a344194f7c20ba8e9afeeaefbe9318c)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: e46096b1e9f173c9f65425c3980b6f32edb3bf99
Parents: ba5d60d
Author: Michael Armbrust 
Authored: Tue Feb 24 10:52:18 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 10:52:32 2015 -0800

--
 .../main/scala/org/apache/spark/sql/DataFrame.scala |  5 +++--
 .../apache/spark/sql/execution/debug/package.scala  |  1 +
 .../org/apache/spark/sql/UserDefinedTypeSuite.scala | 16 +++-
 3 files changed, 19 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e46096b1/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 27ac398..04bf5d9 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
@@ -799,14 +799,15 @@ class DataFrame protected[sql](
* Returns the number of rows in the [[DataFrame]].
* @group action
*/
-  override def count(): Long = groupBy().count().rdd.collect().head.getLong(0)
+  override def count(): Long = groupBy().count().collect().head.getLong(0)
 
   /**
* Returns a new [[DataFrame]] that has exactly `numPartitions` partitions.
* @group rdd
*/
   override def repartition(numPartitions: Int): DataFrame = {
-sqlContext.createDataFrame(rdd.repartition(numPartitions), schema)
+sqlContext.createDataFrame(
+  queryExecution.toRdd.map(_.copy()).repartition(numPartitions), schema)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/e46096b1/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
index 73162b2..ffe388c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
@@ -168,6 +168,7 @@ package object debug {
   case (_: Short, ShortType) =>
   case (_: Boolean, BooleanType) =>
   case (_: Double, DoubleType) =>
+  case (v, udt: UserDefinedType[_]) => typeCheck(v, udt.sqlType)
 
   case (d, t) => sys.error(s"Invalid data found: got $d (${d.getClass}) 
expected $t")
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/e46096b1/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index 5f21d99..9c098df 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -17,10 +17,11 @@
 
 package org.apache.spark.sql
 
+import java.io.File
+
 import scala.beans.{BeanInfo, BeanProperty}
 
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.functions._
 import org.apache.spark.sql.test.TestSQLContext
 import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql}
 import org.apache.spark.sql.test.TestSQLContext.implicits._
@@ -91,4 +92,17 @@ class UserDefinedTypeSuite extends QueryTest {
   sql("SELECT testType(features) from points"),
   Seq(Row(true), Row(true)))
   }
+
+
+  test("UDTs with Parquet") {
+val tempDir = File.createTempFile("parquet", "test")
+tempDir.delete()
+pointsRDD.saveAsParquetFile(tempDir.getCanonicalPath)
+  }
+
+  test("Repartition UDTs with Parquet") {
+val tempDir = File.createTempFile("parquet", "test")
+tempDir.delete()
+pointsRDD.repartition(1).saveAsParquetFile(tempDir.ge

spark git commit: [SPARK-5952][SQL] Lock when using hive metastore client

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master c5ba975ee -> a2b913792


[SPARK-5952][SQL] Lock when using hive metastore client

Author: Michael Armbrust 

Closes #4746 from marmbrus/hiveLock and squashes the following commits:

8b871cf [Michael Armbrust] [SPARK-5952][SQL] Lock when using hive metastore 
client


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

Branch: refs/heads/master
Commit: a2b9137923e0ba328da8fff2fbbfcf2abf50b033
Parents: c5ba975
Author: Michael Armbrust 
Authored: Tue Feb 24 13:39:29 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 13:39:29 2015 -0800

--
 .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 --
 1 file changed, 12 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a2b91379/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 f7ad2ef..2cc8d65 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
@@ -52,6 +52,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
   /** Connection to hive metastore.  Usages should lock on `this`. */
   protected[hive] val client = Hive.get(hive.hiveconf)
 
+  /** Usages should lock on `this`. */
   protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf)
 
   // TODO: Use this everywhere instead of tuples or databaseName, tableName,.
@@ -65,7 +66,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
 val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() {
   override def load(in: QualifiedTableName): LogicalPlan = {
 logDebug(s"Creating new cached data source for $in")
-val table = client.getTable(in.database, in.name)
+val table = synchronized {
+  client.getTable(in.database, in.name)
+}
 val schemaString = table.getProperty("spark.sql.sources.schema")
 val userSpecifiedSchema =
   if (schemaString == null) {
@@ -134,15 +137,18 @@ private[hive] class HiveMetastoreCatalog(hive: 
HiveContext) extends Catalog with
 }
   }
 
-  def hiveDefaultTableFilePath(tableName: String): String = {
+  def hiveDefaultTableFilePath(tableName: String): String = synchronized {
 val currentDatabase = 
client.getDatabase(hive.sessionState.getCurrentDatabase)
+
 hiveWarehouse.getTablePath(currentDatabase, tableName).toString
   }
 
-  def tableExists(tableIdentifier: Seq[String]): Boolean = {
+  def tableExists(tableIdentifier: Seq[String]): Boolean = synchronized {
 val tableIdent = processTableIdentifier(tableIdentifier)
-val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse(
-  hive.sessionState.getCurrentDatabase)
+val databaseName =
+  tableIdent
+.lift(tableIdent.size - 2)
+.getOrElse(hive.sessionState.getCurrentDatabase)
 val tblName = tableIdent.last
 client.getTable(databaseName, tblName, false) != null
   }
@@ -219,7 +225,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
 }
   }
 
-  override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] 
= {
+  override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] 
= synchronized {
 val dbName = if (!caseSensitive) {
   if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None
 } else {


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



spark git commit: [SPARK-5952][SQL] Lock when using hive metastore client

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 a4ff445a9 -> 641423dbf


[SPARK-5952][SQL] Lock when using hive metastore client

Author: Michael Armbrust 

Closes #4746 from marmbrus/hiveLock and squashes the following commits:

8b871cf [Michael Armbrust] [SPARK-5952][SQL] Lock when using hive metastore 
client

(cherry picked from commit a2b9137923e0ba328da8fff2fbbfcf2abf50b033)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 641423dbffd9333ea8d989d0afa7b78426bd3979
Parents: a4ff445
Author: Michael Armbrust 
Authored: Tue Feb 24 13:39:29 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 13:39:42 2015 -0800

--
 .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 --
 1 file changed, 12 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/641423db/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 f7ad2ef..2cc8d65 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
@@ -52,6 +52,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
   /** Connection to hive metastore.  Usages should lock on `this`. */
   protected[hive] val client = Hive.get(hive.hiveconf)
 
+  /** Usages should lock on `this`. */
   protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf)
 
   // TODO: Use this everywhere instead of tuples or databaseName, tableName,.
@@ -65,7 +66,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
 val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() {
   override def load(in: QualifiedTableName): LogicalPlan = {
 logDebug(s"Creating new cached data source for $in")
-val table = client.getTable(in.database, in.name)
+val table = synchronized {
+  client.getTable(in.database, in.name)
+}
 val schemaString = table.getProperty("spark.sql.sources.schema")
 val userSpecifiedSchema =
   if (schemaString == null) {
@@ -134,15 +137,18 @@ private[hive] class HiveMetastoreCatalog(hive: 
HiveContext) extends Catalog with
 }
   }
 
-  def hiveDefaultTableFilePath(tableName: String): String = {
+  def hiveDefaultTableFilePath(tableName: String): String = synchronized {
 val currentDatabase = 
client.getDatabase(hive.sessionState.getCurrentDatabase)
+
 hiveWarehouse.getTablePath(currentDatabase, tableName).toString
   }
 
-  def tableExists(tableIdentifier: Seq[String]): Boolean = {
+  def tableExists(tableIdentifier: Seq[String]): Boolean = synchronized {
 val tableIdent = processTableIdentifier(tableIdentifier)
-val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse(
-  hive.sessionState.getCurrentDatabase)
+val databaseName =
+  tableIdent
+.lift(tableIdent.size - 2)
+.getOrElse(hive.sessionState.getCurrentDatabase)
 val tblName = tableIdent.last
 client.getTable(databaseName, tblName, false) != null
   }
@@ -219,7 +225,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
 }
   }
 
-  override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] 
= {
+  override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] 
= synchronized {
 val dbName = if (!caseSensitive) {
   if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None
 } else {


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



spark git commit: [SPARK-5904][SQL] DataFrame Java API test suites.

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 17ee2460a -> 78a1781a9


[SPARK-5904][SQL] DataFrame Java API test suites.

Added a new test suite to make sure Java DF programs can use varargs properly.
Also moved all suites into test.org.apache.spark package to make sure the 
suites also test for method visibility.

Author: Reynold Xin 

Closes #4751 from rxin/df-tests and squashes the following commits:

1e8b8e4 [Reynold Xin] Fixed imports and renamed JavaAPISuite.
a6ca53b [Reynold Xin] [SPARK-5904][SQL] DataFrame Java API test suites.

(cherry picked from commit 53a1ebf33b5c349ae3a40d7eebf357b839b363af)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 78a1781a96df476a63e1cd61e89107fa4ed7bf55
Parents: 17ee246
Author: Reynold Xin 
Authored: Tue Feb 24 18:51:41 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 18:51:53 2015 -0800

--
 .../apache/spark/sql/api/java/JavaAPISuite.java |  86 
 .../sql/api/java/JavaApplySchemaSuite.java  | 206 ---
 .../org/apache/spark/sql/api/java/JavaDsl.java  | 120 ---
 .../apache/spark/sql/api/java/JavaRowSuite.java | 179 
 .../spark/sql/sources/JavaSaveLoadSuite.java|  97 -
 .../apache/spark/sql/JavaApplySchemaSuite.java  | 204 ++
 .../apache/spark/sql/JavaDataFrameSuite.java|  84 
 .../test/org/apache/spark/sql/JavaRowSuite.java | 179 
 .../test/org/apache/spark/sql/JavaUDFSuite.java |  88 
 .../spark/sql/sources/JavaSaveLoadSuite.java|  98 +
 .../org/apache/spark/sql/DataFrameSuite.scala   |   4 +-
 11 files changed, 655 insertions(+), 690 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/78a1781a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
--
diff --git 
a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java 
b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
deleted file mode 100644
index a21a154..000
--- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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 License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.api.java;
-
-import java.io.Serializable;
-
-import org.apache.spark.sql.test.TestSQLContext$;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.sql.Row;
-import org.apache.spark.sql.SQLContext;
-import org.apache.spark.sql.types.DataTypes;
-
-// The test suite itself is Serializable so that anonymous Function 
implementations can be
-// serialized, as an alternative to converting these anonymous classes to 
static inner classes;
-// see http://stackoverflow.com/questions/758570/.
-public class JavaAPISuite implements Serializable {
-  private transient JavaSparkContext sc;
-  private transient SQLContext sqlContext;
-
-  @Before
-  public void setUp() {
-sqlContext = TestSQLContext$.MODULE$;
-sc = new JavaSparkContext(sqlContext.sparkContext());
-  }
-
-  @After
-  public void tearDown() {
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void udf1Test() {
-// With Java 8 lambdas:
-// sqlContext.registerFunction(
-//   "stringLengthTest", (String str) -> str.length(), 
DataType.IntegerType);
-
-sqlContext.udf().register("stringLengthTest", new UDF1() {
-  @Override
-  public Integer call(String str) throws Exception {
-return str.length();
-  }
-}, DataTypes.IntegerType);
-
-Row result = sqlContext.sql("SELECT stringLengthTest('test')").head();
-assert(result.getInt(0) == 4);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void udf2Test() {

spark git commit: [SPARK-5904][SQL] DataFrame Java API test suites.

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master f816e7390 -> 53a1ebf33


[SPARK-5904][SQL] DataFrame Java API test suites.

Added a new test suite to make sure Java DF programs can use varargs properly.
Also moved all suites into test.org.apache.spark package to make sure the 
suites also test for method visibility.

Author: Reynold Xin 

Closes #4751 from rxin/df-tests and squashes the following commits:

1e8b8e4 [Reynold Xin] Fixed imports and renamed JavaAPISuite.
a6ca53b [Reynold Xin] [SPARK-5904][SQL] DataFrame Java API test suites.


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

Branch: refs/heads/master
Commit: 53a1ebf33b5c349ae3a40d7eebf357b839b363af
Parents: f816e73
Author: Reynold Xin 
Authored: Tue Feb 24 18:51:41 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 18:51:41 2015 -0800

--
 .../apache/spark/sql/api/java/JavaAPISuite.java |  86 
 .../sql/api/java/JavaApplySchemaSuite.java  | 206 ---
 .../org/apache/spark/sql/api/java/JavaDsl.java  | 120 ---
 .../apache/spark/sql/api/java/JavaRowSuite.java | 179 
 .../spark/sql/sources/JavaSaveLoadSuite.java|  97 -
 .../apache/spark/sql/JavaApplySchemaSuite.java  | 204 ++
 .../apache/spark/sql/JavaDataFrameSuite.java|  84 
 .../test/org/apache/spark/sql/JavaRowSuite.java | 179 
 .../test/org/apache/spark/sql/JavaUDFSuite.java |  88 
 .../spark/sql/sources/JavaSaveLoadSuite.java|  98 +
 .../org/apache/spark/sql/DataFrameSuite.scala   |   4 +-
 11 files changed, 655 insertions(+), 690 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/53a1ebf3/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
--
diff --git 
a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java 
b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
deleted file mode 100644
index a21a154..000
--- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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 License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.api.java;
-
-import java.io.Serializable;
-
-import org.apache.spark.sql.test.TestSQLContext$;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.sql.Row;
-import org.apache.spark.sql.SQLContext;
-import org.apache.spark.sql.types.DataTypes;
-
-// The test suite itself is Serializable so that anonymous Function 
implementations can be
-// serialized, as an alternative to converting these anonymous classes to 
static inner classes;
-// see http://stackoverflow.com/questions/758570/.
-public class JavaAPISuite implements Serializable {
-  private transient JavaSparkContext sc;
-  private transient SQLContext sqlContext;
-
-  @Before
-  public void setUp() {
-sqlContext = TestSQLContext$.MODULE$;
-sc = new JavaSparkContext(sqlContext.sparkContext());
-  }
-
-  @After
-  public void tearDown() {
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void udf1Test() {
-// With Java 8 lambdas:
-// sqlContext.registerFunction(
-//   "stringLengthTest", (String str) -> str.length(), 
DataType.IntegerType);
-
-sqlContext.udf().register("stringLengthTest", new UDF1() {
-  @Override
-  public Integer call(String str) throws Exception {
-return str.length();
-  }
-}, DataTypes.IntegerType);
-
-Row result = sqlContext.sql("SELECT stringLengthTest('test')").head();
-assert(result.getInt(0) == 4);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void udf2Test() {
-// With Java 8 lambdas:
-// sqlContext.registerFunction(
-//   "stringLengthTest",
-//   (Str

spark git commit: [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 53a1ebf33 -> fba11c2f5


[SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.

Also added desc/asc function for constructing sorting expressions more 
conveniently. And added a small fix to lift alias out of cast expression.

Author: Reynold Xin 

Closes #4752 from rxin/SPARK-5985 and squashes the following commits:

aeda5ae [Reynold Xin] Added Experimental flag to ColumnName.
047ad03 [Reynold Xin] Lift alias out of cast.
c9cf17c [Reynold Xin] [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.


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

Branch: refs/heads/master
Commit: fba11c2f55dd81e4f6230e7edca3c7b2e01ccd9d
Parents: 53a1ebf
Author: Reynold Xin 
Authored: Tue Feb 24 18:59:23 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 18:59:23 2015 -0800

--
 python/pyspark/sql/dataframe.py | 11 ++--
 python/pyspark/sql/functions.py |  3 ++
 .../scala/org/apache/spark/sql/Column.scala | 13 +++--
 .../scala/org/apache/spark/sql/functions.scala  | 29 
 .../spark/sql/ColumnExpressionSuite.scala   |  4 +++
 .../org/apache/spark/sql/DataFrameSuite.scala   |  4 +++
 6 files changed, 59 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fba11c2f/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 010c38f..6f746d1 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -504,13 +504,18 @@ class DataFrame(object):
 return DataFrame(jdf, self.sql_ctx)
 
 def sort(self, *cols):
-""" Return a new :class:`DataFrame` sorted by the specified column.
+""" Return a new :class:`DataFrame` sorted by the specified column(s).
 
 :param cols: The columns or expressions used for sorting
 
 >>> df.sort(df.age.desc()).collect()
 [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
->>> df.sortBy(df.age.desc()).collect()
+>>> df.orderBy(df.age.desc()).collect()
+[Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
+>>> from pyspark.sql.functions import *
+>>> df.sort(asc("age")).collect()
+[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
+>>> df.orderBy(desc("age"), "name").collect()
 [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
 """
 if not cols:
@@ -520,7 +525,7 @@ class DataFrame(object):
 jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols))
 return DataFrame(jdf, self.sql_ctx)
 
-sortBy = sort
+orderBy = sort
 
 def head(self, n=None):
 """ Return the first `n` rows or the first row if n is None.

http://git-wip-us.apache.org/repos/asf/spark/blob/fba11c2f/python/pyspark/sql/functions.py
--
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index fc61162..8aa4476 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -48,6 +48,9 @@ _functions = {
 'lit': 'Creates a :class:`Column` of literal value.',
 'col': 'Returns a :class:`Column` based on the given column name.',
 'column': 'Returns a :class:`Column` based on the given column name.',
+'asc': 'Returns a sort expression based on the ascending order of the 
given column name.',
+'desc': 'Returns a sort expression based on the descending order of the 
given column name.',
+
 'upper': 'Converts a string expression to upper case.',
 'lower': 'Converts a string expression to upper case.',
 'sqrt': 'Computes the square root of the specified float value.',

http://git-wip-us.apache.org/repos/asf/spark/blob/fba11c2f/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 9807543..a2cc9a9 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
@@ -600,7 +600,11 @@ class Column(protected[sql] val expr: Expression) {
*
* @group expr_ops
*/
-  def cast(to: DataType): Column = Cast(expr, to)
+  def cast(to: DataType): Column = expr match {
+// Lift alias out of cast so we can support 
col.as("name").cast(IntegerType)
+case Alias(childExpr, name) => Alias(Cast(childExpr

spark git commit: [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 78a1781a9 -> 5e233b2c7


[SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.

Also added desc/asc function for constructing sorting expressions more 
conveniently. And added a small fix to lift alias out of cast expression.

Author: Reynold Xin 

Closes #4752 from rxin/SPARK-5985 and squashes the following commits:

aeda5ae [Reynold Xin] Added Experimental flag to ColumnName.
047ad03 [Reynold Xin] Lift alias out of cast.
c9cf17c [Reynold Xin] [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.

(cherry picked from commit fba11c2f55dd81e4f6230e7edca3c7b2e01ccd9d)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 5e233b2c7ec67ca87182230f9bb5fd8121f26419
Parents: 78a1781
Author: Reynold Xin 
Authored: Tue Feb 24 18:59:23 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 18:59:35 2015 -0800

--
 python/pyspark/sql/dataframe.py | 11 ++--
 python/pyspark/sql/functions.py |  3 ++
 .../scala/org/apache/spark/sql/Column.scala | 13 +++--
 .../scala/org/apache/spark/sql/functions.scala  | 29 
 .../spark/sql/ColumnExpressionSuite.scala   |  4 +++
 .../org/apache/spark/sql/DataFrameSuite.scala   |  4 +++
 6 files changed, 59 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5e233b2c/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 010c38f..6f746d1 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -504,13 +504,18 @@ class DataFrame(object):
 return DataFrame(jdf, self.sql_ctx)
 
 def sort(self, *cols):
-""" Return a new :class:`DataFrame` sorted by the specified column.
+""" Return a new :class:`DataFrame` sorted by the specified column(s).
 
 :param cols: The columns or expressions used for sorting
 
 >>> df.sort(df.age.desc()).collect()
 [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
->>> df.sortBy(df.age.desc()).collect()
+>>> df.orderBy(df.age.desc()).collect()
+[Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
+>>> from pyspark.sql.functions import *
+>>> df.sort(asc("age")).collect()
+[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
+>>> df.orderBy(desc("age"), "name").collect()
 [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
 """
 if not cols:
@@ -520,7 +525,7 @@ class DataFrame(object):
 jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols))
 return DataFrame(jdf, self.sql_ctx)
 
-sortBy = sort
+orderBy = sort
 
 def head(self, n=None):
 """ Return the first `n` rows or the first row if n is None.

http://git-wip-us.apache.org/repos/asf/spark/blob/5e233b2c/python/pyspark/sql/functions.py
--
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index fc61162..8aa4476 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -48,6 +48,9 @@ _functions = {
 'lit': 'Creates a :class:`Column` of literal value.',
 'col': 'Returns a :class:`Column` based on the given column name.',
 'column': 'Returns a :class:`Column` based on the given column name.',
+'asc': 'Returns a sort expression based on the ascending order of the 
given column name.',
+'desc': 'Returns a sort expression based on the descending order of the 
given column name.',
+
 'upper': 'Converts a string expression to upper case.',
 'lower': 'Converts a string expression to upper case.',
 'sqrt': 'Computes the square root of the specified float value.',

http://git-wip-us.apache.org/repos/asf/spark/blob/5e233b2c/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 9807543..a2cc9a9 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
@@ -600,7 +600,11 @@ class Column(protected[sql] val expr: Expression) {
*
* @group expr_ops
*/
-  def cast(to: DataType): Column = Cast(expr, to)
+  def cast(to: DataType): Column = expr match {
+// Lift alias out of ca

spark git commit: [SPARK-5286][SQL] SPARK-5286 followup

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 922b43b3c -> 769e092bd


[SPARK-5286][SQL] SPARK-5286 followup

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

Author: Yin Huai 

Closes #4755 from yhuai/SPARK-5286-throwable and squashes the following commits:

4c0c450 [Yin Huai] Catch Throwable instead of Exception.


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

Branch: refs/heads/master
Commit: 769e092bdc51582372093f76dbaece27149cc4ea
Parents: 922b43b
Author: Yin Huai 
Authored: Tue Feb 24 19:51:36 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 19:51:36 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/spark/blob/769e092b/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 c88d0e6..9934a5d 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
@@ -63,10 +63,10 @@ case class DropTable(
 } catch {
   // This table's metadata is not in
   case _: org.apache.hadoop.hive.ql.metadata.InvalidTableException =>
-  // Other exceptions can be caused by users providing wrong parameters in 
OPTIONS
+  // Other Throwables can be caused by users providing wrong parameters in 
OPTIONS
   // (e.g. invalid paths). We catch it and log a warning message.
-  // Users should be able to drop such kinds of tables regardless if there 
is an exception.
-  case e: Exception => log.warn(s"${e.getMessage}")
+  // Users should be able to drop such kinds of tables regardless if there 
is an error.
+  case e: Throwable => log.warn(s"${e.getMessage}")
 }
 hiveContext.invalidateTable(tableName)
 hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName")


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



spark git commit: [SPARK-5286][SQL] SPARK-5286 followup

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 1e9489422 -> e7a748ecf


[SPARK-5286][SQL] SPARK-5286 followup

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

Author: Yin Huai 

Closes #4755 from yhuai/SPARK-5286-throwable and squashes the following commits:

4c0c450 [Yin Huai] Catch Throwable instead of Exception.

(cherry picked from commit 769e092bdc51582372093f76dbaece27149cc4ea)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: e7a748ecfdb22bd7b64d95bbb19347c36ca9ceb2
Parents: 1e94894
Author: Yin Huai 
Authored: Tue Feb 24 19:51:36 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 19:51:48 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/spark/blob/e7a748ec/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 c88d0e6..9934a5d 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
@@ -63,10 +63,10 @@ case class DropTable(
 } catch {
   // This table's metadata is not in
   case _: org.apache.hadoop.hive.ql.metadata.InvalidTableException =>
-  // Other exceptions can be caused by users providing wrong parameters in 
OPTIONS
+  // Other Throwables can be caused by users providing wrong parameters in 
OPTIONS
   // (e.g. invalid paths). We catch it and log a warning message.
-  // Users should be able to drop such kinds of tables regardless if there 
is an exception.
-  case e: Exception => log.warn(s"${e.getMessage}")
+  // Users should be able to drop such kinds of tables regardless if there 
is an error.
+  case e: Throwable => log.warn(s"${e.getMessage}")
 }
 hiveContext.invalidateTable(tableName)
 hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName")


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



spark git commit: [SPARK-5994] [SQL] Python DataFrame documentation fixes

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/master 769e092bd -> d641fbb39


[SPARK-5994] [SQL] Python DataFrame documentation fixes

select empty should NOT be the same as select. make sure selectExpr is behaving 
the same.
join param documentation
link to source doesn't work in jekyll generated file
cross reference of columns (i.e. enabling linking)
show(): move df example before df.show()
move tests in SQLContext out of docstring otherwise doc is too long
Column.desc and .asc doesn't have any documentation
in documentation, sort functions.*)

Author: Davies Liu 

Closes #4756 from davies/df_docs and squashes the following commits:

f30502c [Davies Liu] fix doc
32f0d46 [Davies Liu] fix DataFrame docs


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

Branch: refs/heads/master
Commit: d641fbb39c90b1d734cc55396ca43d7e98788975
Parents: 769e092
Author: Davies Liu 
Authored: Tue Feb 24 20:51:55 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 20:51:55 2015 -0800

--
 docs/_config.yml|   1 +
 python/docs/pyspark.sql.rst |   3 -
 python/pyspark/sql/context.py   | 182 +++
 python/pyspark/sql/dataframe.py |  56 +--
 python/pyspark/sql/functions.py |   1 +
 python/pyspark/sql/tests.py |  68 -
 python/pyspark/sql/types.py |   2 +-
 7 files changed, 130 insertions(+), 183 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d641fbb3/docs/_config.yml
--
diff --git a/docs/_config.yml b/docs/_config.yml
index e2db274..0652927 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -10,6 +10,7 @@ kramdown:
 
 include:
   - _static
+  - _modules
 
 # These allow the documentation to be updated with newer releases
 # of Spark, Scala, and Mesos.

http://git-wip-us.apache.org/repos/asf/spark/blob/d641fbb3/python/docs/pyspark.sql.rst
--
diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst
index e03379e..2e3f69b 100644
--- a/python/docs/pyspark.sql.rst
+++ b/python/docs/pyspark.sql.rst
@@ -7,7 +7,6 @@ Module Context
 .. automodule:: pyspark.sql
 :members:
 :undoc-members:
-:show-inheritance:
 
 
 pyspark.sql.types module
@@ -15,7 +14,6 @@ pyspark.sql.types module
 .. automodule:: pyspark.sql.types
 :members:
 :undoc-members:
-:show-inheritance:
 
 
 pyspark.sql.functions module
@@ -23,4 +21,3 @@ pyspark.sql.functions module
 .. automodule:: pyspark.sql.functions
 :members:
 :undoc-members:
-:show-inheritance:

http://git-wip-us.apache.org/repos/asf/spark/blob/d641fbb3/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 125933c..5d7aeb6 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -129,6 +129,7 @@ class SQLContext(object):
 >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x))
 >>> sqlCtx.sql("SELECT stringLengthString('test')").collect()
 [Row(c0=u'4')]
+
 >>> from pyspark.sql.types import IntegerType
 >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), 
IntegerType())
 >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect()
@@ -197,31 +198,6 @@ class SQLContext(object):
 >>> df = sqlCtx.inferSchema(rdd)
 >>> df.collect()[0]
 Row(field1=1, field2=u'row1')
-
->>> NestedRow = Row("f1", "f2")
->>> nestedRdd1 = sc.parallelize([
-... NestedRow(array('i', [1, 2]), {"row1": 1.0}),
-... NestedRow(array('i', [2, 3]), {"row2": 2.0})])
->>> df = sqlCtx.inferSchema(nestedRdd1)
->>> df.collect()
-[Row(f1=[1, 2], f2={u'row1': 1.0}), ..., f2={u'row2': 2.0})]
-
->>> nestedRdd2 = sc.parallelize([
-... NestedRow([[1, 2], [2, 3]], [1, 2]),
-... NestedRow([[2, 3], [3, 4]], [2, 3])])
->>> df = sqlCtx.inferSchema(nestedRdd2)
->>> df.collect()
-[Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])]
-
->>> from collections import namedtuple
->>> CustomRow = namedtuple('CustomRow', 'field1 field2')
->>> rdd = sc.parallelize(
-... [CustomRow(field1=1, field2="row1"),
-...  CustomRow(field1=2, field2="row2"),
-...  CustomRow(field1=3, field2="row3")])
->>> df = sqlCtx.inferSchema(rdd)
->>> df.collect()[0]
-Row(field1=1, field2=u'row1')
 """
 
  

spark git commit: [SPARK-5994] [SQL] Python DataFrame documentation fixes

2015-02-24 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 e7a748ecf -> 5c421e030


[SPARK-5994] [SQL] Python DataFrame documentation fixes

select empty should NOT be the same as select. make sure selectExpr is behaving 
the same.
join param documentation
link to source doesn't work in jekyll generated file
cross reference of columns (i.e. enabling linking)
show(): move df example before df.show()
move tests in SQLContext out of docstring otherwise doc is too long
Column.desc and .asc doesn't have any documentation
in documentation, sort functions.*)

Author: Davies Liu 

Closes #4756 from davies/df_docs and squashes the following commits:

f30502c [Davies Liu] fix doc
32f0d46 [Davies Liu] fix DataFrame docs

(cherry picked from commit d641fbb39c90b1d734cc55396ca43d7e98788975)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 5c421e030b656b143cf962466157832723d8d1f5
Parents: e7a748e
Author: Davies Liu 
Authored: Tue Feb 24 20:51:55 2015 -0800
Committer: Michael Armbrust 
Committed: Tue Feb 24 20:52:05 2015 -0800

--
 docs/_config.yml|   1 +
 python/docs/pyspark.sql.rst |   3 -
 python/pyspark/sql/context.py   | 182 +++
 python/pyspark/sql/dataframe.py |  56 +--
 python/pyspark/sql/functions.py |   1 +
 python/pyspark/sql/tests.py |  68 -
 python/pyspark/sql/types.py |   2 +-
 7 files changed, 130 insertions(+), 183 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5c421e03/docs/_config.yml
--
diff --git a/docs/_config.yml b/docs/_config.yml
index 50124c5..ef6da03 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -10,6 +10,7 @@ kramdown:
 
 include:
   - _static
+  - _modules
 
 # These allow the documentation to be updated with newer releases
 # of Spark, Scala, and Mesos.

http://git-wip-us.apache.org/repos/asf/spark/blob/5c421e03/python/docs/pyspark.sql.rst
--
diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst
index e03379e..2e3f69b 100644
--- a/python/docs/pyspark.sql.rst
+++ b/python/docs/pyspark.sql.rst
@@ -7,7 +7,6 @@ Module Context
 .. automodule:: pyspark.sql
 :members:
 :undoc-members:
-:show-inheritance:
 
 
 pyspark.sql.types module
@@ -15,7 +14,6 @@ pyspark.sql.types module
 .. automodule:: pyspark.sql.types
 :members:
 :undoc-members:
-:show-inheritance:
 
 
 pyspark.sql.functions module
@@ -23,4 +21,3 @@ pyspark.sql.functions module
 .. automodule:: pyspark.sql.functions
 :members:
 :undoc-members:
-:show-inheritance:

http://git-wip-us.apache.org/repos/asf/spark/blob/5c421e03/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 125933c..5d7aeb6 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -129,6 +129,7 @@ class SQLContext(object):
 >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x))
 >>> sqlCtx.sql("SELECT stringLengthString('test')").collect()
 [Row(c0=u'4')]
+
 >>> from pyspark.sql.types import IntegerType
 >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), 
IntegerType())
 >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect()
@@ -197,31 +198,6 @@ class SQLContext(object):
 >>> df = sqlCtx.inferSchema(rdd)
 >>> df.collect()[0]
 Row(field1=1, field2=u'row1')
-
->>> NestedRow = Row("f1", "f2")
->>> nestedRdd1 = sc.parallelize([
-... NestedRow(array('i', [1, 2]), {"row1": 1.0}),
-... NestedRow(array('i', [2, 3]), {"row2": 2.0})])
->>> df = sqlCtx.inferSchema(nestedRdd1)
->>> df.collect()
-[Row(f1=[1, 2], f2={u'row1': 1.0}), ..., f2={u'row2': 2.0})]
-
->>> nestedRdd2 = sc.parallelize([
-... NestedRow([[1, 2], [2, 3]], [1, 2]),
-... NestedRow([[2, 3], [3, 4]], [2, 3])])
->>> df = sqlCtx.inferSchema(nestedRdd2)
->>> df.collect()
-[Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])]
-
->>> from collections import namedtuple
->>> CustomRow = namedtuple('CustomRow', 'field1 field2')
->>> rdd = sc.parallelize(
-... [CustomRow(field1=1, field2="row1"),
-...  CustomRow(field1=2, field2="row2"),
-...  CustomRow(field1=3, field2="row3")])
->>> df 

spark git commit: [SPARK-5944] [PySpark] fix version in Python API docs

2015-02-25 Thread marmbrus
Repository: spark
Updated Branches:
  refs/heads/branch-1.3 791df93cd -> 9aca3c688


[SPARK-5944] [PySpark] fix version in Python API docs

use RELEASE_VERSION when building the Python API docs

Author: Davies Liu 

Closes #4731 from davies/api_version and squashes the following commits:

c9744c9 [Davies Liu] Update create-release.sh
08cbc3f [Davies Liu] fix python docs

(cherry picked from commit f3f4c87b3d944c10d1200dfe49091ebb2a149be6)
Signed-off-by: Michael Armbrust 


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

Branch: refs/heads/branch-1.3
Commit: 9aca3c688ad9a51bc8e14053ed4daac168028913
Parents: 791df93
Author: Davies Liu 
Authored: Wed Feb 25 15:13:34 2015 -0800
Committer: Michael Armbrust 
Committed: Wed Feb 25 15:13:56 2015 -0800

--
 dev/create-release/create-release.sh | 2 +-
 python/docs/conf.py  | 6 +++---
 python/docs/pyspark.sql.rst  | 2 +-
 python/pyspark/rdd.py| 4 
 4 files changed, 9 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9aca3c68/dev/create-release/create-release.sh
--
diff --git a/dev/create-release/create-release.sh 
b/dev/create-release/create-release.sh
index 607ce1c..da15ce3 100755
--- a/dev/create-release/create-release.sh
+++ b/dev/create-release/create-release.sh
@@ -237,7 +237,7 @@ if [[ ! "$@" =~ --skip-package ]]; then
   sbt/sbt clean
   cd docs
   # Compile docs with Java 7 to use nicer format
-  JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build
+  JAVA_HOME="$JAVA_7_HOME" PRODUCTION=1 RELEASE_VERSION="$RELEASE_VERSION" 
jekyll build
   echo "Copying release documentation"
   rc_docs_folder=${rc_folder}-docs
   ssh $asf_usern...@people.apache.org \

http://git-wip-us.apache.org/repos/asf/spark/blob/9aca3c68/python/docs/conf.py
--
diff --git a/python/docs/conf.py b/python/docs/conf.py
index cbbf7ff..163987d 100644
--- a/python/docs/conf.py
+++ b/python/docs/conf.py
@@ -48,16 +48,16 @@ master_doc = 'index'
 
 # General information about the project.
 project = u'PySpark'
-copyright = u'2014, Author'
+copyright = u''
 
 # The version info for the project you're documenting, acts as replacement for
 # |version| and |release|, also used in various other places throughout the
 # built documents.
 #
 # The short X.Y version.
-version = '1.3-SNAPSHOT'
+version = 'master'
 # The full version, including alpha/beta/rc tags.
-release = '1.3-SNAPSHOT'
+release = os.environ.get('RELEASE_VERSION', version)
 
 # The language for content autogenerated by Sphinx. Refer to documentation
 # for a list of supported languages.

http://git-wip-us.apache.org/repos/asf/spark/blob/9aca3c68/python/docs/pyspark.sql.rst
--
diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst
index 2e3f69b..6259379 100644
--- a/python/docs/pyspark.sql.rst
+++ b/python/docs/pyspark.sql.rst
@@ -17,7 +17,7 @@ pyspark.sql.types module
 
 
 pyspark.sql.functions module
-
+
 .. automodule:: pyspark.sql.functions
 :members:
 :undoc-members:

http://git-wip-us.apache.org/repos/asf/spark/blob/9aca3c68/python/pyspark/rdd.py
--
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index d3148de..cb12fed 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -2111,6 +2111,7 @@ class RDD(object):
 def countApprox(self, timeout, confidence=0.95):
 """
 .. note:: Experimental
+
 Approximate version of count() that returns a potentially incomplete
 result within a timeout, even if not all tasks have finished.
 
@@ -2124,6 +2125,7 @@ class RDD(object):
 def sumApprox(self, timeout, confidence=0.95):
 """
 .. note:: Experimental
+
 Approximate operation to return the sum within a timeout
 or meet the confidence.
 
@@ -2140,6 +2142,7 @@ class RDD(object):
 def meanApprox(self, timeout, confidence=0.95):
 """
 .. note:: Experimental
+
 Approximate operation to return the mean within a timeout
 or meet the confidence.
 
@@ -2156,6 +2159,7 @@ class RDD(object):
 def countApproxDistinct(self, relativeSD=0.05):
 """
 .. note:: Experimental
+
 Return approximate number of distinct elements in the RDD.
 
 The algorithm used is based on streamlib's implementation of


---

<    3   4   5   6   7   8   9   10   11   12   >