spark git commit: [SPARK-15792][SQL] Allows operator to change the verbosity in explain output

2016-06-06 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 0e0904a2f -> 5f731d685


[SPARK-15792][SQL] Allows operator to change the verbosity in explain output

## What changes were proposed in this pull request?

This PR allows customization of verbosity in explain output. After change, 
`dataframe.explain()` and `dataframe.explain(true)` has different verbosity 
output for physical plan.

Currently, this PR only enables verbosity string for operator 
`HashAggregateExec` and `SortAggregateExec`. We will gradually enable verbosity 
string for more operators in future.

**Less verbose mode:** dataframe.explain(extended = false)

`output=[count(a)#85L]` is **NOT** displayed for HashAggregate.

```
scala> Seq((1,2,3)).toDF("a", "b", "c").createTempView("df2")
scala> spark.sql("select count(a) from df2").explain()
== Physical Plan ==
*HashAggregate(key=[], functions=[count(1)])
+- Exchange SinglePartition
   +- *HashAggregate(key=[], functions=[partial_count(1)])
  +- LocalTableScan
```

**Verbose mode:** dataframe.explain(extended = true)

`output=[count(a)#85L]` is displayed for HashAggregate.

```
scala> spark.sql("select count(a) from df2").explain(true)  // 
"output=[count(a)#85L]" is added
...
== Physical Plan ==
*HashAggregate(key=[], functions=[count(1)], output=[count(a)#85L])
+- Exchange SinglePartition
   +- *HashAggregate(key=[], functions=[partial_count(1)], output=[count#87L])
  +- LocalTableScan
```

## How was this patch tested?

Manual test.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #13535 from clockfly/verbose_breakdown_2.


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

Branch: refs/heads/master
Commit: 5f731d6859c4516941e5f90c99c966ef76268864
Parents: 0e0904a
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Mon Jun 6 22:59:25 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon Jun 6 22:59:25 2016 -0700

--
 .../sql/catalyst/expressions/Expression.scala   |  4 
 .../spark/sql/catalyst/plans/QueryPlan.scala|  2 ++
 .../spark/sql/catalyst/trees/TreeNode.scala | 23 +++-
 .../spark/sql/execution/QueryExecution.scala| 14 +++-
 .../sql/execution/WholeStageCodegenExec.scala   |  6 +++--
 .../execution/aggregate/HashAggregateExec.scala | 12 --
 .../execution/aggregate/SortAggregateExec.scala | 12 --
 7 files changed, 55 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5f731d68/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 2ec4621..efe592d 100644
--- 
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
@@ -190,6 +190,10 @@ abstract class Expression extends TreeNode[Expression] {
 case single => single :: Nil
   }
 
+  // Marks this as final, Expression.verboseString should never be called, and 
thus shouldn't be
+  // overridden by concrete classes.
+  final override def verboseString: String = simpleString
+
   override def simpleString: String = toString
 
   override def toString: String = prettyName + flatArguments.mkString("(", ", 
", ")")

http://git-wip-us.apache.org/repos/asf/spark/blob/5f731d68/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index 19a66cf..cf34f4b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -257,6 +257,8 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] 
extends TreeNode[PlanT
 
   override def simpleString: String = statePrefix + super.simpleString
 
+  override def verboseString: String = simpleString
+
   /**
* All the subqueries of current plan.
*/

http://git-wip-us.apache.org/repos/asf/spark/blob/5f731d68/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
--

spark git commit: [SPARK-15632][SQL] Typed Filter should NOT change the Dataset schema

2016-06-06 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 62765cbeb -> a5bec5b81


[SPARK-15632][SQL] Typed Filter should NOT change the Dataset schema

## What changes were proposed in this pull request?

This PR makes sure the typed Filter doesn't change the Dataset schema.

**Before the change:**

```
scala> val df = spark.range(0,9)
scala> df.schema
res12: org.apache.spark.sql.types.StructType = 
StructType(StructField(id,LongType,false))
scala> val afterFilter = df.filter(_=>true)
scala> afterFilter.schema   // !!! schema is CHANGED!!! Column name is changed 
from id to value, nullable is changed from false to true.
res13: org.apache.spark.sql.types.StructType = 
StructType(StructField(value,LongType,true))

```

SerializeFromObject and DeserializeToObject are inserted to wrap the Filter, 
and these two can possibly change the schema of Dataset.

**After the change:**

```
scala> afterFilter.schema   // schema is NOT changed.
res47: org.apache.spark.sql.types.StructType = 
StructType(StructField(id,LongType,false))
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #13529 from clockfly/spark-15632.

(cherry picked from commit 0e0904a2fce3c4447c24f1752307b6d01ffbd0ad)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: a5bec5b81d9e8ce17f1ce509731b030f0f3538e3
Parents: 62765cb
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Mon Jun 6 22:40:21 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon Jun 6 22:40:29 2016 -0700

--
 .../optimizer/TypedFilterOptimizationSuite.scala|  4 +++-
 .../main/scala/org/apache/spark/sql/Dataset.scala   | 16 
 .../test/org/apache/spark/sql/JavaDatasetSuite.java | 13 +
 .../scala/org/apache/spark/sql/DatasetSuite.scala   |  6 ++
 .../sql/execution/WholeStageCodegenSuite.scala  |  2 +-
 5 files changed, 31 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a5bec5b8/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
index 289c16a..63d87bf 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
@@ -57,7 +57,9 @@ class TypedFilterOptimizationSuite extends PlanTest {
 comparePlans(optimized, expected)
   }
 
-  test("embed deserializer in filter condition if there is only one filter") {
+  // TODO: Remove this after we completely fix SPARK-15632 by adding 
optimization rules
+  // for typed filters.
+  ignore("embed deserializer in typed filter condition if there is only one 
filter") {
 val input = LocalRelation('_1.int, '_2.int)
 val f = (i: (Int, Int)) => i._1 > 0
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a5bec5b8/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 96c871d..6cbc27d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -1944,11 +1944,11 @@ class Dataset[T] private[sql](
*/
   @Experimental
   def filter(func: T => Boolean): Dataset[T] = {
-val deserialized = CatalystSerde.deserialize[T](logicalPlan)
+val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer)
 val function = Literal.create(func, ObjectType(classOf[T => Boolean]))
-val condition = Invoke(function, "apply", BooleanType, deserialized.output)
-val filter = Filter(condition, deserialized)
-withTypedPlan(CatalystSerde.serialize[T](filter))
+val condition = Invoke(function, "apply", BooleanType, deserializer :: Nil)
+val filter = Filter(condition, logicalPlan)
+withTypedPlan(filter)
   }
 
   /**
@@ -1961,11 +1961,11 @@ class Dataset[T] private[sql](
*/
   @Experimental
   def filter(func: FilterFunction[T]): Dataset[T] = {
-val deserialized = Ca

spark git commit: [SPARK-15632][SQL] Typed Filter should NOT change the Dataset schema

2016-06-06 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master c409e23ab -> 0e0904a2f


[SPARK-15632][SQL] Typed Filter should NOT change the Dataset schema

## What changes were proposed in this pull request?

This PR makes sure the typed Filter doesn't change the Dataset schema.

**Before the change:**

```
scala> val df = spark.range(0,9)
scala> df.schema
res12: org.apache.spark.sql.types.StructType = 
StructType(StructField(id,LongType,false))
scala> val afterFilter = df.filter(_=>true)
scala> afterFilter.schema   // !!! schema is CHANGED!!! Column name is changed 
from id to value, nullable is changed from false to true.
res13: org.apache.spark.sql.types.StructType = 
StructType(StructField(value,LongType,true))

```

SerializeFromObject and DeserializeToObject are inserted to wrap the Filter, 
and these two can possibly change the schema of Dataset.

**After the change:**

```
scala> afterFilter.schema   // schema is NOT changed.
res47: org.apache.spark.sql.types.StructType = 
StructType(StructField(id,LongType,false))
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #13529 from clockfly/spark-15632.


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

Branch: refs/heads/master
Commit: 0e0904a2fce3c4447c24f1752307b6d01ffbd0ad
Parents: c409e23
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Mon Jun 6 22:40:21 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon Jun 6 22:40:21 2016 -0700

--
 .../optimizer/TypedFilterOptimizationSuite.scala|  4 +++-
 .../main/scala/org/apache/spark/sql/Dataset.scala   | 16 
 .../test/org/apache/spark/sql/JavaDatasetSuite.java | 13 +
 .../scala/org/apache/spark/sql/DatasetSuite.scala   |  6 ++
 .../sql/execution/WholeStageCodegenSuite.scala  |  2 +-
 5 files changed, 31 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0e0904a2/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
index 289c16a..63d87bf 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala
@@ -57,7 +57,9 @@ class TypedFilterOptimizationSuite extends PlanTest {
 comparePlans(optimized, expected)
   }
 
-  test("embed deserializer in filter condition if there is only one filter") {
+  // TODO: Remove this after we completely fix SPARK-15632 by adding 
optimization rules
+  // for typed filters.
+  ignore("embed deserializer in typed filter condition if there is only one 
filter") {
 val input = LocalRelation('_1.int, '_2.int)
 val f = (i: (Int, Int)) => i._1 > 0
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0e0904a2/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 96c871d..6cbc27d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -1944,11 +1944,11 @@ class Dataset[T] private[sql](
*/
   @Experimental
   def filter(func: T => Boolean): Dataset[T] = {
-val deserialized = CatalystSerde.deserialize[T](logicalPlan)
+val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer)
 val function = Literal.create(func, ObjectType(classOf[T => Boolean]))
-val condition = Invoke(function, "apply", BooleanType, deserialized.output)
-val filter = Filter(condition, deserialized)
-withTypedPlan(CatalystSerde.serialize[T](filter))
+val condition = Invoke(function, "apply", BooleanType, deserializer :: Nil)
+val filter = Filter(condition, logicalPlan)
+withTypedPlan(filter)
   }
 
   /**
@@ -1961,11 +1961,11 @@ class Dataset[T] private[sql](
*/
   @Experimental
   def filter(func: FilterFunction[T]): Dataset[T] = {
-val deserialized = CatalystSerde.deserialize[T](logicalPlan)
+val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer)
 val function

spark git commit: [SPARK-15657][SQL] RowEncoder should validate the data type of input object

2016-06-05 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 e6e1d8232 -> 38a626a54


[SPARK-15657][SQL] RowEncoder should validate the data type of input object

## What changes were proposed in this pull request?

This PR improves the error handling of `RowEncoder`. When we create a 
`RowEncoder` with a given schema, we should validate the data type of input 
object. e.g. we should throw an exception when a field is boolean but is 
declared as a string column.

This PR also removes the support to use `Product` as a valid external type of 
struct type.  This support is added at 
https://github.com/apache/spark/pull/9712, but is incomplete, e.g. nested 
product, product in array are both not working.  However, we never officially 
support this feature and I think it's ok to ban it.

## How was this patch tested?

new tests in `RowEncoderSuite`.

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13401 from cloud-fan/bug.

(cherry picked from commit 30c4774f33fed63b7d400d220d710fb432f599a8)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 38a626a54dd0fac0ca460e1ba534048de513bc29
Parents: e6e1d82
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Sun Jun 5 15:59:52 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Sun Jun 5 16:00:00 2016 -0700

--
 .../main/scala/org/apache/spark/sql/Row.scala   | 10 +---
 .../sql/catalyst/encoders/RowEncoder.scala  | 17 --
 .../catalyst/expressions/objects/objects.scala  | 61 +---
 .../sql/catalyst/encoders/RowEncoderSuite.scala | 47 ++-
 4 files changed, 95 insertions(+), 40 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/38a626a5/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
--
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
index a257b83..391001d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
@@ -304,15 +304,7 @@ trait Row extends Serializable {
*
* @throws ClassCastException when data type does not match.
*/
-  def getStruct(i: Int): Row = {
-// Product and Row both are recognized as StructType in a Row
-val t = get(i)
-if (t.isInstanceOf[Product]) {
-  Row.fromTuple(t.asInstanceOf[Product])
-} else {
-  t.asInstanceOf[Row]
-}
-  }
+  def getStruct(i: Int): Row = getAs[Row](i)
 
   /**
* Returns the value at position i.

http://git-wip-us.apache.org/repos/asf/spark/blob/38a626a5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
index 6cd7b34..67fca15 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
@@ -51,7 +51,7 @@ import org.apache.spark.unsafe.types.UTF8String
  *   BinaryType -> byte array
  *   ArrayType -> scala.collection.Seq or Array
  *   MapType -> scala.collection.Map
- *   StructType -> org.apache.spark.sql.Row or Product
+ *   StructType -> org.apache.spark.sql.Row
  * }}}
  */
 object RowEncoder {
@@ -121,11 +121,15 @@ object RowEncoder {
 
 case t @ ArrayType(et, _) => et match {
   case BooleanType | ByteType | ShortType | IntegerType | LongType | 
FloatType | DoubleType =>
+// TODO: validate input type for primitive array.
 NewInstance(
   classOf[GenericArrayData],
   inputObject :: Nil,
   dataType = t)
-  case _ => MapObjects(serializerFor(_, et), inputObject, 
externalDataTypeForInput(et))
+  case _ => MapObjects(
+element => serializerFor(ValidateExternalType(element, et), et),
+inputObject,
+ObjectType(classOf[Object]))
 }
 
 case t @ MapType(kt, vt, valueNullable) =>
@@ -151,8 +155,9 @@ object RowEncoder {
 case StructType(fields) =>
   val nonNullOutput = CreateNamedStruct(fields.zipWithIndex.flatMap { case 
(field, index) =>
 val fieldValue = serializerFor(
-  GetExternalRowField(
-inputObject, index, field.name, 
external

spark git commit: [SPARK-15657][SQL] RowEncoder should validate the data type of input object

2016-06-05 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 8a9110510 -> 30c4774f3


[SPARK-15657][SQL] RowEncoder should validate the data type of input object

## What changes were proposed in this pull request?

This PR improves the error handling of `RowEncoder`. When we create a 
`RowEncoder` with a given schema, we should validate the data type of input 
object. e.g. we should throw an exception when a field is boolean but is 
declared as a string column.

This PR also removes the support to use `Product` as a valid external type of 
struct type.  This support is added at 
https://github.com/apache/spark/pull/9712, but is incomplete, e.g. nested 
product, product in array are both not working.  However, we never officially 
support this feature and I think it's ok to ban it.

## How was this patch tested?

new tests in `RowEncoderSuite`.

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13401 from cloud-fan/bug.


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

Branch: refs/heads/master
Commit: 30c4774f33fed63b7d400d220d710fb432f599a8
Parents: 8a91105
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Sun Jun 5 15:59:52 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Sun Jun 5 15:59:52 2016 -0700

--
 .../main/scala/org/apache/spark/sql/Row.scala   | 10 +---
 .../sql/catalyst/encoders/RowEncoder.scala  | 17 --
 .../catalyst/expressions/objects/objects.scala  | 61 +---
 .../sql/catalyst/encoders/RowEncoderSuite.scala | 47 ++-
 4 files changed, 95 insertions(+), 40 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/30c4774f/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
--
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
index a257b83..391001d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
@@ -304,15 +304,7 @@ trait Row extends Serializable {
*
* @throws ClassCastException when data type does not match.
*/
-  def getStruct(i: Int): Row = {
-// Product and Row both are recognized as StructType in a Row
-val t = get(i)
-if (t.isInstanceOf[Product]) {
-  Row.fromTuple(t.asInstanceOf[Product])
-} else {
-  t.asInstanceOf[Row]
-}
-  }
+  def getStruct(i: Int): Row = getAs[Row](i)
 
   /**
* Returns the value at position i.

http://git-wip-us.apache.org/repos/asf/spark/blob/30c4774f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
index 6cd7b34..67fca15 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
@@ -51,7 +51,7 @@ import org.apache.spark.unsafe.types.UTF8String
  *   BinaryType -> byte array
  *   ArrayType -> scala.collection.Seq or Array
  *   MapType -> scala.collection.Map
- *   StructType -> org.apache.spark.sql.Row or Product
+ *   StructType -> org.apache.spark.sql.Row
  * }}}
  */
 object RowEncoder {
@@ -121,11 +121,15 @@ object RowEncoder {
 
 case t @ ArrayType(et, _) => et match {
   case BooleanType | ByteType | ShortType | IntegerType | LongType | 
FloatType | DoubleType =>
+// TODO: validate input type for primitive array.
 NewInstance(
   classOf[GenericArrayData],
   inputObject :: Nil,
   dataType = t)
-  case _ => MapObjects(serializerFor(_, et), inputObject, 
externalDataTypeForInput(et))
+  case _ => MapObjects(
+element => serializerFor(ValidateExternalType(element, et), et),
+inputObject,
+ObjectType(classOf[Object]))
 }
 
 case t @ MapType(kt, vt, valueNullable) =>
@@ -151,8 +155,9 @@ object RowEncoder {
 case StructType(fields) =>
   val nonNullOutput = CreateNamedStruct(fields.zipWithIndex.flatMap { case 
(field, index) =>
 val fieldValue = serializerFor(
-  GetExternalRowField(
-inputObject, index, field.name, 
externalDataTypeForInput(field.dataType)),
+  ValidateExternalType(
+GetExternalRowField(inputObject, index, field.name),
+

spark git commit: [SPARK-15140][SQL] make the semantics of null input object for encoder clear

2016-06-03 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 52376e067 -> 7315acf89


[SPARK-15140][SQL] make the semantics of null input object for encoder clear

## What changes were proposed in this pull request?

For input object of non-flat type, we can't encode it to row if it's null, as 
Spark SQL doesn't allow row to be null, only its columns can be null.

This PR explicitly add this constraint and throw exception if users break it.

## How was this patch tested?

several new tests

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13469 from cloud-fan/null-object.

(cherry picked from commit 11c83f83d5172167cb64513d5311b4178797d40e)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 7315acf896b2474a4b7513434f5ba2faf468abd9
Parents: 52376e0
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri Jun 3 14:28:19 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Jun 3 14:28:26 2016 -0700

--
 .../sql/catalyst/encoders/ExpressionEncoder.scala  | 13 ++---
 .../spark/sql/catalyst/encoders/RowEncoder.scala   |  7 +++
 .../sql/catalyst/expressions/objects/objects.scala |  4 ++--
 .../spark/sql/catalyst/encoders/RowEncoderSuite.scala  |  8 
 .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 10 ++
 5 files changed, 33 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7315acf8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
index cc59d06..688082d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, 
JavaTypeInference, ScalaRefle
 import org.apache.spark.sql.catalyst.analysis.{Analyzer, GetColumnByOrdinal, 
SimpleAnalyzer, UnresolvedAttribute, UnresolvedExtractValue}
 import org.apache.spark.sql.catalyst.expressions._
 import 
org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, 
GenerateUnsafeProjection}
-import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, NewInstance}
+import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, 
Invoke, NewInstance}
 import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts
 import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, 
DeserializeToObject, LocalRelation}
 import org.apache.spark.sql.types.{ObjectType, StructField, StructType}
@@ -50,8 +50,15 @@ object ExpressionEncoder {
 val cls = mirror.runtimeClass(tpe)
 val flat = !ScalaReflection.definedByConstructorParams(tpe)
 
-val inputObject = BoundReference(0, ScalaReflection.dataTypeFor[T], 
nullable = false)
-val serializer = ScalaReflection.serializerFor[T](inputObject)
+val inputObject = BoundReference(0, ScalaReflection.dataTypeFor[T], 
nullable = true)
+val nullSafeInput = if (flat) {
+  inputObject
+} else {
+  // For input object of non-flat type, we can't encode it to row if it's 
null, as Spark SQL
+  // doesn't allow top-level row to be null, only its columns can be null.
+  AssertNotNull(inputObject, Seq("top level non-flat input object"))
+}
+val serializer = ScalaReflection.serializerFor[T](nullSafeInput)
 val deserializer = ScalaReflection.deserializerFor[T]
 
 val schema = ScalaReflection.schemaFor[T] match {

http://git-wip-us.apache.org/repos/asf/spark/blob/7315acf8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
index 3c6ae1c..6cd7b34 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
@@ -57,8 +57,8 @@ import org.apache.spark.unsafe.types.UTF8String
 object RowEncoder {
   def apply(schema: StructType): ExpressionEncoder[Row] = {
 val cls = classOf[Row]
- 

spark git commit: [SPARK-15140][SQL] make the semantics of null input object for encoder clear

2016-06-03 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 28ad0f7b0 -> 11c83f83d


[SPARK-15140][SQL] make the semantics of null input object for encoder clear

## What changes were proposed in this pull request?

For input object of non-flat type, we can't encode it to row if it's null, as 
Spark SQL doesn't allow row to be null, only its columns can be null.

This PR explicitly add this constraint and throw exception if users break it.

## How was this patch tested?

several new tests

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13469 from cloud-fan/null-object.


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

Branch: refs/heads/master
Commit: 11c83f83d5172167cb64513d5311b4178797d40e
Parents: 28ad0f7
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri Jun 3 14:28:19 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Jun 3 14:28:19 2016 -0700

--
 .../sql/catalyst/encoders/ExpressionEncoder.scala  | 13 ++---
 .../spark/sql/catalyst/encoders/RowEncoder.scala   |  7 +++
 .../sql/catalyst/expressions/objects/objects.scala |  4 ++--
 .../spark/sql/catalyst/encoders/RowEncoderSuite.scala  |  8 
 .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 10 ++
 5 files changed, 33 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/11c83f83/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
index cc59d06..688082d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, 
JavaTypeInference, ScalaRefle
 import org.apache.spark.sql.catalyst.analysis.{Analyzer, GetColumnByOrdinal, 
SimpleAnalyzer, UnresolvedAttribute, UnresolvedExtractValue}
 import org.apache.spark.sql.catalyst.expressions._
 import 
org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, 
GenerateUnsafeProjection}
-import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, NewInstance}
+import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, 
Invoke, NewInstance}
 import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts
 import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, 
DeserializeToObject, LocalRelation}
 import org.apache.spark.sql.types.{ObjectType, StructField, StructType}
@@ -50,8 +50,15 @@ object ExpressionEncoder {
 val cls = mirror.runtimeClass(tpe)
 val flat = !ScalaReflection.definedByConstructorParams(tpe)
 
-val inputObject = BoundReference(0, ScalaReflection.dataTypeFor[T], 
nullable = false)
-val serializer = ScalaReflection.serializerFor[T](inputObject)
+val inputObject = BoundReference(0, ScalaReflection.dataTypeFor[T], 
nullable = true)
+val nullSafeInput = if (flat) {
+  inputObject
+} else {
+  // For input object of non-flat type, we can't encode it to row if it's 
null, as Spark SQL
+  // doesn't allow top-level row to be null, only its columns can be null.
+  AssertNotNull(inputObject, Seq("top level non-flat input object"))
+}
+val serializer = ScalaReflection.serializerFor[T](nullSafeInput)
 val deserializer = ScalaReflection.deserializerFor[T]
 
 val schema = ScalaReflection.schemaFor[T] match {

http://git-wip-us.apache.org/repos/asf/spark/blob/11c83f83/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
index 3c6ae1c..6cd7b34 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
@@ -57,8 +57,8 @@ import org.apache.spark.unsafe.types.UTF8String
 object RowEncoder {
   def apply(schema: StructType): ExpressionEncoder[Row] = {
 val cls = classOf[Row]
-val inputObject = BoundReference(0, ObjectType(cls), nullable = false)
-val serializer = serializerFor(inputObject, schema)
+val

spark git commit: [SPARK-15547][SQL] nested case class in encoder can have different number of fields from the real schema

2016-06-03 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master eb10b481c -> 61b80d552


[SPARK-15547][SQL] nested case class in encoder can have different number of 
fields from the real schema

## What changes were proposed in this pull request?

There are 2 kinds of `GetStructField`:

1. resolved from `UnresolvedExtractValue`, and it will have a `name` property.
2. created when we build deserializer expression for nested tuple, no `name` 
property.

When we want to validate the ordinals of nested tuple, we should only catch 
`GetStructField` without the name property.

## How was this patch tested?

new test in `EncoderResolutionSuite`

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13474 from cloud-fan/ordinal-check.


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

Branch: refs/heads/master
Commit: 61b80d552aafb262b5f817f7bc9c0acd0328715b
Parents: eb10b48
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri Jun 3 14:26:24 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Jun 3 14:26:24 2016 -0700

--
 .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 7 ++-
 .../spark/sql/catalyst/encoders/EncoderResolutionSuite.scala  | 6 ++
 2 files changed, 12 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/61b80d55/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 4f6b483..0e68656 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
@@ -1964,7 +1964,12 @@ class Analyzer(
  */
 private def validateNestedTupleFields(deserializer: Expression): Unit = {
   val structChildToOrdinals = deserializer
-.collect { case g: GetStructField => g }
+// There are 2 kinds of `GetStructField`:
+//   1. resolved from `UnresolvedExtractValue`, and it will have a 
`name` property.
+//   2. created when we build deserializer expression for nested 
tuple, no `name` property.
+// Here we want to validate the ordinals of nested tuple, so we should 
only catch
+// `GetStructField` without the name property.
+.collect { case g: GetStructField if g.name.isEmpty => g }
 .groupBy(_.child)
 .mapValues(_.map(_.ordinal).distinct.sorted)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/61b80d55/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
index 7251202..802397d 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
@@ -115,6 +115,12 @@ class EncoderResolutionSuite extends PlanTest {
 }
   }
 
+  test("nested case class can have different number of fields from the real 
schema") {
+val encoder = ExpressionEncoder[(String, StringIntClass)]
+val attrs = Seq('a.string, 'b.struct('a.string, 'b.int, 'c.int))
+encoder.resolveAndBind(attrs)
+  }
+
   test("throw exception if real type is not compatible with encoder schema") {
 val msg1 = intercept[AnalysisException] {
   ExpressionEncoder[StringIntClass].resolveAndBind(Seq('a.string, 'b.long))


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



spark git commit: [SPARK-15494][SQL] encoder code cleanup

2016-06-03 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 3a9ee549c -> deda53a9e


[SPARK-15494][SQL] encoder code cleanup

## What changes were proposed in this pull request?

Our encoder framework has been evolved a lot, this PR tries to clean up the 
code to make it more readable and emphasise the concept that encoder should be 
used as a container of serde expressions.

1. move validation logic to analyzer instead of encoder
2. only have a `resolveAndBind` method in encoder instead of `resolve` and 
`bind`, as we don't have the encoder life cycle concept anymore.
3. `Dataset` don't need to keep a resolved encoder, as there is no such concept 
anymore. bound encoder is still needed to do serialization outside of query 
framework.
4. Using `BoundReference` to represent an unresolved field in deserializer 
expression is kind of weird, this PR adds a `GetColumnByOrdinal` for this 
purpose. (serializer expression still use `BoundReference`, we can replace it 
with `GetColumnByOrdinal` in follow-ups)

## How was this patch tested?

existing test

Author: Wenchen Fan <wenc...@databricks.com>
Author: Cheng Lian <l...@databricks.com>

Closes #13269 from cloud-fan/clean-encoder.

(cherry picked from commit 190ff274fd71662023a804cf98400c71f9f7da4f)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: deda53a9e43c8be39f9e5df88371a7904c1c0e9a
Parents: 3a9ee54
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri Jun 3 00:43:02 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Jun 3 00:43:10 2016 -0700

--
 .../linalg/UDTSerializationBenchmark.scala  |   2 +-
 .../scala/org/apache/spark/sql/Encoders.scala   |   3 +-
 .../spark/sql/catalyst/JavaTypeInference.scala  |   6 +-
 .../spark/sql/catalyst/ScalaReflection.scala| 307 +--
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  53 +++-
 .../sql/catalyst/analysis/unresolved.scala  |   7 +
 .../catalyst/encoders/ExpressionEncoder.scala   | 134 ++--
 .../sql/catalyst/encoders/RowEncoder.scala  |   8 +-
 .../sql/catalyst/plans/logical/object.scala |  19 +-
 .../encoders/EncoderResolutionSuite.scala   |  42 +--
 .../encoders/ExpressionEncoderSuite.scala   |  11 +-
 .../sql/catalyst/encoders/RowEncoderSuite.scala |  14 +-
 .../scala/org/apache/spark/sql/Dataset.scala|  48 +--
 .../spark/sql/KeyValueGroupedDataset.scala  |  26 +-
 .../spark/sql/RelationalGroupedDataset.scala|   2 +-
 .../aggregate/TypedAggregateExpression.scala|   6 +-
 .../scala/org/apache/spark/sql/functions.scala  |   2 +-
 .../org/apache/spark/sql/DatasetSuite.scala |   8 +-
 .../scala/org/apache/spark/sql/QueryTest.scala  |   8 +-
 .../sql/execution/GroupedIteratorSuite.scala|   6 +-
 .../apache/spark/sql/streaming/StreamTest.scala |   4 +-
 21 files changed, 324 insertions(+), 392 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/deda53a9/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
--
diff --git 
a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
 
b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
index be7110a..8b439e6 100644
--- 
a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
+++ 
b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
@@ -29,7 +29,7 @@ object UDTSerializationBenchmark {
 val iters = 1e2.toInt
 val numRows = 1e3.toInt
 
-val encoder = ExpressionEncoder[Vector].defaultBinding
+val encoder = ExpressionEncoder[Vector].resolveAndBind()
 
 val vectors = (1 to numRows).map { i =>
   Vectors.dense(Array.fill(1e5.toInt)(1.0 * i))

http://git-wip-us.apache.org/repos/asf/spark/blob/deda53a9/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala
--
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala
index fa96f82..673c587 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala
@@ -23,6 +23,7 @@ import scala.reflect.{classTag, ClassTag}
 import scala.reflect.runtime.universe.TypeTag
 
 import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst

spark git commit: [SPARK-15494][SQL] encoder code cleanup

2016-06-03 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master b9fcfb3bd -> 190ff274f


[SPARK-15494][SQL] encoder code cleanup

## What changes were proposed in this pull request?

Our encoder framework has been evolved a lot, this PR tries to clean up the 
code to make it more readable and emphasise the concept that encoder should be 
used as a container of serde expressions.

1. move validation logic to analyzer instead of encoder
2. only have a `resolveAndBind` method in encoder instead of `resolve` and 
`bind`, as we don't have the encoder life cycle concept anymore.
3. `Dataset` don't need to keep a resolved encoder, as there is no such concept 
anymore. bound encoder is still needed to do serialization outside of query 
framework.
4. Using `BoundReference` to represent an unresolved field in deserializer 
expression is kind of weird, this PR adds a `GetColumnByOrdinal` for this 
purpose. (serializer expression still use `BoundReference`, we can replace it 
with `GetColumnByOrdinal` in follow-ups)

## How was this patch tested?

existing test

Author: Wenchen Fan <wenc...@databricks.com>
Author: Cheng Lian <l...@databricks.com>

Closes #13269 from cloud-fan/clean-encoder.


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

Branch: refs/heads/master
Commit: 190ff274fd71662023a804cf98400c71f9f7da4f
Parents: b9fcfb3
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri Jun 3 00:43:02 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Jun 3 00:43:02 2016 -0700

--
 .../linalg/UDTSerializationBenchmark.scala  |   2 +-
 .../scala/org/apache/spark/sql/Encoders.scala   |   3 +-
 .../spark/sql/catalyst/JavaTypeInference.scala  |   6 +-
 .../spark/sql/catalyst/ScalaReflection.scala| 307 +--
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  53 +++-
 .../sql/catalyst/analysis/unresolved.scala  |   7 +
 .../catalyst/encoders/ExpressionEncoder.scala   | 134 ++--
 .../sql/catalyst/encoders/RowEncoder.scala  |   8 +-
 .../sql/catalyst/plans/logical/object.scala |  19 +-
 .../encoders/EncoderResolutionSuite.scala   |  42 +--
 .../encoders/ExpressionEncoderSuite.scala   |  11 +-
 .../sql/catalyst/encoders/RowEncoderSuite.scala |  14 +-
 .../scala/org/apache/spark/sql/Dataset.scala|  48 +--
 .../spark/sql/KeyValueGroupedDataset.scala  |  26 +-
 .../spark/sql/RelationalGroupedDataset.scala|   2 +-
 .../aggregate/TypedAggregateExpression.scala|   6 +-
 .../scala/org/apache/spark/sql/functions.scala  |   2 +-
 .../org/apache/spark/sql/DatasetSuite.scala |   8 +-
 .../scala/org/apache/spark/sql/QueryTest.scala  |   8 +-
 .../sql/execution/GroupedIteratorSuite.scala|   6 +-
 .../apache/spark/sql/streaming/StreamTest.scala |   4 +-
 21 files changed, 324 insertions(+), 392 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/190ff274/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
--
diff --git 
a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
 
b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
index be7110a..8b439e6 100644
--- 
a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
+++ 
b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala
@@ -29,7 +29,7 @@ object UDTSerializationBenchmark {
 val iters = 1e2.toInt
 val numRows = 1e3.toInt
 
-val encoder = ExpressionEncoder[Vector].defaultBinding
+val encoder = ExpressionEncoder[Vector].resolveAndBind()
 
 val vectors = (1 to numRows).map { i =>
   Vectors.dense(Array.fill(1e5.toInt)(1.0 * i))

http://git-wip-us.apache.org/repos/asf/spark/blob/190ff274/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala
--
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala
index fa96f82..673c587 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala
@@ -23,6 +23,7 @@ import scala.reflect.{classTag, ClassTag}
 import scala.reflect.runtime.universe.TypeTag
 
 import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal
 import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder}
 import 
org.apache.spark.sql.catalyst.expressions.

spark git commit: [SPARK-14959][SQL] handle partitioned table directories in distributed filesystem

2016-06-02 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 6dde27404 -> 76aa45d35


[SPARK-14959][SQL] handle partitioned table directories in distributed 
filesystem

## What changes were proposed in this pull request?
# The root cause:
When `DataSource.resolveRelation` is trying to build `ListingFileCatalog` 
object, `ListLeafFiles` is invoked where a list of `FileStatus` objects are 
retrieved from the provided path. These FileStatus objects include directories 
for the partitions (id=0 and id=2 in the jira). However, these directory 
`FileStatus` objects also try to invoke `getFileBlockLocations` where directory 
is not allowed for `DistributedFileSystem`, hence the exception happens.

This PR is to remove the block of code that invokes `getFileBlockLocations` for 
every FileStatus object of the provided path. Instead, we call 
`HadoopFsRelation.listLeafFiles` directly because this utility method filters 
out the directories before calling `getFileBlockLocations` for generating 
`LocatedFileStatus` objects.

## How was this patch tested?
Regtest is run. Manual test:
```
scala> 
spark.read.format("parquet").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_part").show
+-+---+
| text| id|
+-+---+
|hello|  0|
|world|  0|
|hello|  1|
|there|  1|
+-+---+

   
spark.read.format("orc").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_orc").show
+-+---+
| text| id|
+-+---+
|hello|  0|
|world|  0|
|hello|  1|
|there|  1|
+-+---+
```
I also tried it with 2 level of partitioning.
I have not found a way to add test case in the unit test bucket that can test a 
real hdfs file location. Any suggestions will be appreciated.

Author: Xin Wu <xi...@us.ibm.com>

Closes #13463 from xwu0226/SPARK-14959.


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

Branch: refs/heads/master
Commit: 76aa45d359d034e9ccaac64b36738d47e1e42f2c
Parents: 6dde274
Author: Xin Wu <xi...@us.ibm.com>
Authored: Thu Jun 2 22:49:17 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 22:49:17 2016 -0700

--
 .../datasources/ListingFileCatalog.scala| 36 ++--
 .../datasources/fileSourceInterfaces.scala  | 10 ++
 .../datasources/FileSourceStrategySuite.scala   |  1 +
 3 files changed, 14 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/76aa45d3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
index 644e5d6..dd3c96a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
@@ -83,40 +83,10 @@ class ListingFileCatalog(
   val statuses: Seq[FileStatus] = paths.flatMap { path =>
 val fs = path.getFileSystem(hadoopConf)
 logInfo(s"Listing $path on driver")
-
-val statuses = {
-  val stats = 
Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus])
-  if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-}
-
-statuses.map {
-  case f: LocatedFileStatus => f
-
-  // NOTE:
-  //
-  // - Although S3/S3A/S3N file system can be quite slow for remote 
file metadata
-  //   operations, calling `getFileBlockLocations` does no harm here 
since these file system
-  //   implementations don't actually issue RPC for this method.
-  //
-  // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should a
-  //   a big deal since we always use to `listLeafFilesInParallel` 
when the number of paths
-  //   exceeds threshold.
-  case f =>
-HadoopFsRelation.createLocatedFileStatus(f, 
fs.getFileBlockLocations(f, 0, f.getLen))
-}
-  }.filterNot { status =>
-val name = status.getPath.getName
-HadoopFsRelation.shouldFilterOut(name)
-  }
-
-  val (dirs, files) = statuses.partition(_.isDirectory)
-
-  // It uses [[LinkedHashSet]] since the order of files can affect the 
results. (SPARK-11500)
-  if (dirs.isEmpty) {
-mutable.LinkedHashSet(files: _*)
-  } else {
-  

spark git commit: [SPARK-14959][SQL] handle partitioned table directories in distributed filesystem

2016-06-02 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 d5bd64a20 -> 1e13d09c5


[SPARK-14959][SQL] handle partitioned table directories in distributed 
filesystem

## What changes were proposed in this pull request?
# The root cause:
When `DataSource.resolveRelation` is trying to build `ListingFileCatalog` 
object, `ListLeafFiles` is invoked where a list of `FileStatus` objects are 
retrieved from the provided path. These FileStatus objects include directories 
for the partitions (id=0 and id=2 in the jira). However, these directory 
`FileStatus` objects also try to invoke `getFileBlockLocations` where directory 
is not allowed for `DistributedFileSystem`, hence the exception happens.

This PR is to remove the block of code that invokes `getFileBlockLocations` for 
every FileStatus object of the provided path. Instead, we call 
`HadoopFsRelation.listLeafFiles` directly because this utility method filters 
out the directories before calling `getFileBlockLocations` for generating 
`LocatedFileStatus` objects.

## How was this patch tested?
Regtest is run. Manual test:
```
scala> 
spark.read.format("parquet").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_part").show
+-+---+
| text| id|
+-+---+
|hello|  0|
|world|  0|
|hello|  1|
|there|  1|
+-+---+

   
spark.read.format("orc").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_orc").show
+-+---+
| text| id|
+-+---+
|hello|  0|
|world|  0|
|hello|  1|
|there|  1|
+-+---+
```
I also tried it with 2 level of partitioning.
I have not found a way to add test case in the unit test bucket that can test a 
real hdfs file location. Any suggestions will be appreciated.

Author: Xin Wu <xi...@us.ibm.com>

Closes #13463 from xwu0226/SPARK-14959.

(cherry picked from commit 76aa45d359d034e9ccaac64b36738d47e1e42f2c)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 1e13d09c526ec37c344950b07d938751bbd6fd0a
Parents: d5bd64a
Author: Xin Wu <xi...@us.ibm.com>
Authored: Thu Jun 2 22:49:17 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 22:49:24 2016 -0700

--
 .../datasources/ListingFileCatalog.scala| 36 ++--
 .../datasources/fileSourceInterfaces.scala  | 10 ++
 .../datasources/FileSourceStrategySuite.scala   |  1 +
 3 files changed, 14 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1e13d09c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
index 644e5d6..dd3c96a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
@@ -83,40 +83,10 @@ class ListingFileCatalog(
   val statuses: Seq[FileStatus] = paths.flatMap { path =>
 val fs = path.getFileSystem(hadoopConf)
 logInfo(s"Listing $path on driver")
-
-val statuses = {
-  val stats = 
Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus])
-  if (pathFilter != null) stats.filter(f => 
pathFilter.accept(f.getPath)) else stats
-}
-
-statuses.map {
-  case f: LocatedFileStatus => f
-
-  // NOTE:
-  //
-  // - Although S3/S3A/S3N file system can be quite slow for remote 
file metadata
-  //   operations, calling `getFileBlockLocations` does no harm here 
since these file system
-  //   implementations don't actually issue RPC for this method.
-  //
-  // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should a
-  //   a big deal since we always use to `listLeafFilesInParallel` 
when the number of paths
-  //   exceeds threshold.
-  case f =>
-HadoopFsRelation.createLocatedFileStatus(f, 
fs.getFileBlockLocations(f, 0, f.getLen))
-}
-  }.filterNot { status =>
-val name = status.getPath.getName
-HadoopFsRelation.shouldFilterOut(name)
-  }
-
-  val (dirs, files) = statuses.partition(_.isDirectory)
-
-  // It uses [[LinkedHashSet]] since the order of files can aff

spark git commit: [SPARK-15733][SQL] Makes the explain output less verbose by hiding some verbose output like None, null, empty List, and etc.

2016-06-02 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 8be6e8cc5 -> d5bd64a20


[SPARK-15733][SQL] Makes the explain output less verbose by hiding some verbose 
output like None, null, empty List, and etc.

## What changes were proposed in this pull request?

This PR makes the explain output less verbose by hiding some verbose output 
like `None`, `null`, empty List `[]`, empty set `{}`, and etc.

**Before change**:

```
== Physical Plan ==
ExecutedCommand
:  +- ShowTablesCommand None, None
```

**After change**:

```
== Physical Plan ==
ExecutedCommand
:  +- ShowTablesCommand
```

## How was this patch tested?

Manual test.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #13470 from clockfly/verbose_breakdown_4.

(cherry picked from commit 6dde27404cb3d921d75dd6afca4b383f9df5976a)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: d5bd64a20db2377fa5efc053607de730411489cb
Parents: 8be6e8c
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Thu Jun 2 22:45:37 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 22:45:46 2016 -0700

--
 .../spark/sql/catalyst/trees/TreeNode.scala   | 18 +-
 1 file changed, 13 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d5bd64a2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index 3ebd815..50481cd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -427,13 +427,21 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product {
   private lazy val allChildren: Set[TreeNode[_]] = (children ++ 
innerChildren).toSet[TreeNode[_]]
 
   /** Returns a string representing the arguments to this node, minus any 
children */
-  def argString: String = productIterator.flatMap {
+  def argString: String = stringArgs.flatMap {
 case tn: TreeNode[_] if allChildren.contains(tn) => Nil
 case Some(tn: TreeNode[_]) if allChildren.contains(tn) => Nil
-case tn: TreeNode[_] => s"${tn.simpleString}" :: Nil
-case seq: Seq[BaseType] if seq.toSet.subsetOf(children.toSet) => Nil
-case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil
-case set: Set[_] => set.mkString("{", ",", "}") :: Nil
+case Some(tn: TreeNode[_]) => tn.simpleString :: Nil
+case tn: TreeNode[_] => tn.simpleString :: Nil
+case seq: Seq[Any] if 
seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil
+case iter: Iterable[_] if iter.isEmpty => Nil
+case seq: Seq[_] => seq.mkString("[", ", ", "]") :: Nil
+case set: Set[_] => set.mkString("{", ", ", "}") :: Nil
+case array: Array[_] if array.isEmpty => Nil
+case array: Array[_] => array.mkString("[", ", ", "]") :: Nil
+case null => Nil
+case None => Nil
+case Some(null) => Nil
+case Some(any) => any :: Nil
 case other => other :: Nil
   }.mkString(", ")
 


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



spark git commit: [SPARK-15732][SQL] better error message when use java reserved keyword as field name

2016-06-02 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 d02f2926b -> 180dd53b7


[SPARK-15732][SQL] better error message when use java reserved keyword as field 
name

## What changes were proposed in this pull request?

When users create a case class and use java reserved keyword as field name, 
spark sql will generate illegal java code and throw exception at runtime.

This PR checks the field names when building the encoder, and if illegal field 
names are used, throw exception immediately with a good error message.

## How was this patch tested?

new test in DatasetSuite

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13485 from cloud-fan/java.

(cherry picked from commit 6323e4bd763eafe23c619a89552c00e1244e4690)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 180dd53b77a042235070e27b732c8f18165894e3
Parents: d02f292
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Thu Jun 2 18:13:04 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 18:13:12 2016 -0700

--
 .../org/apache/spark/sql/catalyst/ScalaReflection.scala  | 11 +++
 .../test/scala/org/apache/spark/sql/DatasetSuite.scala   | 10 ++
 2 files changed, 21 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/180dd53b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index 052cc48..4750861 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -582,6 +582,11 @@ object ScalaReflection extends ScalaReflection {
 case t if definedByConstructorParams(t) =>
   val params = getConstructorParameters(t)
   val nonNullOutput = CreateNamedStruct(params.flatMap { case 
(fieldName, fieldType) =>
+if (javaKeywords.contains(fieldName)) {
+  throw new UnsupportedOperationException(s"`$fieldName` is a 
reserved keyword and " +
+"cannot be used as field name\n" + 
walkedTypePath.mkString("\n"))
+}
+
 val fieldValue = Invoke(inputObject, fieldName, 
dataTypeFor(fieldType))
 val clsName = getClassNameFromType(fieldType)
 val newPath = s"""- field (class: "$clsName", name: 
"$fieldName")""" +: walkedTypePath
@@ -720,6 +725,12 @@ object ScalaReflection extends ScalaReflection {
 tpe <:< localTypeOf[Product] || tpe <:< 
localTypeOf[DefinedByConstructorParams]
   }
 
+  private val javaKeywords = Set("abstract", "assert", "boolean", "break", 
"byte", "case", "catch",
+"char", "class", "const", "continue", "default", "do", "double", "else", 
"extends", "false",
+"final", "finally", "float", "for", "goto", "if", "implements", "import", 
"instanceof", "int",
+"interface", "long", "native", "new", "null", "package", "private", 
"protected", "public",
+"return", "short", "static", "strictfp", "super", "switch", 
"synchronized", "this", "throw",
+"throws", "transient", "true", "try", "void", "volatile", "while")
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/180dd53b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index a3881ff..df8f4b0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -786,6 +786,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext 
{
 va

spark git commit: [SPARK-15732][SQL] better error message when use java reserved keyword as field name

2016-06-02 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master d1c1fbc34 -> 6323e4bd7


[SPARK-15732][SQL] better error message when use java reserved keyword as field 
name

## What changes were proposed in this pull request?

When users create a case class and use java reserved keyword as field name, 
spark sql will generate illegal java code and throw exception at runtime.

This PR checks the field names when building the encoder, and if illegal field 
names are used, throw exception immediately with a good error message.

## How was this patch tested?

new test in DatasetSuite

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13485 from cloud-fan/java.


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

Branch: refs/heads/master
Commit: 6323e4bd763eafe23c619a89552c00e1244e4690
Parents: d1c1fbc
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Thu Jun 2 18:13:04 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 18:13:04 2016 -0700

--
 .../org/apache/spark/sql/catalyst/ScalaReflection.scala  | 11 +++
 .../test/scala/org/apache/spark/sql/DatasetSuite.scala   | 10 ++
 2 files changed, 21 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6323e4bd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index 052cc48..4750861 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -582,6 +582,11 @@ object ScalaReflection extends ScalaReflection {
 case t if definedByConstructorParams(t) =>
   val params = getConstructorParameters(t)
   val nonNullOutput = CreateNamedStruct(params.flatMap { case 
(fieldName, fieldType) =>
+if (javaKeywords.contains(fieldName)) {
+  throw new UnsupportedOperationException(s"`$fieldName` is a 
reserved keyword and " +
+"cannot be used as field name\n" + 
walkedTypePath.mkString("\n"))
+}
+
 val fieldValue = Invoke(inputObject, fieldName, 
dataTypeFor(fieldType))
 val clsName = getClassNameFromType(fieldType)
 val newPath = s"""- field (class: "$clsName", name: 
"$fieldName")""" +: walkedTypePath
@@ -720,6 +725,12 @@ object ScalaReflection extends ScalaReflection {
 tpe <:< localTypeOf[Product] || tpe <:< 
localTypeOf[DefinedByConstructorParams]
   }
 
+  private val javaKeywords = Set("abstract", "assert", "boolean", "break", 
"byte", "case", "catch",
+"char", "class", "const", "continue", "default", "do", "double", "else", 
"extends", "false",
+"final", "finally", "float", "for", "goto", "if", "implements", "import", 
"instanceof", "int",
+"interface", "long", "native", "new", "null", "package", "private", 
"protected", "public",
+"return", "short", "static", "strictfp", "super", "switch", 
"synchronized", "this", "throw",
+"throws", "transient", "true", "try", "void", "volatile", "while")
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/6323e4bd/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index a3881ff..df8f4b0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -786,6 +786,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext 
{
 val result = joined.collect().toSet
 assert(result == Set(ClassData("a", 1) -> null, ClassData("b", 2) -> 

spark git commit: [SPARK-15734][SQL] Avoids printing internal row in explain output

2016-06-02 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 810472a92 -> 1153a4098


[SPARK-15734][SQL] Avoids printing internal row in explain output

## What changes were proposed in this pull request?

This PR avoids printing internal rows in explain output for some operators.

**Before change:**

```
scala> (1 to 10).toSeq.map(_ => (1,2,3)).toDF().createTempView("df3")
scala> spark.sql("select * from df3 where 1=2").explain(true)
...
== Analyzed Logical Plan ==
_1: int, _2: int, _3: int
Project [_1#37,_2#38,_3#39]
+- Filter (1 = 2)
   +- SubqueryAlias df3
  +- LocalRelation [_1#37,_2#38,_3#39], 
[[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3]]
...
== Physical Plan ==
LocalTableScan [_1#37,_2#38,_3#39]
```

**After change:**

```
scala> spark.sql("select * from df3 where 1=2").explain(true)
...
== Analyzed Logical Plan ==
_1: int, _2: int, _3: int
Project [_1#58,_2#59,_3#60]
+- Filter (1 = 2)
   +- SubqueryAlias df3
  +- LocalRelation [_1#58,_2#59,_3#60]
...
== Physical Plan ==
LocalTableScan , [_1#58,_2#59,_3#60]
```

## How was this patch tested?
Manual test.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #13471 from clockfly/verbose_breakdown_5.

(cherry picked from commit 985d532812cf176d0e12b799c723f917282b6813)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 1153a4098956dd4ba408ab6a1c0d4263de2ef0a7
Parents: 810472a
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Thu Jun 2 16:21:33 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 16:21:40 2016 -0700

--
 .../spark/sql/catalyst/plans/logical/LocalRelation.scala | 8 +++-
 .../scala/org/apache/spark/sql/execution/ExistingRDD.scala   | 2 ++
 .../org/apache/spark/sql/execution/LocalTableScanExec.scala  | 8 
 3 files changed, 17 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1153a409/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
index 5813b74..87b8647 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
@@ -57,7 +57,13 @@ case class LocalRelation(output: Seq[Attribute], data: 
Seq[InternalRow] = Nil)
 LocalRelation(output.map(_.newInstance()), data).asInstanceOf[this.type]
   }
 
-  override protected def stringArgs = Iterator(output)
+  override protected def stringArgs: Iterator[Any] = {
+if (data.isEmpty) {
+  Iterator("", output)
+} else {
+  Iterator(output)
+}
+  }
 
   override def sameResult(plan: LogicalPlan): Boolean = plan match {
 case LocalRelation(otherOutput, otherData) =>

http://git-wip-us.apache.org/repos/asf/spark/blob/1153a409/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index fef3255..b8b3926 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -91,6 +91,8 @@ private[sql] case class LogicalRDD(
 case _ => false
   }
 
+  override protected def stringArgs: Iterator[Any] = Iterator(output)
+
   override def producedAttributes: AttributeSet = outputSet
 
   @transient override lazy val statistics: Statistics = Statistics(

http://git-wip-us.apache.org/repos/asf/spark/blob/1153a409/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
index c5e78b0..df2f238 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
@@ 

spark git commit: [SPARK-15734][SQL] Avoids printing internal row in explain output

2016-06-02 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 431542765 -> 985d53281


[SPARK-15734][SQL] Avoids printing internal row in explain output

## What changes were proposed in this pull request?

This PR avoids printing internal rows in explain output for some operators.

**Before change:**

```
scala> (1 to 10).toSeq.map(_ => (1,2,3)).toDF().createTempView("df3")
scala> spark.sql("select * from df3 where 1=2").explain(true)
...
== Analyzed Logical Plan ==
_1: int, _2: int, _3: int
Project [_1#37,_2#38,_3#39]
+- Filter (1 = 2)
   +- SubqueryAlias df3
  +- LocalRelation [_1#37,_2#38,_3#39], 
[[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3]]
...
== Physical Plan ==
LocalTableScan [_1#37,_2#38,_3#39]
```

**After change:**

```
scala> spark.sql("select * from df3 where 1=2").explain(true)
...
== Analyzed Logical Plan ==
_1: int, _2: int, _3: int
Project [_1#58,_2#59,_3#60]
+- Filter (1 = 2)
   +- SubqueryAlias df3
  +- LocalRelation [_1#58,_2#59,_3#60]
...
== Physical Plan ==
LocalTableScan , [_1#58,_2#59,_3#60]
```

## How was this patch tested?
Manual test.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #13471 from clockfly/verbose_breakdown_5.


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

Branch: refs/heads/master
Commit: 985d532812cf176d0e12b799c723f917282b6813
Parents: 4315427
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Thu Jun 2 16:21:33 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 16:21:33 2016 -0700

--
 .../spark/sql/catalyst/plans/logical/LocalRelation.scala | 8 +++-
 .../scala/org/apache/spark/sql/execution/ExistingRDD.scala   | 2 ++
 .../org/apache/spark/sql/execution/LocalTableScanExec.scala  | 8 
 3 files changed, 17 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/985d5328/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
index 5813b74..87b8647 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
@@ -57,7 +57,13 @@ case class LocalRelation(output: Seq[Attribute], data: 
Seq[InternalRow] = Nil)
 LocalRelation(output.map(_.newInstance()), data).asInstanceOf[this.type]
   }
 
-  override protected def stringArgs = Iterator(output)
+  override protected def stringArgs: Iterator[Any] = {
+if (data.isEmpty) {
+  Iterator("", output)
+} else {
+  Iterator(output)
+}
+  }
 
   override def sameResult(plan: LogicalPlan): Boolean = plan match {
 case LocalRelation(otherOutput, otherData) =>

http://git-wip-us.apache.org/repos/asf/spark/blob/985d5328/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index fef3255..b8b3926 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -91,6 +91,8 @@ private[sql] case class LogicalRDD(
 case _ => false
   }
 
+  override protected def stringArgs: Iterator[Any] = Iterator(output)
+
   override def producedAttributes: AttributeSet = outputSet
 
   @transient override lazy val statistics: Statistics = Statistics(

http://git-wip-us.apache.org/repos/asf/spark/blob/985d5328/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
index c5e78b0..df2f238 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala
@@ -48,6 +48,14 @@ private[sql] case class LocalTableScanExec(
 }
   }
 
+  override protected def stringArgs: Iterator[Any] = {
+  

spark git commit: [SPARK-15719][SQL] Disables writing Parquet summary files by default

2016-06-02 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 847ccf793 -> 810472a92


[SPARK-15719][SQL] Disables writing Parquet summary files by default

## What changes were proposed in this pull request?

This PR disables writing Parquet summary files by default (i.e., when Hadoop 
configuration "parquet.enable.summary-metadata" is not set).

Please refer to [SPARK-15719][1] for more details.

## How was this patch tested?

New test case added in `ParquetQuerySuite` to check no summary files are 
written by default.

[1]: https://issues.apache.org/jira/browse/SPARK-15719

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

Closes #13455 from liancheng/spark-15719-disable-parquet-summary-files.

(cherry picked from commit 431542765785304edb76a19885fbc5f9b8ae7d64)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 810472a928811f078ef42825d1cf16cf711fcefb
Parents: 847ccf7
Author: Cheng Lian <l...@databricks.com>
Authored: Thu Jun 2 16:16:27 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jun 2 16:16:37 2016 -0700

--
 .../datasources/parquet/ParquetFileFormat.scala |  7 -
 .../datasources/parquet/ParquetIOSuite.scala| 20 +++--
 .../datasources/parquet/ParquetQuerySuite.scala | 31 
 .../parquet/ParquetSchemaSuite.scala| 19 ++--
 .../sources/ParquetHadoopFsRelationSuite.scala  | 29 ++
 5 files changed, 62 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/810472a9/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
index ff7962d..ada9cd4 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
@@ -124,6 +124,11 @@ private[sql] class ParquetFileFormat
 // Sets compression scheme
 conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec)
 
+// SPARK-15719: Disables writing Parquet summary files by default.
+if (conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) {
+  conf.setBoolean(ParquetOutputFormat.ENABLE_JOB_SUMMARY, false)
+}
+
 new OutputWriterFactory {
   override def newInstance(
   path: String,
@@ -786,7 +791,7 @@ private[sql] object ParquetFileFormat extends Logging {
 //
 // Parquet requires `FileStatus`es to read footers.  Here we try to send 
cached `FileStatus`es
 // to executor side to avoid fetching them again.  However, `FileStatus` 
is not `Serializable`
-// but only `Writable`.  What makes it worth, for some reason, 
`FileStatus` doesn't play well
+// but only `Writable`.  What makes it worse, for some reason, 
`FileStatus` doesn't play well
 // with `SerializableWritable[T]` and always causes a weird 
`IllegalStateException`.  These
 // facts virtually prevents us to serialize `FileStatus`es.
 //

http://git-wip-us.apache.org/repos/asf/spark/blob/810472a9/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
index d0107aa..92f2db3 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
@@ -515,17 +515,19 @@ class ParquetIOSuite extends QueryTest with ParquetTest 
with SharedSQLContext {
 
 val hadoopConf = spark.sessionState.newHadoopConfWithOptions(extraOptions)
 
-withTempPath { dir =>
-  val path = s"${dir.getCanonicalPath}/part-r-0.parquet"
-  spark.range(1 << 16).selectExpr("(id % 4) AS i")
-
.coalesce(1).write.options(extraOptions).mode("overwrite").parquet(path)
+withSQLConf(ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true") {
+  withTempPath { dir =>
+val path

spark git commit: [SPARK-15441][SQL] support null object in Dataset outer-join

2016-06-01 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 8cdc0d4da -> 91812226f


[SPARK-15441][SQL] support null object in Dataset outer-join

## What changes were proposed in this pull request?

Currently we can't encode top level null object into internal row, as Spark SQL 
doesn't allow row to be null, only its columns can be null.

This is not a problem before, as we assume the input object is never null. 
However, for outer join, we do need the semantics of null object.

This PR fixes this problem by making both join sides produce a single column, 
i.e. nest the logical plan output(by `CreateStruct`), so that we have an extra 
level to represent top level null obejct.

## How was this patch tested?

new test in `DatasetSuite`

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13425 from cloud-fan/outer-join2.

(cherry picked from commit 8640cdb836b4964e4af891d9959af64a2e1f304e)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 91812226fc18d476be3dcef1351ff5f5ef6ed86f
Parents: 8cdc0d4
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Wed Jun 1 16:16:54 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed Jun 1 16:18:23 2016 -0700

--
 .../catalyst/encoders/ExpressionEncoder.scala   |  3 +-
 .../catalyst/expressions/objects/objects.scala  |  1 -
 .../scala/org/apache/spark/sql/Dataset.scala| 67 ++--
 .../org/apache/spark/sql/DatasetSuite.scala | 23 +++
 4 files changed, 59 insertions(+), 35 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/91812226/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
index f21a39a..2296946 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
@@ -125,12 +125,13 @@ object ExpressionEncoder {
 }
   } else {
 val input = BoundReference(index, enc.schema, nullable = true)
-enc.deserializer.transformUp {
+val deserialized = enc.deserializer.transformUp {
   case UnresolvedAttribute(nameParts) =>
 assert(nameParts.length == 1)
 UnresolvedExtractValue(input, Literal(nameParts.head))
   case BoundReference(ordinal, dt, _) => GetStructField(input, ordinal)
 }
+If(IsNull(input), Literal.create(null, deserialized.dataType), 
deserialized)
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/91812226/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
index 2f2323f..c2e3ab8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions.objects
 
 import java.lang.reflect.Modifier
 
-import scala.annotation.tailrec
 import scala.language.existentials
 import scala.reflect.ClassTag
 

http://git-wip-us.apache.org/repos/asf/spark/blob/91812226/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 3a6ec45..369b772 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -747,31 +747,62 @@ class Dataset[T] private[sql](
*/
   @Experimental
   def joinWith[U](other: Dataset[U], condition: Column, joinType: String): 
Dataset[(T, U)] = {
-val left = this.logicalPlan
-val right = other.logicalPlan
-
-val joined = sparkSession.sessionState.executePlan(Join(left, right, 
joinType =
-  JoinType(joinType), Some(condition.expr)))
-val leftOutput = joined.analyzed.output.

spark git commit: [SPARK-15441][SQL] support null object in Dataset outer-join

2016-06-01 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 7bb64aae2 -> 8640cdb83


[SPARK-15441][SQL] support null object in Dataset outer-join

## What changes were proposed in this pull request?

Currently we can't encode top level null object into internal row, as Spark SQL 
doesn't allow row to be null, only its columns can be null.

This is not a problem before, as we assume the input object is never null. 
However, for outer join, we do need the semantics of null object.

This PR fixes this problem by making both join sides produce a single column, 
i.e. nest the logical plan output(by `CreateStruct`), so that we have an extra 
level to represent top level null obejct.

## How was this patch tested?

new test in `DatasetSuite`

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13425 from cloud-fan/outer-join2.


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

Branch: refs/heads/master
Commit: 8640cdb836b4964e4af891d9959af64a2e1f304e
Parents: 7bb64aa
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Wed Jun 1 16:16:54 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed Jun 1 16:16:54 2016 -0700

--
 .../catalyst/encoders/ExpressionEncoder.scala   |  3 +-
 .../catalyst/expressions/objects/objects.scala  |  1 -
 .../scala/org/apache/spark/sql/Dataset.scala| 67 ++--
 .../org/apache/spark/sql/DatasetSuite.scala | 23 +++
 4 files changed, 59 insertions(+), 35 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8640cdb8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
index f21a39a..2296946 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
@@ -125,12 +125,13 @@ object ExpressionEncoder {
 }
   } else {
 val input = BoundReference(index, enc.schema, nullable = true)
-enc.deserializer.transformUp {
+val deserialized = enc.deserializer.transformUp {
   case UnresolvedAttribute(nameParts) =>
 assert(nameParts.length == 1)
 UnresolvedExtractValue(input, Literal(nameParts.head))
   case BoundReference(ordinal, dt, _) => GetStructField(input, ordinal)
 }
+If(IsNull(input), Literal.create(null, deserialized.dataType), 
deserialized)
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8640cdb8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
index 2f2323f..c2e3ab8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions.objects
 
 import java.lang.reflect.Modifier
 
-import scala.annotation.tailrec
 import scala.language.existentials
 import scala.reflect.ClassTag
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8640cdb8/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 3a6ec45..369b772 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -747,31 +747,62 @@ class Dataset[T] private[sql](
*/
   @Experimental
   def joinWith[U](other: Dataset[U], condition: Column, joinType: String): 
Dataset[(T, U)] = {
-val left = this.logicalPlan
-val right = other.logicalPlan
-
-val joined = sparkSession.sessionState.executePlan(Join(left, right, 
joinType =
-  JoinType(joinType), Some(condition.expr)))
-val leftOutput = joined.analyzed.output.take(left.output.length)
-val rightOutput = joined.analyzed.output.takeRight(right.output.length)
+// Creates a Join node and re

spark git commit: [SPARK-9876] [BRANCH-2.0] Revert "[SPARK-9876][SQL] Update Parquet to 1.8.1."

2016-06-01 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 e033fd50f -> 8cdc0d4da


[SPARK-9876] [BRANCH-2.0] Revert "[SPARK-9876][SQL] Update Parquet to 1.8.1."

## What changes were proposed in this pull request?
Since we are pretty late in the 2.0 release cycle, it is not clear if this 
upgrade can be tested thoroughly and if we can resolve the regression issue 
that we observed before. This PR reverts #13280 from branch 2.0.

## How was this patch tested?
Existing tests

This reverts commit 776d183c82b424ef7c3cae30537d8afe9b9eee83.

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

Closes #13450 from yhuai/revertParquet1.8.1-branch-2.0.


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

Branch: refs/heads/branch-2.0
Commit: 8cdc0d4da6324f48e1a25496dc4bc7635d9472ef
Parents: e033fd5
Author: Yin Huai <yh...@databricks.com>
Authored: Wed Jun 1 16:05:13 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed Jun 1 16:05:13 2016 -0700

--
 dev/deps/spark-deps-hadoop-2.2  | 11 +--
 dev/deps/spark-deps-hadoop-2.3  | 11 +--
 dev/deps/spark-deps-hadoop-2.4  | 11 +--
 dev/deps/spark-deps-hadoop-2.6  | 11 +--
 dev/deps/spark-deps-hadoop-2.7  | 11 +--
 pom.xml |  2 +-
 .../SpecificParquetRecordReaderBase.java| 20 ++---
 .../parquet/CatalystReadSupport.scala   | 12 +--
 .../parquet/CatalystSchemaConverter.scala   | 16 
 .../datasources/parquet/ParquetFilters.scala| 83 
 .../parquet/ParquetSchemaSuite.scala| 20 ++---
 11 files changed, 117 insertions(+), 91 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8cdc0d4d/dev/deps/spark-deps-hadoop-2.2
--
diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2
index b5c38a6..96001ea 100644
--- a/dev/deps/spark-deps-hadoop-2.2
+++ b/dev/deps/spark-deps-hadoop-2.2
@@ -129,13 +129,14 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.3.jar
-parquet-column-1.8.1.jar
-parquet-common-1.8.1.jar
-parquet-encoding-1.8.1.jar
+parquet-column-1.7.0.jar
+parquet-common-1.7.0.jar
+parquet-encoding-1.7.0.jar
 parquet-format-2.3.0-incubating.jar
-parquet-hadoop-1.8.1.jar
+parquet-generator-1.7.0.jar
+parquet-hadoop-1.7.0.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.8.1.jar
+parquet-jackson-1.7.0.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/8cdc0d4d/dev/deps/spark-deps-hadoop-2.3
--
diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3
index 969df04..9f3d9ad 100644
--- a/dev/deps/spark-deps-hadoop-2.3
+++ b/dev/deps/spark-deps-hadoop-2.3
@@ -136,13 +136,14 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.3.jar
-parquet-column-1.8.1.jar
-parquet-common-1.8.1.jar
-parquet-encoding-1.8.1.jar
+parquet-column-1.7.0.jar
+parquet-common-1.7.0.jar
+parquet-encoding-1.7.0.jar
 parquet-format-2.3.0-incubating.jar
-parquet-hadoop-1.8.1.jar
+parquet-generator-1.7.0.jar
+parquet-hadoop-1.7.0.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.8.1.jar
+parquet-jackson-1.7.0.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/8cdc0d4d/dev/deps/spark-deps-hadoop-2.4
--
diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4
index f0491ec..77d5266 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -136,13 +136,14 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.3.jar
-parquet-column-1.8.1.jar
-parquet-common-1.8.1.jar
-parquet-encoding-1.8.1.jar
+parquet-column-1.7.0.jar
+parquet-common-1.7.0.jar
+parquet-encoding-1.7.0.jar
 parquet-format-2.3.0-incubating.jar
-parquet-hadoop-1.8.1.jar
+parquet-generator-1.7.0.jar
+parquet-hadoop-1.7.0.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.8.1.jar
+parquet-jackson-1.7.0.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/8cdc0d4d/dev/deps/spark-deps-hadoop-2.6
--
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index b3dced6..9afe50f 100644
--- a/dev/deps/spark

spark git commit: [SPARK-15269][SQL] Removes unexpected empty table directories created while creating external Spark SQL data sourcet tables.

2016-06-01 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 44052a707 -> e033fd50f


[SPARK-15269][SQL] Removes unexpected empty table directories created while 
creating external Spark SQL data sourcet tables.

This PR is an alternative to #13120 authored by xwu0226.

## What changes were proposed in this pull request?

When creating an external Spark SQL data source table and persisting its 
metadata to Hive metastore, we don't use the standard Hive `Table.dataLocation` 
field because Hive only allows directory paths as data locations while Spark 
SQL also allows file paths. However, if we don't set `Table.dataLocation`, Hive 
always creates an unexpected empty table directory under database location, but 
doesn't remove it while dropping the table (because the table is external).

This PR works around this issue by explicitly setting `Table.dataLocation` and 
then manullay removing the created directory after creating the external table.

Please refer to [this JIRA comment][1] for more details about why we chose this 
approach as a workaround.

[1]: 
https://issues.apache.org/jira/browse/SPARK-15269?focusedCommentId=15297408=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15297408

## How was this patch tested?

1. A new test case is added in `HiveQuerySuite` for this case
2. Updated `ShowCreateTableSuite` to use the same table name in all test cases. 
(This is how I hit this issue at the first place.)

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

Closes #13270 from liancheng/spark-15269-unpleasant-fix.

(cherry picked from commit 7bb64aae27f670531699f59d3f410e38866609b7)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: e033fd50f0fcefb2a6cffb763ff7e026b0066c07
Parents: 44052a7
Author: Cheng Lian <l...@databricks.com>
Authored: Wed Jun 1 16:02:27 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed Jun 1 16:02:37 2016 -0700

--
 .../apache/spark/sql/AnalysisException.scala|  5 ++-
 .../sql/catalyst/catalog/SessionCatalog.scala   |  2 +-
 .../org/apache/spark/sql/SparkSession.scala |  6 +--
 .../command/createDataSourceTables.scala|  4 +-
 .../apache/spark/sql/internal/HiveSerDe.scala   |  1 -
 .../spark/sql/hive/HiveExternalCatalog.scala| 45 ++--
 .../apache/spark/sql/hive/HiveSharedState.scala |  4 +-
 .../spark/sql/hive/client/HiveClientImpl.scala  | 22 --
 .../sql/hive/HiveExternalCatalogSuite.scala |  3 +-
 .../sql/hive/MetastoreDataSourcesSuite.scala| 13 ++
 .../spark/sql/hive/ShowCreateTableSuite.scala   | 36 
 11 files changed, 105 insertions(+), 36 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e033fd50/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 d2003fd..6911843 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
@@ -32,8 +32,9 @@ class AnalysisException protected[sql] (
 val message: String,
 val line: Option[Int] = None,
 val startPosition: Option[Int] = None,
-val plan: Option[LogicalPlan] = None)
-  extends Exception with Serializable {
+val plan: Option[LogicalPlan] = None,
+val cause: Option[Throwable] = None)
+  extends Exception(message, cause.orNull) with Serializable {
 
   def withPosition(line: Option[Int], startPosition: Option[Int]): 
AnalysisException = {
 val newException = new AnalysisException(message, line, startPosition)

http://git-wip-us.apache.org/repos/asf/spark/blob/e033fd50/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index cf9286e..371c198 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -22,7 +22,7 @@ import javax.annotation.concurrent.GuardedBy
 import scala.collection.mutable
 
 import org.apache.hadoop.conf.Configuration
-import org.ap

spark git commit: [SPARK-15269][SQL] Removes unexpected empty table directories created while creating external Spark SQL data sourcet tables.

2016-06-01 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 9e2643b21 -> 7bb64aae2


[SPARK-15269][SQL] Removes unexpected empty table directories created while 
creating external Spark SQL data sourcet tables.

This PR is an alternative to #13120 authored by xwu0226.

## What changes were proposed in this pull request?

When creating an external Spark SQL data source table and persisting its 
metadata to Hive metastore, we don't use the standard Hive `Table.dataLocation` 
field because Hive only allows directory paths as data locations while Spark 
SQL also allows file paths. However, if we don't set `Table.dataLocation`, Hive 
always creates an unexpected empty table directory under database location, but 
doesn't remove it while dropping the table (because the table is external).

This PR works around this issue by explicitly setting `Table.dataLocation` and 
then manullay removing the created directory after creating the external table.

Please refer to [this JIRA comment][1] for more details about why we chose this 
approach as a workaround.

[1]: 
https://issues.apache.org/jira/browse/SPARK-15269?focusedCommentId=15297408=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15297408

## How was this patch tested?

1. A new test case is added in `HiveQuerySuite` for this case
2. Updated `ShowCreateTableSuite` to use the same table name in all test cases. 
(This is how I hit this issue at the first place.)

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

Closes #13270 from liancheng/spark-15269-unpleasant-fix.


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

Branch: refs/heads/master
Commit: 7bb64aae27f670531699f59d3f410e38866609b7
Parents: 9e2643b
Author: Cheng Lian <l...@databricks.com>
Authored: Wed Jun 1 16:02:27 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed Jun 1 16:02:27 2016 -0700

--
 .../apache/spark/sql/AnalysisException.scala|  5 ++-
 .../sql/catalyst/catalog/SessionCatalog.scala   |  2 +-
 .../org/apache/spark/sql/SparkSession.scala |  6 +--
 .../command/createDataSourceTables.scala|  4 +-
 .../apache/spark/sql/internal/HiveSerDe.scala   |  1 -
 .../spark/sql/hive/HiveExternalCatalog.scala| 45 ++--
 .../apache/spark/sql/hive/HiveSharedState.scala |  4 +-
 .../spark/sql/hive/client/HiveClientImpl.scala  | 22 --
 .../sql/hive/HiveExternalCatalogSuite.scala |  3 +-
 .../sql/hive/MetastoreDataSourcesSuite.scala| 13 ++
 .../spark/sql/hive/ShowCreateTableSuite.scala   | 36 
 11 files changed, 105 insertions(+), 36 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7bb64aae/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 d2003fd..6911843 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
@@ -32,8 +32,9 @@ class AnalysisException protected[sql] (
 val message: String,
 val line: Option[Int] = None,
 val startPosition: Option[Int] = None,
-val plan: Option[LogicalPlan] = None)
-  extends Exception with Serializable {
+val plan: Option[LogicalPlan] = None,
+val cause: Option[Throwable] = None)
+  extends Exception(message, cause.orNull) with Serializable {
 
   def withPosition(line: Option[Int], startPosition: Option[Int]): 
AnalysisException = {
 val newException = new AnalysisException(message, line, startPosition)

http://git-wip-us.apache.org/repos/asf/spark/blob/7bb64aae/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index cf9286e..371c198 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -22,7 +22,7 @@ import javax.annotation.concurrent.GuardedBy
 import scala.collection.mutable
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.{FileSystem, Path}
 
 import org.apache.spark.int

spark git commit: [SPARK-14343][SQL] Proper column pruning for text data source

2016-06-01 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 8fb125bdf -> cb254ecb1


[SPARK-14343][SQL] Proper column pruning for text data source

## What changes were proposed in this pull request?

Text data source ignores requested schema, and may give wrong result when the 
only data column is not requested. This may happen when only partitioning 
column(s) are requested for a partitioned text table.

## How was this patch tested?

New test case added in `TextSuite`.

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

Closes #13431 from liancheng/spark-14343-partitioned-text-table.

(cherry picked from commit 1f43562daf9454428796317203d9dcc9030a46eb)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: cb254ecb1f351c00f7fe4c3c9cc41c46beda90b4
Parents: 8fb125b
Author: Cheng Lian <l...@databricks.com>
Authored: Wed Jun 1 07:30:55 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed Jun 1 07:31:52 2016 -0700

--
 .../datasources/text/TextFileFormat.scala   | 31 +---
 .../execution/datasources/text/TextSuite.scala  | 17 +--
 2 files changed, 35 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cb254ecb/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
index 1e5bce4..9c03ab2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
@@ -92,20 +92,31 @@ class TextFileFormat extends FileFormat with 
DataSourceRegister {
   filters: Seq[Filter],
   options: Map[String, String],
   hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
+assert(
+  requiredSchema.length <= 1,
+  "Text data source only produces a single data column named \"value\".")
+
 val broadcastedHadoopConf =
   sparkSession.sparkContext.broadcast(new 
SerializableConfiguration(hadoopConf))
 
 (file: PartitionedFile) => {
-  val unsafeRow = new UnsafeRow(1)
-  val bufferHolder = new BufferHolder(unsafeRow)
-  val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1)
-
-  new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map { 
line =>
-// Writes to an UnsafeRow directly
-bufferHolder.reset()
-unsafeRowWriter.write(0, line.getBytes, 0, line.getLength)
-unsafeRow.setTotalSize(bufferHolder.totalSize())
-unsafeRow
+  val reader = new HadoopFileLinesReader(file, 
broadcastedHadoopConf.value.value)
+
+  if (requiredSchema.isEmpty) {
+val emptyUnsafeRow = new UnsafeRow(0)
+reader.map(_ => emptyUnsafeRow)
+  } else {
+val unsafeRow = new UnsafeRow(1)
+val bufferHolder = new BufferHolder(unsafeRow)
+val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1)
+
+reader.map { line =>
+  // Writes to an UnsafeRow directly
+  bufferHolder.reset()
+  unsafeRowWriter.write(0, line.getBytes, 0, line.getLength)
+  unsafeRow.setTotalSize(bufferHolder.totalSize())
+  unsafeRow
+}
   }
 }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/cb254ecb/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
index b5e51e9..7b6981f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
@@ -19,9 +19,6 @@ package org.apache.spark.sql.execution.datasources.text
 
 import java.io.File
 
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.SequenceFile.CompressionType
 import org.apache.hadoop.io.compress.GzipCodec
 
@@ -31,6 +28,7 @@ import org.apache.spark.sql.types.{StringType, StructType}
 import

spark git commit: [SPARK-14343][SQL] Proper column pruning for text data source

2016-06-01 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 6563d72b1 -> 1f43562da


[SPARK-14343][SQL] Proper column pruning for text data source

## What changes were proposed in this pull request?

Text data source ignores requested schema, and may give wrong result when the 
only data column is not requested. This may happen when only partitioning 
column(s) are requested for a partitioned text table.

## How was this patch tested?

New test case added in `TextSuite`.

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

Closes #13431 from liancheng/spark-14343-partitioned-text-table.


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

Branch: refs/heads/master
Commit: 1f43562daf9454428796317203d9dcc9030a46eb
Parents: 6563d72
Author: Cheng Lian <l...@databricks.com>
Authored: Wed Jun 1 07:30:55 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed Jun 1 07:30:55 2016 -0700

--
 .../datasources/text/TextFileFormat.scala   | 31 +---
 .../execution/datasources/text/TextSuite.scala  | 17 +--
 2 files changed, 35 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1f43562d/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
index 1e5bce4..9c03ab2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
@@ -92,20 +92,31 @@ class TextFileFormat extends FileFormat with 
DataSourceRegister {
   filters: Seq[Filter],
   options: Map[String, String],
   hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
+assert(
+  requiredSchema.length <= 1,
+  "Text data source only produces a single data column named \"value\".")
+
 val broadcastedHadoopConf =
   sparkSession.sparkContext.broadcast(new 
SerializableConfiguration(hadoopConf))
 
 (file: PartitionedFile) => {
-  val unsafeRow = new UnsafeRow(1)
-  val bufferHolder = new BufferHolder(unsafeRow)
-  val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1)
-
-  new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map { 
line =>
-// Writes to an UnsafeRow directly
-bufferHolder.reset()
-unsafeRowWriter.write(0, line.getBytes, 0, line.getLength)
-unsafeRow.setTotalSize(bufferHolder.totalSize())
-unsafeRow
+  val reader = new HadoopFileLinesReader(file, 
broadcastedHadoopConf.value.value)
+
+  if (requiredSchema.isEmpty) {
+val emptyUnsafeRow = new UnsafeRow(0)
+reader.map(_ => emptyUnsafeRow)
+  } else {
+val unsafeRow = new UnsafeRow(1)
+val bufferHolder = new BufferHolder(unsafeRow)
+val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1)
+
+reader.map { line =>
+  // Writes to an UnsafeRow directly
+  bufferHolder.reset()
+  unsafeRowWriter.write(0, line.getBytes, 0, line.getLength)
+  unsafeRow.setTotalSize(bufferHolder.totalSize())
+  unsafeRow
+}
   }
 }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/1f43562d/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
index b5e51e9..7b6981f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
@@ -19,9 +19,6 @@ package org.apache.spark.sql.execution.datasources.text
 
 import java.io.File
 
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.SequenceFile.CompressionType
 import org.apache.hadoop.io.compress.GzipCodec
 
@@ -31,6 +28,7 @@ import org.apache.spark.sql.types.{StringType, StructType}
 import org.apache.spark.util.Utils
 
 class TextSuite extends QueryTest with SharedSQLContext {
+  import testImplicits._
 
   test("reading text f

spark git commit: [SPARK-15112][SQL] Disables EmbedSerializerInFilter for plan fragments that change schema

2016-05-30 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 be2d23dfd -> 6dae027a6


[SPARK-15112][SQL] Disables EmbedSerializerInFilter for plan fragments that 
change schema

## What changes were proposed in this pull request?

`EmbedSerializerInFilter` implicitly assumes that the plan fragment being 
optimized doesn't change plan schema, which is reasonable because 
`Dataset.filter` should never change the schema.

However, due to another issue involving `DeserializeToObject` and 
`SerializeFromObject`, typed filter *does* change plan schema (see 
[SPARK-15632][1]). This breaks `EmbedSerializerInFilter` and causes corrupted 
data.

This PR disables `EmbedSerializerInFilter` when there's a schema change to 
avoid data corruption. The schema change issue should be addressed in follow-up 
PRs.

## How was this patch tested?

New test case added in `DatasetSuite`.

[1]: https://issues.apache.org/jira/browse/SPARK-15632

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

Closes #13362 from liancheng/spark-15112-corrupted-filter.

(cherry picked from commit 1360a6d636dd812a27955fc85df8e0255db60dfa)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 6dae027a6cdd7c862963f71e1ea08f7f1b4b3506
Parents: be2d23d
Author: Cheng Lian <l...@databricks.com>
Authored: Sun May 29 23:19:12 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Sun May 29 23:19:29 2016 -0700

--
 .../sql/catalyst/optimizer/Optimizer.scala  | 21 +++-
 .../org/apache/spark/sql/DatasetSuite.scala | 16 ++-
 2 files changed, 35 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6dae027a/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 48d7009..688c77d 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
@@ -1597,7 +1597,19 @@ case class GetCurrentDatabase(sessionCatalog: 
SessionCatalog) extends Rule[Logic
  */
 object EmbedSerializerInFilter extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan transform {
-case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject)) 
=>
+case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject))
+  // SPARK-15632: Conceptually, filter operator should never introduce 
schema change. This
+  // optimization rule also relies on this assumption. However, Dataset 
typed filter operator
+  // does introduce schema changes in some cases. Thus, we only enable 
this optimization when
+  //
+  //  1. either input and output schemata are exactly the same, or
+  //  2. both input and output schemata are single-field schema and share 
the same type.
+  //
+  // The 2nd case is included because encoders for primitive types always 
have only a single
+  // field with hard-coded field name "value".
+  // TODO Cleans this up after fixing SPARK-15632.
+  if s.schema == d.child.schema || samePrimitiveType(s.schema, 
d.child.schema) =>
+
   val numObjects = condition.collect {
 case a: Attribute if a == d.output.head => a
   }.length
@@ -1622,6 +1634,13 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] 
{
 Project(objAttrs, filter)
   }
   }
+
+  def samePrimitiveType(lhs: StructType, rhs: StructType): Boolean = {
+(lhs, rhs) match {
+  case (StructType(Array(f1)), StructType(Array(f2))) => f1.dataType == 
f2.dataType
+  case _ => false
+}
+  }
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/6dae027a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index e395007..8fc4dc9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -706,7 +706,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
 val dataset = Seq(1, 2, 3).toDS()
 dataset.crea

spark git commit: [SPARK-15112][SQL] Disables EmbedSerializerInFilter for plan fragments that change schema

2016-05-30 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master ce1572d16 -> 1360a6d63


[SPARK-15112][SQL] Disables EmbedSerializerInFilter for plan fragments that 
change schema

## What changes were proposed in this pull request?

`EmbedSerializerInFilter` implicitly assumes that the plan fragment being 
optimized doesn't change plan schema, which is reasonable because 
`Dataset.filter` should never change the schema.

However, due to another issue involving `DeserializeToObject` and 
`SerializeFromObject`, typed filter *does* change plan schema (see 
[SPARK-15632][1]). This breaks `EmbedSerializerInFilter` and causes corrupted 
data.

This PR disables `EmbedSerializerInFilter` when there's a schema change to 
avoid data corruption. The schema change issue should be addressed in follow-up 
PRs.

## How was this patch tested?

New test case added in `DatasetSuite`.

[1]: https://issues.apache.org/jira/browse/SPARK-15632

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

Closes #13362 from liancheng/spark-15112-corrupted-filter.


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

Branch: refs/heads/master
Commit: 1360a6d636dd812a27955fc85df8e0255db60dfa
Parents: ce1572d
Author: Cheng Lian <l...@databricks.com>
Authored: Sun May 29 23:19:12 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Sun May 29 23:19:12 2016 -0700

--
 .../sql/catalyst/optimizer/Optimizer.scala  | 21 +++-
 .../org/apache/spark/sql/DatasetSuite.scala | 16 ++-
 2 files changed, 35 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1360a6d6/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 48d7009..688c77d 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
@@ -1597,7 +1597,19 @@ case class GetCurrentDatabase(sessionCatalog: 
SessionCatalog) extends Rule[Logic
  */
 object EmbedSerializerInFilter extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan transform {
-case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject)) 
=>
+case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject))
+  // SPARK-15632: Conceptually, filter operator should never introduce 
schema change. This
+  // optimization rule also relies on this assumption. However, Dataset 
typed filter operator
+  // does introduce schema changes in some cases. Thus, we only enable 
this optimization when
+  //
+  //  1. either input and output schemata are exactly the same, or
+  //  2. both input and output schemata are single-field schema and share 
the same type.
+  //
+  // The 2nd case is included because encoders for primitive types always 
have only a single
+  // field with hard-coded field name "value".
+  // TODO Cleans this up after fixing SPARK-15632.
+  if s.schema == d.child.schema || samePrimitiveType(s.schema, 
d.child.schema) =>
+
   val numObjects = condition.collect {
 case a: Attribute if a == d.output.head => a
   }.length
@@ -1622,6 +1634,13 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] 
{
 Project(objAttrs, filter)
   }
   }
+
+  def samePrimitiveType(lhs: StructType, rhs: StructType): Boolean = {
+(lhs, rhs) match {
+  case (StructType(Array(f1)), StructType(Array(f2))) => f1.dataType == 
f2.dataType
+  case _ => false
+}
+  }
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/1360a6d6/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index e395007..8fc4dc9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -706,7 +706,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
 val dataset = Seq(1, 2, 3).toDS()
 dataset.createOrReplaceTempView("tempView")
 
-// Overrrides the existing temporary view with same name
+// Overrides the existing t

spark git commit: [SPARK-9876][SQL] Update Parquet to 1.8.1.

2016-05-27 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 9c137b2e3 -> dc6e94157


[SPARK-9876][SQL] Update Parquet to 1.8.1.

## What changes were proposed in this pull request?

This includes minimal changes to get Spark using the current release of 
Parquet, 1.8.1.

## How was this patch tested?

This uses the existing Parquet tests.

Author: Ryan Blue <b...@apache.org>

Closes #13280 from rdblue/SPARK-9876-update-parquet.

(cherry picked from commit 776d183c82b424ef7c3cae30537d8afe9b9eee83)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: dc6e94157ce08df91aa1a31db8e5ec733a1ab0c5
Parents: 9c137b2
Author: Ryan Blue <b...@apache.org>
Authored: Fri May 27 16:59:38 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri May 27 16:59:50 2016 -0700

--
 dev/deps/spark-deps-hadoop-2.2  | 11 ++-
 dev/deps/spark-deps-hadoop-2.3  | 11 ++-
 dev/deps/spark-deps-hadoop-2.4  | 11 ++-
 dev/deps/spark-deps-hadoop-2.6  | 11 ++-
 dev/deps/spark-deps-hadoop-2.7  | 11 ++-
 pom.xml |  2 +-
 .../SpecificParquetRecordReaderBase.java| 20 +++--
 .../parquet/CatalystReadSupport.scala   | 12 ++-
 .../parquet/CatalystSchemaConverter.scala   | 16 
 .../datasources/parquet/ParquetFilters.scala| 83 
 .../parquet/ParquetSchemaSuite.scala| 20 +++--
 11 files changed, 91 insertions(+), 117 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/dc6e9415/dev/deps/spark-deps-hadoop-2.2
--
diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2
index 578691c..deec033 100644
--- a/dev/deps/spark-deps-hadoop-2.2
+++ b/dev/deps/spark-deps-hadoop-2.2
@@ -129,14 +129,13 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.8.jar
-parquet-column-1.7.0.jar
-parquet-common-1.7.0.jar
-parquet-encoding-1.7.0.jar
+parquet-column-1.8.1.jar
+parquet-common-1.8.1.jar
+parquet-encoding-1.8.1.jar
 parquet-format-2.3.0-incubating.jar
-parquet-generator-1.7.0.jar
-parquet-hadoop-1.7.0.jar
+parquet-hadoop-1.8.1.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.7.0.jar
+parquet-jackson-1.8.1.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/dc6e9415/dev/deps/spark-deps-hadoop-2.3
--
diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3
index fc6306f..43c7dd3 100644
--- a/dev/deps/spark-deps-hadoop-2.3
+++ b/dev/deps/spark-deps-hadoop-2.3
@@ -136,14 +136,13 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.8.jar
-parquet-column-1.7.0.jar
-parquet-common-1.7.0.jar
-parquet-encoding-1.7.0.jar
+parquet-column-1.8.1.jar
+parquet-common-1.8.1.jar
+parquet-encoding-1.8.1.jar
 parquet-format-2.3.0-incubating.jar
-parquet-generator-1.7.0.jar
-parquet-hadoop-1.7.0.jar
+parquet-hadoop-1.8.1.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.7.0.jar
+parquet-jackson-1.8.1.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/dc6e9415/dev/deps/spark-deps-hadoop-2.4
--
diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4
index dee1417..7186b30 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -136,14 +136,13 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.8.jar
-parquet-column-1.7.0.jar
-parquet-common-1.7.0.jar
-parquet-encoding-1.7.0.jar
+parquet-column-1.8.1.jar
+parquet-common-1.8.1.jar
+parquet-encoding-1.8.1.jar
 parquet-format-2.3.0-incubating.jar
-parquet-generator-1.7.0.jar
-parquet-hadoop-1.7.0.jar
+parquet-hadoop-1.8.1.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.7.0.jar
+parquet-jackson-1.8.1.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/dc6e9415/dev/deps/spark-deps-hadoop-2.6
--
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index 9695661..3e4ed74 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -144,14 +144,13 @@ opencsv-2.3.jar
 o

spark git commit: [SPARK-9876][SQL] Update Parquet to 1.8.1.

2016-05-27 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 019afd9c7 -> 776d183c8


[SPARK-9876][SQL] Update Parquet to 1.8.1.

## What changes were proposed in this pull request?

This includes minimal changes to get Spark using the current release of 
Parquet, 1.8.1.

## How was this patch tested?

This uses the existing Parquet tests.

Author: Ryan Blue <b...@apache.org>

Closes #13280 from rdblue/SPARK-9876-update-parquet.


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

Branch: refs/heads/master
Commit: 776d183c82b424ef7c3cae30537d8afe9b9eee83
Parents: 019afd9
Author: Ryan Blue <b...@apache.org>
Authored: Fri May 27 16:59:38 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri May 27 16:59:38 2016 -0700

--
 dev/deps/spark-deps-hadoop-2.2  | 11 ++-
 dev/deps/spark-deps-hadoop-2.3  | 11 ++-
 dev/deps/spark-deps-hadoop-2.4  | 11 ++-
 dev/deps/spark-deps-hadoop-2.6  | 11 ++-
 dev/deps/spark-deps-hadoop-2.7  | 11 ++-
 pom.xml |  2 +-
 .../SpecificParquetRecordReaderBase.java| 20 +++--
 .../parquet/CatalystReadSupport.scala   | 12 ++-
 .../parquet/CatalystSchemaConverter.scala   | 16 
 .../datasources/parquet/ParquetFilters.scala| 83 
 .../parquet/ParquetSchemaSuite.scala| 20 +++--
 11 files changed, 91 insertions(+), 117 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/776d183c/dev/deps/spark-deps-hadoop-2.2
--
diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2
index 578691c..deec033 100644
--- a/dev/deps/spark-deps-hadoop-2.2
+++ b/dev/deps/spark-deps-hadoop-2.2
@@ -129,14 +129,13 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.8.jar
-parquet-column-1.7.0.jar
-parquet-common-1.7.0.jar
-parquet-encoding-1.7.0.jar
+parquet-column-1.8.1.jar
+parquet-common-1.8.1.jar
+parquet-encoding-1.8.1.jar
 parquet-format-2.3.0-incubating.jar
-parquet-generator-1.7.0.jar
-parquet-hadoop-1.7.0.jar
+parquet-hadoop-1.8.1.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.7.0.jar
+parquet-jackson-1.8.1.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/776d183c/dev/deps/spark-deps-hadoop-2.3
--
diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3
index fc6306f..43c7dd3 100644
--- a/dev/deps/spark-deps-hadoop-2.3
+++ b/dev/deps/spark-deps-hadoop-2.3
@@ -136,14 +136,13 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.8.jar
-parquet-column-1.7.0.jar
-parquet-common-1.7.0.jar
-parquet-encoding-1.7.0.jar
+parquet-column-1.8.1.jar
+parquet-common-1.8.1.jar
+parquet-encoding-1.8.1.jar
 parquet-format-2.3.0-incubating.jar
-parquet-generator-1.7.0.jar
-parquet-hadoop-1.7.0.jar
+parquet-hadoop-1.8.1.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.7.0.jar
+parquet-jackson-1.8.1.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/776d183c/dev/deps/spark-deps-hadoop-2.4
--
diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4
index dee1417..7186b30 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -136,14 +136,13 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.8.jar
-parquet-column-1.7.0.jar
-parquet-common-1.7.0.jar
-parquet-encoding-1.7.0.jar
+parquet-column-1.8.1.jar
+parquet-common-1.8.1.jar
+parquet-encoding-1.8.1.jar
 parquet-format-2.3.0-incubating.jar
-parquet-generator-1.7.0.jar
-parquet-hadoop-1.7.0.jar
+parquet-hadoop-1.8.1.jar
 parquet-hadoop-bundle-1.6.0.jar
-parquet-jackson-1.7.0.jar
+parquet-jackson-1.8.1.jar
 pmml-model-1.2.15.jar
 pmml-schema-1.2.15.jar
 protobuf-java-2.5.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/776d183c/dev/deps/spark-deps-hadoop-2.6
--
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index 9695661..3e4ed74 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -144,14 +144,13 @@ opencsv-2.3.jar
 oro-2.0.8.jar
 osgi-resource-locator-1.0.1.jar
 paranamer-2.8.jar
-parquet-column-1.7.0.jar
-parquet-common-1.7.0.jar
-parquet-encodi

spark git commit: [SPARK-15550][SQL] Dataset.show() should show contents nested products as rows

2016-05-26 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 29681cca1 -> 4939c2a12


[SPARK-15550][SQL] Dataset.show() should show contents nested products as rows

## What changes were proposed in this pull request?

This PR addresses two related issues:

1. `Dataset.showString()` should show case classes/Java beans at all levels as 
rows, while master code only handles top level ones.

2. `Dataset.showString()` should show full contents produced the underlying 
query plan

   Dataset is only a view of the underlying query plan. Columns not referred by 
the encoder are still reachable using methods like `Dataset.col`. So it 
probably makes more sense to show full contents of the query plan.

## How was this patch tested?

Two new test cases are added in `DatasetSuite` to check `.showString()` output.

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

Closes #13331 from liancheng/spark-15550-ds-show.

(cherry picked from commit e7082caeb4a53c1ee172d136894eece1ac880f65)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 4939c2a12c854a87a020a7e759e4f87810f16710
Parents: 29681cc
Author: Cheng Lian <l...@databricks.com>
Authored: Thu May 26 16:23:48 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu May 26 16:23:56 2016 -0700

--
 .../scala/org/apache/spark/sql/Dataset.scala| 10 +--
 .../org/apache/spark/sql/DatasetSuite.scala | 68 ++--
 2 files changed, 52 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4939c2a1/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 961ae32..85f0cf8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -237,19 +237,13 @@ class Dataset[T] private[sql](
*/
   private[sql] def showString(_numRows: Int, truncate: Boolean = true): String 
= {
 val numRows = _numRows.max(0)
-val takeResult = take(numRows + 1)
+val takeResult = toDF().take(numRows + 1)
 val hasMoreData = takeResult.length > numRows
 val data = takeResult.take(numRows)
 
 // For array values, replace Seq and Array with square brackets
 // For cells that are beyond 20 characters, replace it with the first 17 
and "..."
-val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map {
-  case r: Row => r
-  case tuple: Product => Row.fromTuple(tuple)
-  case definedByCtor: DefinedByConstructorParams =>
-
Row.fromSeq(ScalaReflection.getConstructorParameterValues(definedByCtor))
-  case o => Row(o)
-}.map { row =>
+val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map { row =>
   row.toSeq.map { cell =>
 val str = cell match {
   case null => "null"

http://git-wip-us.apache.org/repos/asf/spark/blob/4939c2a1/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 05de79e..32320a6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -22,9 +22,8 @@ import java.sql.{Date, Timestamp}
 
 import scala.language.postfixOps
 
-import org.scalatest.words.MatcherWords.be
-
 import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder}
+import org.apache.spark.sql.catalyst.util.sideBySide
 import org.apache.spark.sql.execution.streaming.MemoryStream
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.test.SharedSQLContext
@@ -217,7 +216,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
 val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
 checkDataset(
   ds.filter(_._1 == "b").select(expr("_1").as[String]),
-  ("b"))
+  "b")
   }
 
   test("foreach") {
@@ -436,20 +435,6 @@ class DatasetSuite extends QueryTest with SharedSQLContext 
{
 assert(ds.toString == "[_1: int, _2: int]")
   }
 
-  test("showString: Kryo encoder") {
-implicit val k

spark git commit: [SPARK-15550][SQL] Dataset.show() should show contents nested products as rows

2016-05-26 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master fe6de16f7 -> e7082caeb


[SPARK-15550][SQL] Dataset.show() should show contents nested products as rows

## What changes were proposed in this pull request?

This PR addresses two related issues:

1. `Dataset.showString()` should show case classes/Java beans at all levels as 
rows, while master code only handles top level ones.

2. `Dataset.showString()` should show full contents produced the underlying 
query plan

   Dataset is only a view of the underlying query plan. Columns not referred by 
the encoder are still reachable using methods like `Dataset.col`. So it 
probably makes more sense to show full contents of the query plan.

## How was this patch tested?

Two new test cases are added in `DatasetSuite` to check `.showString()` output.

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

Closes #13331 from liancheng/spark-15550-ds-show.


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

Branch: refs/heads/master
Commit: e7082caeb4a53c1ee172d136894eece1ac880f65
Parents: fe6de16
Author: Cheng Lian <l...@databricks.com>
Authored: Thu May 26 16:23:48 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu May 26 16:23:48 2016 -0700

--
 .../scala/org/apache/spark/sql/Dataset.scala| 10 +--
 .../org/apache/spark/sql/DatasetSuite.scala | 68 ++--
 2 files changed, 52 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e7082cae/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 961ae32..85f0cf8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -237,19 +237,13 @@ class Dataset[T] private[sql](
*/
   private[sql] def showString(_numRows: Int, truncate: Boolean = true): String 
= {
 val numRows = _numRows.max(0)
-val takeResult = take(numRows + 1)
+val takeResult = toDF().take(numRows + 1)
 val hasMoreData = takeResult.length > numRows
 val data = takeResult.take(numRows)
 
 // For array values, replace Seq and Array with square brackets
 // For cells that are beyond 20 characters, replace it with the first 17 
and "..."
-val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map {
-  case r: Row => r
-  case tuple: Product => Row.fromTuple(tuple)
-  case definedByCtor: DefinedByConstructorParams =>
-
Row.fromSeq(ScalaReflection.getConstructorParameterValues(definedByCtor))
-  case o => Row(o)
-}.map { row =>
+val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map { row =>
   row.toSeq.map { cell =>
 val str = cell match {
   case null => "null"

http://git-wip-us.apache.org/repos/asf/spark/blob/e7082cae/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 05de79e..32320a6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -22,9 +22,8 @@ import java.sql.{Date, Timestamp}
 
 import scala.language.postfixOps
 
-import org.scalatest.words.MatcherWords.be
-
 import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder}
+import org.apache.spark.sql.catalyst.util.sideBySide
 import org.apache.spark.sql.execution.streaming.MemoryStream
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.test.SharedSQLContext
@@ -217,7 +216,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
 val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
 checkDataset(
   ds.filter(_._1 == "b").select(expr("_1").as[String]),
-  ("b"))
+  "b")
   }
 
   test("foreach") {
@@ -436,20 +435,6 @@ class DatasetSuite extends QueryTest with SharedSQLContext 
{
 assert(ds.toString == "[_1: int, _2: int]")
   }
 
-  test("showString: Kryo encoder") {
-implicit val kryoEncoder = Encoders.kryo[KryoData]
-val ds = Seq(KryoData(1), KryoData(2)).toDS()
-
-val expectedAnswer = """

spark git commit: [SPARK-15498][TESTS] fix slow tests

2016-05-24 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 e13cfd6d2 -> fb7b90f61


[SPARK-15498][TESTS] fix slow tests

## What changes were proposed in this pull request?

This PR fixes 3 slow tests:

1. `ParquetQuerySuite.read/write wide table`: This is not a good unit test as 
it runs more than 5 minutes. This PR removes it and add a new regression test 
in `CodeGenerationSuite`, which is more "unit".
2. `ParquetQuerySuite.returning batch for wide table`: reduce the threshold and 
use smaller data size.
3. `DatasetSuite.SPARK-14554: Dataset.map may generate wrong java code for wide 
table`: Improve `CodeFormatter.format`(introduced at 
https://github.com/apache/spark/pull/12979) can dramatically speed this it up.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13273 from cloud-fan/test.

(cherry picked from commit 50b660d725269dc0c11e0d350ddd7fc8b19539a0)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: fb7b90f61a7390b5ddb182413f628530cc72bee0
Parents: e13cfd6
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Tue May 24 21:23:39 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue May 24 21:23:49 2016 -0700

--
 .../expressions/codegen/CodeFormatter.scala |  48 +
 .../expressions/codegen/CodeGenerator.scala |  22 ++--
 .../expressions/CodeGenerationSuite.scala   |  14 +++
 .../codegen/CodeFormatterSuite.scala| 107 +--
 .../datasources/parquet/ParquetQuerySuite.scala |  16 +--
 5 files changed, 128 insertions(+), 79 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fb7b90f6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
index 855ae64..05b7c96 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions.codegen
 
-import org.apache.commons.lang3.StringUtils
+import java.util.regex.Matcher
 
 /**
  * An utility class that indents a block of code based on the curly braces and 
parentheses.
@@ -26,13 +26,17 @@ import org.apache.commons.lang3.StringUtils
  * Written by Matei Zaharia.
  */
 object CodeFormatter {
+  val commentHolder = """\/\*(.+?)\*\/""".r
+
   def format(code: CodeAndComment): String = {
-new CodeFormatter().addLines(
-  StringUtils.replaceEach(
-code.body,
-code.comment.keys.toArray,
-code.comment.values.toArray)
-).result
+val formatter = new CodeFormatter
+code.body.split("\n").foreach { line =>
+  val commentReplaced = commentHolder.replaceAllIn(
+line.trim,
+m => 
code.comment.get(m.group(1)).map(Matcher.quoteReplacement).getOrElse(m.group(0)))
+  formatter.addLine(commentReplaced)
+}
+formatter.result()
   }
 
   def stripExtraNewLines(input: String): String = {
@@ -53,16 +57,28 @@ object CodeFormatter {
   def stripOverlappingComments(codeAndComment: CodeAndComment): CodeAndComment 
= {
 val code = new StringBuilder
 val map = codeAndComment.comment
+
+def getComment(line: String): Option[String] = {
+  if (line.startsWith("/*") && line.endsWith("*/")) {
+map.get(line.substring(2, line.length - 2))
+  } else {
+None
+  }
+}
+
 var lastLine: String = "dummy"
 codeAndComment.body.split('\n').foreach { l =>
   val line = l.trim()
-  val skip = lastLine.startsWith("/*") && lastLine.endsWith("*/") &&
-line.startsWith("/*") && line.endsWith("*/") &&
-map(lastLine).substring(3).contains(map(line).substring(3))
+
+  val skip = getComment(lastLine).zip(getComment(line)).exists {
+case (lastComment, currentComment) =>
+  lastComment.substring(3).contains(currentComment.substring(3))
+  }
+
   if (!skip) {
-code.append(line)
-code.append("\n")
+code.append(line).appen

spark git commit: [SPARK-15498][TESTS] fix slow tests

2016-05-24 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 4acababca -> 50b660d72


[SPARK-15498][TESTS] fix slow tests

## What changes were proposed in this pull request?

This PR fixes 3 slow tests:

1. `ParquetQuerySuite.read/write wide table`: This is not a good unit test as 
it runs more than 5 minutes. This PR removes it and add a new regression test 
in `CodeGenerationSuite`, which is more "unit".
2. `ParquetQuerySuite.returning batch for wide table`: reduce the threshold and 
use smaller data size.
3. `DatasetSuite.SPARK-14554: Dataset.map may generate wrong java code for wide 
table`: Improve `CodeFormatter.format`(introduced at 
https://github.com/apache/spark/pull/12979) can dramatically speed this it up.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenc...@databricks.com>

Closes #13273 from cloud-fan/test.


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

Branch: refs/heads/master
Commit: 50b660d725269dc0c11e0d350ddd7fc8b19539a0
Parents: 4acabab
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Tue May 24 21:23:39 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue May 24 21:23:39 2016 -0700

--
 .../expressions/codegen/CodeFormatter.scala |  48 +
 .../expressions/codegen/CodeGenerator.scala |  22 ++--
 .../expressions/CodeGenerationSuite.scala   |  14 +++
 .../codegen/CodeFormatterSuite.scala| 107 +--
 .../datasources/parquet/ParquetQuerySuite.scala |  16 +--
 5 files changed, 128 insertions(+), 79 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/50b660d7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
index 855ae64..05b7c96 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions.codegen
 
-import org.apache.commons.lang3.StringUtils
+import java.util.regex.Matcher
 
 /**
  * An utility class that indents a block of code based on the curly braces and 
parentheses.
@@ -26,13 +26,17 @@ import org.apache.commons.lang3.StringUtils
  * Written by Matei Zaharia.
  */
 object CodeFormatter {
+  val commentHolder = """\/\*(.+?)\*\/""".r
+
   def format(code: CodeAndComment): String = {
-new CodeFormatter().addLines(
-  StringUtils.replaceEach(
-code.body,
-code.comment.keys.toArray,
-code.comment.values.toArray)
-).result
+val formatter = new CodeFormatter
+code.body.split("\n").foreach { line =>
+  val commentReplaced = commentHolder.replaceAllIn(
+line.trim,
+m => 
code.comment.get(m.group(1)).map(Matcher.quoteReplacement).getOrElse(m.group(0)))
+  formatter.addLine(commentReplaced)
+}
+formatter.result()
   }
 
   def stripExtraNewLines(input: String): String = {
@@ -53,16 +57,28 @@ object CodeFormatter {
   def stripOverlappingComments(codeAndComment: CodeAndComment): CodeAndComment 
= {
 val code = new StringBuilder
 val map = codeAndComment.comment
+
+def getComment(line: String): Option[String] = {
+  if (line.startsWith("/*") && line.endsWith("*/")) {
+map.get(line.substring(2, line.length - 2))
+  } else {
+None
+  }
+}
+
 var lastLine: String = "dummy"
 codeAndComment.body.split('\n').foreach { l =>
   val line = l.trim()
-  val skip = lastLine.startsWith("/*") && lastLine.endsWith("*/") &&
-line.startsWith("/*") && line.endsWith("*/") &&
-map(lastLine).substring(3).contains(map(line).substring(3))
+
+  val skip = getComment(lastLine).zip(getComment(line)).exists {
+case (lastComment, currentComment) =>
+  lastComment.substring(3).contains(currentComment.substring(3))
+  }
+
   if (!skip) {
-code.append(line)
-code.append("\n")
+code.append(line).append("\n")
   }
+
   lastLine = line
 }
 new CodeAndComment(code.result().trim(), map)
@@ -117,8 +133,9 @@ private class Co

spark git commit: [SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively

2016-05-23 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 ca271c792 -> 3def56120


[SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively

## What changes were proposed in this pull request?
Currently command `ADD FILE|JAR ` is supported natively in 
SparkSQL. However, when this command is run, the file/jar is added to the 
resources that can not be looked up by `LIST FILE(s)|JAR(s)` command because 
the `LIST` command is passed to Hive command processor in Spark-SQL or simply 
not supported in Spark-shell. There is no way users can find out what 
files/jars are added to the spark context.
Refer to [Hive 
commands](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli)

This PR is to support following commands:
`LIST (FILE[s] [filepath ...] | JAR[s] [jarfile ...])`

### For example:
# LIST FILE(s)
```
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt")
res1: org.apache.spark.sql.DataFrame = []
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt")
res2: org.apache.spark.sql.DataFrame = []

scala> spark.sql("list file 
hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt").show(false)
+--+
|result|
+--+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
+--+

scala> spark.sql("list files").show(false)
+--+
|result|
+--+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt |
+--+
```

# LIST JAR(s)
```
scala> spark.sql("add jar 
/Users/xinwu/spark/core/src/test/resources/TestUDTF.jar")
res9: org.apache.spark.sql.DataFrame = [result: int]

scala> spark.sql("list jar TestUDTF.jar").show(false)
+-+
|result   |
+-+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+-+

scala> spark.sql("list jars").show(false)
+-+
|result   |
+-+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+-+
```
## How was this patch tested?
New test cases are added for Spark-SQL, Spark-Shell and SparkContext API code 
path.

Author: Xin Wu <xi...@us.ibm.com>
Author: xin Wu <xi...@us.ibm.com>

Closes #13212 from xwu0226/list_command.

(cherry picked from commit 01659bc50cd3d53815d205d005c3678e714c08e0)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 3def56120e0a57238c95f64e7bb84f973b22c065
Parents: ca271c7
Author: Xin Wu <xi...@us.ibm.com>
Authored: Mon May 23 17:32:01 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon May 23 17:32:15 2016 -0700

--
 .../scala/org/apache/spark/SparkContext.scala   |  10 
 core/src/test/resources/TestUDTF.jar| Bin 0 -> 1328 bytes
 .../org/apache/spark/SparkContextSuite.scala|  14 -
 .../apache/spark/sql/catalyst/parser/SqlBase.g4 |   5 +-
 .../spark/sql/execution/SparkSqlParser.scala|  39 +++--
 .../spark/sql/execution/command/resources.scala |  57 ++-
 .../hive/thriftserver/SparkSQLCLIDriver.scala   |  10 ++--
 .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes
 .../spark/sql/hive/thriftserver/CliSuite.scala  |  19 +++
 .../sql/hive/execution/HiveQuerySuite.scala |  12 
 10 files changed, 149 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3def5612/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index e6cdd0d..351024b 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -1387,6 +1387,11 @@ class SparkContext(config: SparkConf) extends Logging 
with ExecutorAllocationCli
   }
 
   /**
+   * Returns a list of file paths that are added to resources.
+   */

spark git commit: [SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively

2016-05-23 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master a8e97d17b -> 01659bc50


[SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively

## What changes were proposed in this pull request?
Currently command `ADD FILE|JAR ` is supported natively in 
SparkSQL. However, when this command is run, the file/jar is added to the 
resources that can not be looked up by `LIST FILE(s)|JAR(s)` command because 
the `LIST` command is passed to Hive command processor in Spark-SQL or simply 
not supported in Spark-shell. There is no way users can find out what 
files/jars are added to the spark context.
Refer to [Hive 
commands](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli)

This PR is to support following commands:
`LIST (FILE[s] [filepath ...] | JAR[s] [jarfile ...])`

### For example:
# LIST FILE(s)
```
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt")
res1: org.apache.spark.sql.DataFrame = []
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt")
res2: org.apache.spark.sql.DataFrame = []

scala> spark.sql("list file 
hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt").show(false)
+--+
|result|
+--+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
+--+

scala> spark.sql("list files").show(false)
+--+
|result|
+--+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt |
+--+
```

# LIST JAR(s)
```
scala> spark.sql("add jar 
/Users/xinwu/spark/core/src/test/resources/TestUDTF.jar")
res9: org.apache.spark.sql.DataFrame = [result: int]

scala> spark.sql("list jar TestUDTF.jar").show(false)
+-+
|result   |
+-+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+-+

scala> spark.sql("list jars").show(false)
+-+
|result   |
+-+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+-+
```
## How was this patch tested?
New test cases are added for Spark-SQL, Spark-Shell and SparkContext API code 
path.

Author: Xin Wu <xi...@us.ibm.com>
Author: xin Wu <xi...@us.ibm.com>

Closes #13212 from xwu0226/list_command.


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

Branch: refs/heads/master
Commit: 01659bc50cd3d53815d205d005c3678e714c08e0
Parents: a8e97d1
Author: Xin Wu <xi...@us.ibm.com>
Authored: Mon May 23 17:32:01 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon May 23 17:32:01 2016 -0700

--
 .../scala/org/apache/spark/SparkContext.scala   |  10 
 core/src/test/resources/TestUDTF.jar| Bin 0 -> 1328 bytes
 .../org/apache/spark/SparkContextSuite.scala|  14 -
 .../apache/spark/sql/catalyst/parser/SqlBase.g4 |   5 +-
 .../spark/sql/execution/SparkSqlParser.scala|  39 +++--
 .../spark/sql/execution/command/resources.scala |  57 ++-
 .../hive/thriftserver/SparkSQLCLIDriver.scala   |  10 ++--
 .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes
 .../spark/sql/hive/thriftserver/CliSuite.scala  |  19 +++
 .../sql/hive/execution/HiveQuerySuite.scala |  12 
 10 files changed, 149 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/01659bc5/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index e6cdd0d..351024b 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -1387,6 +1387,11 @@ class SparkContext(config: SparkConf) extends Logging 
with ExecutorAllocationCli
   }
 
   /**
+   * Returns a list of file paths that are added to resources.
+   */
+  def listFiles(): Seq[String] = addedFiles.keySet.toSeq
+
+  /**
* Add a file to be downloaded with this Spark job on 

spark git commit: [SPARK-14031][SQL] speedup CSV writer

2016-05-23 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 4462da707 -> 6eb8ec6f4


[SPARK-14031][SQL] speedup CSV writer

## What changes were proposed in this pull request?

Currently, we create an CSVWriter for every row, it's very expensive and memory 
hungry, took about 15 seconds to write out 1 mm rows (two columns).

This PR will write the rows in batch mode, create a CSVWriter for every 1k 
rows, which could write out 1 mm rows in about 1 seconds (15X faster).

## How was this patch tested?

Manually benchmark it.

Author: Davies Liu <dav...@databricks.com>

Closes #13229 from davies/csv_writer.

(cherry picked from commit 80091b8a6840b562cf76341926e5b828d4def7e2)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 6eb8ec6f41d7e460ba7f706590d2da62c045aead
Parents: 4462da7
Author: Davies Liu <dav...@databricks.com>
Authored: Mon May 23 10:48:25 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon May 23 10:48:39 2016 -0700

--
 .../execution/datasources/csv/CSVParser.scala   | 19 +++-
 .../execution/datasources/csv/CSVRelation.scala | 23 +---
 2 files changed, 29 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6eb8ec6f/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
index ae797a1..111995d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
@@ -76,17 +76,26 @@ private[sql] class LineCsvWriter(params: CSVOptions, 
headers: Seq[String]) exten
   writerSettings.setQuoteAllFields(false)
   writerSettings.setHeaders(headers: _*)
 
-  def writeRow(row: Seq[String], includeHeader: Boolean): String = {
-val buffer = new ByteArrayOutputStream()
-val outputWriter = new OutputStreamWriter(buffer, StandardCharsets.UTF_8)
-val writer = new CsvWriter(outputWriter, writerSettings)
+  private var buffer = new ByteArrayOutputStream()
+  private var writer = new CsvWriter(
+new OutputStreamWriter(buffer, StandardCharsets.UTF_8),
+writerSettings)
 
+  def writeRow(row: Seq[String], includeHeader: Boolean): Unit = {
 if (includeHeader) {
   writer.writeHeaders()
 }
 writer.writeRow(row.toArray: _*)
+  }
+
+  def flush(): String = {
 writer.close()
-buffer.toString.stripLineEnd
+val lines = buffer.toString.stripLineEnd
+buffer = new ByteArrayOutputStream()
+writer = new CsvWriter(
+  new OutputStreamWriter(buffer, StandardCharsets.UTF_8),
+  writerSettings)
+lines
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/6eb8ec6f/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
index 4f2d438..9849484 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
@@ -176,8 +176,8 @@ private[sql] class CsvOutputWriter(
 }.getRecordWriter(context)
   }
 
-  private var firstRow: Boolean = params.headerFlag
-
+  private val FLUSH_BATCH_SIZE = 1024L
+  private var records: Long = 0L
   private val csvWriter = new LineCsvWriter(params, 
dataSchema.fieldNames.toSeq)
 
   private def rowToString(row: Seq[Any]): Seq[String] = row.map { field =>
@@ -191,16 +191,23 @@ private[sql] class CsvOutputWriter(
   override def write(row: Row): Unit = throw new 
UnsupportedOperationException("call writeInternal")
 
   override protected[sql] def writeInternal(row: InternalRow): Unit = {
-// TODO: Instead of converting and writing every row, we should use the 
univocity buffer
-val resultString = csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), 
firstRow)
-if (firstRow) {
-  firstRow = false
+csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), records == 0L && 
params.headerFlag)
+records += 1
+if 

spark git commit: [SPARK-14031][SQL] speedup CSV writer

2016-05-23 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master dafcb05c2 -> 80091b8a6


[SPARK-14031][SQL] speedup CSV writer

## What changes were proposed in this pull request?

Currently, we create an CSVWriter for every row, it's very expensive and memory 
hungry, took about 15 seconds to write out 1 mm rows (two columns).

This PR will write the rows in batch mode, create a CSVWriter for every 1k 
rows, which could write out 1 mm rows in about 1 seconds (15X faster).

## How was this patch tested?

Manually benchmark it.

Author: Davies Liu <dav...@databricks.com>

Closes #13229 from davies/csv_writer.


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

Branch: refs/heads/master
Commit: 80091b8a6840b562cf76341926e5b828d4def7e2
Parents: dafcb05
Author: Davies Liu <dav...@databricks.com>
Authored: Mon May 23 10:48:25 2016 -0700
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon May 23 10:48:25 2016 -0700

--
 .../execution/datasources/csv/CSVParser.scala   | 19 +++-
 .../execution/datasources/csv/CSVRelation.scala | 23 +---
 2 files changed, 29 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/80091b8a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
index ae797a1..111995d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
@@ -76,17 +76,26 @@ private[sql] class LineCsvWriter(params: CSVOptions, 
headers: Seq[String]) exten
   writerSettings.setQuoteAllFields(false)
   writerSettings.setHeaders(headers: _*)
 
-  def writeRow(row: Seq[String], includeHeader: Boolean): String = {
-val buffer = new ByteArrayOutputStream()
-val outputWriter = new OutputStreamWriter(buffer, StandardCharsets.UTF_8)
-val writer = new CsvWriter(outputWriter, writerSettings)
+  private var buffer = new ByteArrayOutputStream()
+  private var writer = new CsvWriter(
+new OutputStreamWriter(buffer, StandardCharsets.UTF_8),
+writerSettings)
 
+  def writeRow(row: Seq[String], includeHeader: Boolean): Unit = {
 if (includeHeader) {
   writer.writeHeaders()
 }
 writer.writeRow(row.toArray: _*)
+  }
+
+  def flush(): String = {
 writer.close()
-buffer.toString.stripLineEnd
+val lines = buffer.toString.stripLineEnd
+buffer = new ByteArrayOutputStream()
+writer = new CsvWriter(
+  new OutputStreamWriter(buffer, StandardCharsets.UTF_8),
+  writerSettings)
+lines
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/80091b8a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
index 4f2d438..9849484 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
@@ -176,8 +176,8 @@ private[sql] class CsvOutputWriter(
 }.getRecordWriter(context)
   }
 
-  private var firstRow: Boolean = params.headerFlag
-
+  private val FLUSH_BATCH_SIZE = 1024L
+  private var records: Long = 0L
   private val csvWriter = new LineCsvWriter(params, 
dataSchema.fieldNames.toSeq)
 
   private def rowToString(row: Seq[Any]): Seq[String] = row.map { field =>
@@ -191,16 +191,23 @@ private[sql] class CsvOutputWriter(
   override def write(row: Row): Unit = throw new 
UnsupportedOperationException("call writeInternal")
 
   override protected[sql] def writeInternal(row: InternalRow): Unit = {
-// TODO: Instead of converting and writing every row, we should use the 
univocity buffer
-val resultString = csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), 
firstRow)
-if (firstRow) {
-  firstRow = false
+csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), records == 0L && 
params.headerFlag)
+records += 1
+if (records % FLUSH_BATCH_SIZE == 0) {
+  flush()
+}
+  }
+
+  private def flush(): Unit = {
+val lines = csvWriter.flush()
+i

spark git commit: [SPARK-15334][SQL][HOTFIX] Fixes compilation error for Scala 2.10

2016-05-18 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 14751cd0d -> a122a3e70


[SPARK-15334][SQL][HOTFIX] Fixes compilation error for Scala 2.10

## What changes were proposed in this pull request?

This PR fixes a Scala 2.10 compilation failure introduced in PR #13127.

## How was this patch tested?

Jenkins build.

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

Closes #13166 from liancheng/hotfix-for-scala-2.10.

(cherry picked from commit c4a45fd855252f0f7130cdcc08486591fadab7df)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: a122a3e709479a6510d97dfa81f3d6219bb79de0
Parents: 14751cd
Author: Cheng Lian <l...@databricks.com>
Authored: Wed May 18 18:58:24 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 18 18:58:54 2016 +0800

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


http://git-wip-us.apache.org/repos/asf/spark/blob/a122a3e7/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
index 57e8db7..d46c4e7 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
@@ -137,7 +137,7 @@ class VersionsSuite extends SparkFunSuite with Logging {
   hiveMetastoreVersion = version,
   hadoopVersion = VersionInfo.getVersion,
   sparkConf = sparkConf,
-  hadoopConf,
+  hadoopConf = hadoopConf,
   config = buildConf(),
   ivyPath = ivyPath).createClient()
 }


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



spark git commit: [SPARK-15334][SQL][HOTFIX] Fixes compilation error for Scala 2.10

2016-05-18 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master d2f81df1b -> c4a45fd85


[SPARK-15334][SQL][HOTFIX] Fixes compilation error for Scala 2.10

## What changes were proposed in this pull request?

This PR fixes a Scala 2.10 compilation failure introduced in PR #13127.

## How was this patch tested?

Jenkins build.

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

Closes #13166 from liancheng/hotfix-for-scala-2.10.


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

Branch: refs/heads/master
Commit: c4a45fd855252f0f7130cdcc08486591fadab7df
Parents: d2f81df
Author: Cheng Lian <l...@databricks.com>
Authored: Wed May 18 18:58:24 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 18 18:58:24 2016 +0800

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


http://git-wip-us.apache.org/repos/asf/spark/blob/c4a45fd8/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
index 57e8db7..d46c4e7 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
@@ -137,7 +137,7 @@ class VersionsSuite extends SparkFunSuite with Logging {
   hiveMetastoreVersion = version,
   hadoopVersion = VersionInfo.getVersion,
   sparkConf = sparkConf,
-  hadoopConf,
+  hadoopConf = hadoopConf,
   config = buildConf(),
   ivyPath = ivyPath).createClient()
 }


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



spark git commit: [SPARK-15307][SQL] speed up listing files for data source

2016-05-18 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 c66da74b1 -> 35c25beac


[SPARK-15307][SQL] speed up listing files for data source

## What changes were proposed in this pull request?

Currently, listing files is very slow if there is thousands files, especially 
on local file system, because:
1) FileStatus.getPermission() is very slow on local file system, which is 
launch a subprocess and parse the stdout.
2) Create an JobConf is very expensive (ClassUtil.findContainingJar() is slow).

This PR improve these by:
1) Use another constructor of LocatedFileStatus to avoid calling 
FileStatus.getPermission, the permissions are not used for data sources.
2) Only create an JobConf once within one task.

## How was this patch tested?

Manually tests on a partitioned table with 1828 partitions, decrease the time 
to load the table from 22 seconds to 1.6 seconds (Most of time are spent in 
merging schema now).

Author: Davies Liu <dav...@databricks.com>

Closes #13094 from davies/listing.

(cherry picked from commit 33814f887aea339c99e14ce7f14ca6fcc6875015)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 35c25beaca3e9070ccb6dcca33c415a26e6f1f1c
Parents: c66da74
Author: Davies Liu <dav...@databricks.com>
Authored: Wed May 18 18:46:57 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 18 18:47:19 2016 +0800

--
 .../datasources/ListingFileCatalog.scala|  9 ++---
 .../datasources/fileSourceInterfaces.scala  | 38 ++--
 2 files changed, 33 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/35c25bea/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
index 5cee2b9..644e5d6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
@@ -77,12 +77,12 @@ class ListingFileCatalog(
 if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
   HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession.sparkContext)
 } else {
+  // Dummy jobconf to get to the pathFilter defined in configuration
+  val jobConf = new JobConf(hadoopConf, this.getClass)
+  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
   val statuses: Seq[FileStatus] = paths.flatMap { path =>
 val fs = path.getFileSystem(hadoopConf)
 logInfo(s"Listing $path on driver")
-// Dummy jobconf to get to the pathFilter defined in configuration
-val jobConf = new JobConf(hadoopConf, this.getClass)
-val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
 
 val statuses = {
   val stats = 
Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus])
@@ -101,7 +101,8 @@ class ListingFileCatalog(
   // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should a
   //   a big deal since we always use to `listLeafFilesInParallel` 
when the number of paths
   //   exceeds threshold.
-  case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, 
f.getLen))
+  case f =>
+HadoopFsRelation.createLocatedFileStatus(f, 
fs.getFileBlockLocations(f, 0, f.getLen))
 }
   }.filterNot { status =>
 val name = status.getPath.getName

http://git-wip-us.apache.org/repos/asf/spark/blob/35c25bea/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
index b516297..8d332df 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
@@ -348,28 +348,40 @@ private[sql] object HadoopFsRelation extends Logging {
 pathName == "_SUCCESS" || pathName == "

spark git commit: [SPARK-15307][SQL] speed up listing files for data source

2016-05-18 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 6e02aec44 -> 33814f887


[SPARK-15307][SQL] speed up listing files for data source

## What changes were proposed in this pull request?

Currently, listing files is very slow if there is thousands files, especially 
on local file system, because:
1) FileStatus.getPermission() is very slow on local file system, which is 
launch a subprocess and parse the stdout.
2) Create an JobConf is very expensive (ClassUtil.findContainingJar() is slow).

This PR improve these by:
1) Use another constructor of LocatedFileStatus to avoid calling 
FileStatus.getPermission, the permissions are not used for data sources.
2) Only create an JobConf once within one task.

## How was this patch tested?

Manually tests on a partitioned table with 1828 partitions, decrease the time 
to load the table from 22 seconds to 1.6 seconds (Most of time are spent in 
merging schema now).

Author: Davies Liu <dav...@databricks.com>

Closes #13094 from davies/listing.


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

Branch: refs/heads/master
Commit: 33814f887aea339c99e14ce7f14ca6fcc6875015
Parents: 6e02aec
Author: Davies Liu <dav...@databricks.com>
Authored: Wed May 18 18:46:57 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 18 18:46:57 2016 +0800

--
 .../datasources/ListingFileCatalog.scala|  9 ++---
 .../datasources/fileSourceInterfaces.scala  | 38 ++--
 2 files changed, 33 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/33814f88/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
index 5cee2b9..644e5d6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala
@@ -77,12 +77,12 @@ class ListingFileCatalog(
 if (paths.length >= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
   HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, 
sparkSession.sparkContext)
 } else {
+  // Dummy jobconf to get to the pathFilter defined in configuration
+  val jobConf = new JobConf(hadoopConf, this.getClass)
+  val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
   val statuses: Seq[FileStatus] = paths.flatMap { path =>
 val fs = path.getFileSystem(hadoopConf)
 logInfo(s"Listing $path on driver")
-// Dummy jobconf to get to the pathFilter defined in configuration
-val jobConf = new JobConf(hadoopConf, this.getClass)
-val pathFilter = FileInputFormat.getInputPathFilter(jobConf)
 
 val statuses = {
   val stats = 
Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus])
@@ -101,7 +101,8 @@ class ListingFileCatalog(
   // - Here we are calling `getFileBlockLocations` in a sequential 
manner, but it should a
   //   a big deal since we always use to `listLeafFilesInParallel` 
when the number of paths
   //   exceeds threshold.
-  case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, 
f.getLen))
+  case f =>
+HadoopFsRelation.createLocatedFileStatus(f, 
fs.getFileBlockLocations(f, 0, f.getLen))
 }
   }.filterNot { status =>
 val name = status.getPath.getName

http://git-wip-us.apache.org/repos/asf/spark/blob/33814f88/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
index b516297..8d332df 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
@@ -348,28 +348,40 @@ private[sql] object HadoopFsRelation extends Logging {
 pathName == "_SUCCESS" || pathName == "_temporary" || 
pathName.startsWith(".")
   }
 
+  /**
+   * Create a LocatedFileStatus using FileStatus and block loca

[3/3] spark git commit: [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable

2016-05-17 Thread lian
[SPARK-15171][SQL] Remove the references to deprecated method 
dataset.registerTempTable

## What changes were proposed in this pull request?

Update the unit test code, examples, and documents to remove calls to 
deprecated method `dataset.registerTempTable`.

## How was this patch tested?

This PR only changes the unit test code, examples, and comments. It should be 
safe.
This is a follow up of PR https://github.com/apache/spark/pull/12945 which was 
merged.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #13098 from clockfly/spark-15171-remove-deprecation.

(cherry picked from commit 25b315e6cad7c27b62dcaa2c194293c1115fdfb3)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 5f5270ead86d5294af6c871e36112e2a833e9d64
Parents: 1db3741
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Wed May 18 09:01:59 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 18 09:05:34 2016 +0800

--
 docs/sql-programming-guide.md   |  48 -
 docs/streaming-programming-guide.md |  12 +--
 .../apache/spark/examples/sql/JavaSparkSQL.java |   8 +-
 .../streaming/JavaSqlNetworkWordCount.java  |   2 +-
 examples/src/main/python/sql.py |   2 +-
 .../python/streaming/sql_network_wordcount.py   |   2 +-
 .../apache/spark/examples/sql/RDDRelation.scala |   6 +-
 .../spark/examples/sql/hive/HiveFromSpark.scala |   4 +-
 .../streaming/SqlNetworkWordCount.scala |   2 +-
 .../org/apache/spark/ml/JavaPipelineSuite.java  |   2 +-
 .../JavaLogisticRegressionSuite.java|  10 +-
 .../regression/JavaLinearRegressionSuite.java   |   4 +-
 python/pyspark/sql/context.py   |   4 +-
 python/pyspark/sql/readwriter.py|   2 +-
 python/pyspark/sql/session.py   |   2 +-
 python/pyspark/sql/tests.py |  25 ++---
 .../scala/org/apache/spark/sql/SQLContext.scala |   2 +-
 .../apache/spark/sql/JavaApplySchemaSuite.java  |   8 +-
 .../spark/sql/sources/JavaSaveLoadSuite.java|   2 +-
 .../org/apache/spark/sql/CachedTableSuite.scala |  60 +--
 .../spark/sql/ColumnExpressionSuite.scala   |   2 +-
 .../spark/sql/DataFrameTimeWindowingSuite.scala |   2 +-
 .../apache/spark/sql/DataFrameWindowSuite.scala |  22 ++--
 .../scala/org/apache/spark/sql/JoinSuite.scala  |   4 +-
 .../org/apache/spark/sql/ListTablesSuite.scala  |   4 +-
 .../org/apache/spark/sql/SQLContextSuite.scala  |   2 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala| 103 ++-
 .../sql/ScalaReflectionRelationSuite.scala  |  10 +-
 .../org/apache/spark/sql/SubquerySuite.scala|   8 +-
 .../scala/org/apache/spark/sql/UDFSuite.scala   |  12 +--
 .../apache/spark/sql/UserDefinedTypeSuite.scala |   2 +-
 .../spark/sql/execution/PlannerSuite.scala  |  10 +-
 .../benchmark/AggregateBenchmark.scala  |   3 +-
 .../columnar/InMemoryColumnarQuerySuite.scala   |   8 +-
 .../columnar/PartitionBatchPruningSuite.scala   |   2 +-
 .../execution/datasources/json/JsonSuite.scala  |  58 +--
 .../ParquetPartitionDiscoverySuite.scala|  10 +-
 .../datasources/parquet/ParquetQuerySuite.scala |   4 +-
 .../parquet/ParquetReadBenchmark.scala  |  20 ++--
 .../datasources/parquet/TPCDSBenchmark.scala|   2 +-
 .../sql/execution/metric/SQLMetricsSuite.scala  |   8 +-
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala   |   2 +-
 .../sql/sources/CreateTableAsSelectSuite.scala  |   2 +-
 .../apache/spark/sql/sources/InsertSuite.scala  |   6 +-
 .../spark/sql/sources/SaveLoadSuite.scala   |   4 +-
 .../spark/sql/streaming/StreamSuite.scala   |   2 +-
 .../org/apache/spark/sql/test/SQLTestData.scala |  46 -
 .../spark/sql/hive/JavaDataFrameSuite.java  |   2 +-
 .../sql/hive/JavaMetastoreDataSourcesSuite.java |   2 +-
 .../spark/sql/hive/ErrorPositionSuite.scala |   4 +-
 .../spark/sql/hive/HiveParquetSuite.scala   |   4 +-
 .../spark/sql/hive/HiveSparkSubmitSuite.scala   |   8 +-
 .../sql/hive/InsertIntoHiveTableSuite.scala |  12 +--
 .../sql/hive/MetastoreDataSourcesSuite.scala|   8 +-
 .../hive/ParquetHiveCompatibilitySuite.scala|   2 +-
 .../spark/sql/hive/QueryPartitionSuite.scala|   2 +-
 .../apache/spark/sql/hive/StatisticsSuite.scala |   2 +-
 .../org/apache/spark/sql/hive/UDFSuite.scala|   2 +-
 .../hive/execution/AggregationQuerySuite.scala  |   8 +-
 .../sql/hive/execution/HiveExplainSuite.scala   |   2 +-
 .../execution/HiveOperatorQueryableSuite.scala  |   4 +-
 .../spark/sql/hive/execution/HivePlanTest.scala |   2 +-
 .../sql/hive/execut

[2/3] spark git commit: [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable

2016-05-17 Thread lian
http://git-wip-us.apache.org/repos/asf/spark/blob/5f5270ea/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
index d2e1ea1..2a5295d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
@@ -78,7 +78,7 @@ class PlannerSuite extends SharedSQLContext {
 val schema = StructType(fields)
 val row = Row.fromSeq(Seq.fill(fields.size)(null))
 val rowRDD = sparkContext.parallelize(row :: Nil)
-spark.createDataFrame(rowRDD, schema).registerTempTable("testLimit")
+spark.createDataFrame(rowRDD, 
schema).createOrReplaceTempView("testLimit")
 
 val planned = sql(
   """
@@ -132,7 +132,7 @@ class PlannerSuite extends SharedSQLContext {
   test("InMemoryRelation statistics propagation") {
 withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "81920") {
   withTempTable("tiny") {
-testData.limit(3).registerTempTable("tiny")
+testData.limit(3).createOrReplaceTempView("tiny")
 sql("CACHE TABLE tiny")
 
 val a = testData.as("a")
@@ -199,9 +199,9 @@ class PlannerSuite extends SharedSQLContext {
 
   test("PartitioningCollection") {
 withTempTable("normal", "small", "tiny") {
-  testData.registerTempTable("normal")
-  testData.limit(10).registerTempTable("small")
-  testData.limit(3).registerTempTable("tiny")
+  testData.createOrReplaceTempView("normal")
+  testData.limit(10).createOrReplaceTempView("small")
+  testData.limit(3).createOrReplaceTempView("tiny")
 
   // Disable broadcast join
   withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {

http://git-wip-us.apache.org/repos/asf/spark/blob/5f5270ea/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
index b31338e..bf3a39c 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
@@ -134,7 +134,8 @@ class AggregateBenchmark extends BenchmarkBase {
 val N = 20 << 22
 
 val benchmark = new Benchmark("Aggregate w keys", N)
-sparkSession.range(N).selectExpr("id", "floor(rand() * 1) as 
k").registerTempTable("test")
+sparkSession.range(N).selectExpr("id", "floor(rand() * 1) as k")
+  .createOrReplaceTempView("test")
 
 def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by 
k, k").collect()
 

http://git-wip-us.apache.org/repos/asf/spark/blob/5f5270ea/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
index 2099d4e..e2fb913 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
@@ -42,7 +42,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with 
SharedSQLContext {
   test("default size avoids broadcast") {
 // TODO: Improve this test when we have better statistics
 sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString))
-  .toDF().registerTempTable("sizeTst")
+  .toDF().createOrReplaceTempView("sizeTst")
 spark.catalog.cacheTable("sizeTst")
 assert(
   spark.table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes >
@@ -92,7 +92,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with 
SharedSQLContext {
 
   test("SPARK-2729 regression: timestamp data type") {
 val timestamps = (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time")
-timestamps.registerTempTable("timestamps")
+timestamps.createOrReplaceTempView("timestamps")
 
 checkAnswer(
   sql("SELECT time FROM timestamps"),
@@ -133,7 +133,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with 
SharedSQLContext {
 
 assert(df.schema.head.dataType === DecimalType(15, 10))
 
-df.cache().registerTempTable("test_fixed_decimal")
+df.cache().createOrReplaceTempView("test_fixed_decimal")
 checkAnswer(
   sql("SELECT * FROM test_fixed_decimal"),

[2/3] spark git commit: [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable

2016-05-17 Thread lian
http://git-wip-us.apache.org/repos/asf/spark/blob/25b315e6/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
index d2e1ea1..2a5295d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
@@ -78,7 +78,7 @@ class PlannerSuite extends SharedSQLContext {
 val schema = StructType(fields)
 val row = Row.fromSeq(Seq.fill(fields.size)(null))
 val rowRDD = sparkContext.parallelize(row :: Nil)
-spark.createDataFrame(rowRDD, schema).registerTempTable("testLimit")
+spark.createDataFrame(rowRDD, 
schema).createOrReplaceTempView("testLimit")
 
 val planned = sql(
   """
@@ -132,7 +132,7 @@ class PlannerSuite extends SharedSQLContext {
   test("InMemoryRelation statistics propagation") {
 withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "81920") {
   withTempTable("tiny") {
-testData.limit(3).registerTempTable("tiny")
+testData.limit(3).createOrReplaceTempView("tiny")
 sql("CACHE TABLE tiny")
 
 val a = testData.as("a")
@@ -199,9 +199,9 @@ class PlannerSuite extends SharedSQLContext {
 
   test("PartitioningCollection") {
 withTempTable("normal", "small", "tiny") {
-  testData.registerTempTable("normal")
-  testData.limit(10).registerTempTable("small")
-  testData.limit(3).registerTempTable("tiny")
+  testData.createOrReplaceTempView("normal")
+  testData.limit(10).createOrReplaceTempView("small")
+  testData.limit(3).createOrReplaceTempView("tiny")
 
   // Disable broadcast join
   withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {

http://git-wip-us.apache.org/repos/asf/spark/blob/25b315e6/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
index b31338e..bf3a39c 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala
@@ -134,7 +134,8 @@ class AggregateBenchmark extends BenchmarkBase {
 val N = 20 << 22
 
 val benchmark = new Benchmark("Aggregate w keys", N)
-sparkSession.range(N).selectExpr("id", "floor(rand() * 1) as 
k").registerTempTable("test")
+sparkSession.range(N).selectExpr("id", "floor(rand() * 1) as k")
+  .createOrReplaceTempView("test")
 
 def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by 
k, k").collect()
 

http://git-wip-us.apache.org/repos/asf/spark/blob/25b315e6/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
index 2099d4e..e2fb913 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
@@ -42,7 +42,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with 
SharedSQLContext {
   test("default size avoids broadcast") {
 // TODO: Improve this test when we have better statistics
 sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString))
-  .toDF().registerTempTable("sizeTst")
+  .toDF().createOrReplaceTempView("sizeTst")
 spark.catalog.cacheTable("sizeTst")
 assert(
   spark.table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes >
@@ -92,7 +92,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with 
SharedSQLContext {
 
   test("SPARK-2729 regression: timestamp data type") {
 val timestamps = (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time")
-timestamps.registerTempTable("timestamps")
+timestamps.createOrReplaceTempView("timestamps")
 
 checkAnswer(
   sql("SELECT time FROM timestamps"),
@@ -133,7 +133,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with 
SharedSQLContext {
 
 assert(df.schema.head.dataType === DecimalType(15, 10))
 
-df.cache().registerTempTable("test_fixed_decimal")
+df.cache().createOrReplaceTempView("test_fixed_decimal")
 checkAnswer(
   sql("SELECT * FROM test_fixed_decimal"),

[3/3] spark git commit: [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable

2016-05-17 Thread lian
[SPARK-15171][SQL] Remove the references to deprecated method 
dataset.registerTempTable

## What changes were proposed in this pull request?

Update the unit test code, examples, and documents to remove calls to 
deprecated method `dataset.registerTempTable`.

## How was this patch tested?

This PR only changes the unit test code, examples, and comments. It should be 
safe.
This is a follow up of PR https://github.com/apache/spark/pull/12945 which was 
merged.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #13098 from clockfly/spark-15171-remove-deprecation.


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

Branch: refs/heads/master
Commit: 25b315e6cad7c27b62dcaa2c194293c1115fdfb3
Parents: b674e67
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Wed May 18 09:01:59 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 18 09:01:59 2016 +0800

--
 docs/sql-programming-guide.md   |  48 -
 docs/streaming-programming-guide.md |  12 +--
 .../apache/spark/examples/sql/JavaSparkSQL.java |   8 +-
 .../streaming/JavaSqlNetworkWordCount.java  |   2 +-
 examples/src/main/python/sql.py |   2 +-
 .../python/streaming/sql_network_wordcount.py   |   2 +-
 .../apache/spark/examples/sql/RDDRelation.scala |   6 +-
 .../spark/examples/sql/hive/HiveFromSpark.scala |   4 +-
 .../streaming/SqlNetworkWordCount.scala |   2 +-
 .../org/apache/spark/ml/JavaPipelineSuite.java  |   2 +-
 .../JavaLogisticRegressionSuite.java|  10 +-
 .../regression/JavaLinearRegressionSuite.java   |   4 +-
 python/pyspark/sql/context.py   |   4 +-
 python/pyspark/sql/readwriter.py|   2 +-
 python/pyspark/sql/session.py   |   2 +-
 python/pyspark/sql/tests.py |  25 ++---
 .../scala/org/apache/spark/sql/SQLContext.scala |   2 +-
 .../apache/spark/sql/JavaApplySchemaSuite.java  |   8 +-
 .../spark/sql/sources/JavaSaveLoadSuite.java|   2 +-
 .../org/apache/spark/sql/CachedTableSuite.scala |  60 +--
 .../spark/sql/ColumnExpressionSuite.scala   |   2 +-
 .../spark/sql/DataFrameTimeWindowingSuite.scala |   2 +-
 .../apache/spark/sql/DataFrameWindowSuite.scala |  22 ++--
 .../scala/org/apache/spark/sql/JoinSuite.scala  |   4 +-
 .../org/apache/spark/sql/ListTablesSuite.scala  |   4 +-
 .../org/apache/spark/sql/SQLContextSuite.scala  |   2 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala| 103 ++-
 .../sql/ScalaReflectionRelationSuite.scala  |  10 +-
 .../org/apache/spark/sql/SubquerySuite.scala|   8 +-
 .../scala/org/apache/spark/sql/UDFSuite.scala   |  12 +--
 .../apache/spark/sql/UserDefinedTypeSuite.scala |   2 +-
 .../spark/sql/execution/PlannerSuite.scala  |  10 +-
 .../benchmark/AggregateBenchmark.scala  |   3 +-
 .../columnar/InMemoryColumnarQuerySuite.scala   |   8 +-
 .../columnar/PartitionBatchPruningSuite.scala   |   2 +-
 .../execution/datasources/json/JsonSuite.scala  |  58 +--
 .../ParquetPartitionDiscoverySuite.scala|  10 +-
 .../datasources/parquet/ParquetQuerySuite.scala |   4 +-
 .../parquet/ParquetReadBenchmark.scala  |  20 ++--
 .../datasources/parquet/TPCDSBenchmark.scala|   2 +-
 .../sql/execution/metric/SQLMetricsSuite.scala  |   8 +-
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala   |   2 +-
 .../sql/sources/CreateTableAsSelectSuite.scala  |   2 +-
 .../apache/spark/sql/sources/InsertSuite.scala  |   6 +-
 .../spark/sql/sources/SaveLoadSuite.scala   |   4 +-
 .../spark/sql/streaming/StreamSuite.scala   |   2 +-
 .../org/apache/spark/sql/test/SQLTestData.scala |  46 -
 .../spark/sql/hive/JavaDataFrameSuite.java  |   2 +-
 .../sql/hive/JavaMetastoreDataSourcesSuite.java |   2 +-
 .../spark/sql/hive/ErrorPositionSuite.scala |   4 +-
 .../spark/sql/hive/HiveParquetSuite.scala   |   4 +-
 .../spark/sql/hive/HiveSparkSubmitSuite.scala   |   8 +-
 .../sql/hive/InsertIntoHiveTableSuite.scala |  12 +--
 .../sql/hive/MetastoreDataSourcesSuite.scala|   8 +-
 .../hive/ParquetHiveCompatibilitySuite.scala|   2 +-
 .../spark/sql/hive/QueryPartitionSuite.scala|   2 +-
 .../apache/spark/sql/hive/StatisticsSuite.scala |   2 +-
 .../org/apache/spark/sql/hive/UDFSuite.scala|   2 +-
 .../hive/execution/AggregationQuerySuite.scala  |   8 +-
 .../sql/hive/execution/HiveExplainSuite.scala   |   2 +-
 .../execution/HiveOperatorQueryableSuite.scala  |   4 +-
 .../spark/sql/hive/execution/HivePlanTest.scala |   2 +-
 .../sql/hive/execution/HiveQuerySuite.scala |  18 ++--
 .../hive/execution/HiveResolutionSuite.scala|  10 +-
 .../sql/hive/execution/HiveTa

[1/3] spark git commit: [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable

2016-05-17 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master b674e67c2 -> 25b315e6c


http://git-wip-us.apache.org/repos/asf/spark/blob/25b315e6/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 ac9a393..81f3ea8 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
@@ -102,14 +102,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
 
   test("SPARK-6835: udtf in lateral view") {
 val df = Seq((1, 1)).toDF("c1", "c2")
-df.registerTempTable("table1")
+df.createOrReplaceTempView("table1")
 val query = sql("SELECT c1, v FROM table1 LATERAL VIEW stack(3, 1, c1 + 1, 
c1 + 2) d AS v")
 checkAnswer(query, Row(1, 1) :: Row(1, 2) :: Row(1, 3) :: Nil)
   }
 
   test("SPARK-13651: generator outputs shouldn't be resolved from its child's 
output") {
 withTempTable("src") {
-  Seq(("id1", "value1")).toDF("key", "value").registerTempTable("src")
+  Seq(("id1", "value1")).toDF("key", 
"value").createOrReplaceTempView("src")
   val query =
 sql("SELECT genoutput.* FROM src " +
   "LATERAL VIEW explode(map('key1', 100, 'key2', 200)) genoutput AS 
key, value")
@@ -135,8 +135,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
   Order(1, "Atlas", "MTB", 434, "2015-01-07", "John D", "Pacifica", "CA", 
20151),
   Order(11, "Swift", "YFlikr", 137, "2015-01-23", "John D", "Hayward", 
"CA", 20151))
 
-orders.toDF.registerTempTable("orders1")
-orderUpdates.toDF.registerTempTable("orderupdates1")
+orders.toDF.createOrReplaceTempView("orders1")
+orderUpdates.toDF.createOrReplaceTempView("orderupdates1")
 
 sql(
   """CREATE TABLE orders(
@@ -305,7 +305,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
 
   test("SPARK-5371: union with null and sum") {
 val df = Seq((1, 1)).toDF("c1", "c2")
-df.registerTempTable("table1")
+df.createOrReplaceTempView("table1")
 
 val query = sql(
   """
@@ -329,7 +329,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
 
   test("CTAS with WITH clause") {
 val df = Seq((1, 1)).toDF("c1", "c2")
-df.registerTempTable("table1")
+df.createOrReplaceTempView("table1")
 
 sql(
   """
@@ -346,7 +346,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
   }
 
   test("explode nested Field") {
-Seq(NestedArray1(NestedArray2(Seq(1, 2, 
3.toDF.registerTempTable("nestedArray")
+Seq(NestedArray1(NestedArray2(Seq(1, 2, 
3.toDF.createOrReplaceTempView("nestedArray")
 checkAnswer(
   sql("SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) a AS ints"),
   Row(1) :: Row(2) :: Row(3) :: Nil)
@@ -543,7 +543,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
   }
 
   test("specifying the column list for CTAS") {
-Seq((1, "11"), (2, "22")).toDF("key", 
"value").registerTempTable("mytable1")
+Seq((1, "11"), (2, "22")).toDF("key", 
"value").createOrReplaceTempView("mytable1")
 
 sql("create table gen__tmp(a int, b string) as select key, value from 
mytable1")
 checkAnswer(
@@ -598,7 +598,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
 
   test("double nested data") {
 sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil)
-  .toDF().registerTempTable("nested")
+  .toDF().createOrReplaceTempView("nested")
 checkAnswer(
   sql("SELECT f1.f2.f3 FROM nested"),
   Row(1))
@@ -682,7 +682,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
   test("SPARK-4963 DataFrame sample on mutable row return wrong result") {
 sql("SELECT * FROM src WHERE key % 2 = 0")
   .sample(withReplacement = false, fraction = 0.3)
-  .registerTempTable("sampled")
+  .createOrReplaceTempView("sampled")
 (1 to 10).foreach { i =>
   checkAnswer(
 sql("SELECT * FROM sampled WHERE key % 2 = 1"),
@@ -707,7 +707,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
 
 val rowRdd = sparkContext.parallelize(row :: Nil)
 
-hiveContext.createDataFrame(rowRdd, schema).registerTempTable("testTable")
+hiveContext.createDataFrame(rowRdd, 
schema).createOrReplaceTempView("testTable")
 
 sql(
   """CREATE TABLE nullValuesInInnerComplexTypes
@@ -733,14 +733,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
 
   test("SPARK-4296 Grouping field with Hive 

spark git commit: [SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempView

2016-05-12 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 6b69b8c0c -> 9098b1a17


[SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempView

## What changes were proposed in this pull request?

Deprecates registerTempTable and add dataset.createTempView, 
dataset.createOrReplaceTempView.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #12945 from clockfly/spark-15171.

(cherry picked from commit 33c6eb5218ce3c31cc9f632a67fd2c7057569683)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 9098b1a1746d9affd894cb44ed169f3cf3566b14
Parents: 6b69b8c
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Thu May 12 15:51:53 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu May 12 15:54:33 2016 +0800

--
 .../sbt_app_sql/src/main/scala/SqlApp.scala |  4 +-
 .../apache/spark/examples/sql/JavaSparkSQL.java |  8 +--
 .../streaming/JavaSqlNetworkWordCount.java  |  2 +-
 examples/src/main/python/sql.py |  2 +-
 .../python/streaming/sql_network_wordcount.py   |  2 +-
 .../apache/spark/examples/sql/RDDRelation.scala |  4 +-
 .../streaming/SqlNetworkWordCount.scala |  2 +-
 .../spark/ml/feature/SQLTransformer.scala   |  5 +-
 python/pyspark/sql/catalog.py   | 26 +++---
 python/pyspark/sql/context.py   |  4 +-
 python/pyspark/sql/dataframe.py | 51 ++--
 python/pyspark/sql/session.py   |  6 +--
 .../analysis/DistinctAggregationRewriter.scala  |  2 +-
 .../sql/catalyst/catalog/SessionCatalog.scala   |  2 +-
 .../sql/catalyst/analysis/AnalysisTest.scala|  2 +-
 .../analysis/DecimalPrecisionSuite.scala|  2 +-
 .../catalyst/catalog/SessionCatalogSuite.scala  | 26 +-
 .../org/apache/spark/sql/DataFrameWriter.scala  |  2 +-
 .../scala/org/apache/spark/sql/Dataset.scala| 30 +++-
 .../scala/org/apache/spark/sql/SQLContext.scala |  4 +-
 .../org/apache/spark/sql/SparkSession.scala | 17 +++
 .../org/apache/spark/sql/catalog/Catalog.scala  |  8 +--
 .../spark/sql/execution/SparkSqlParser.scala|  2 +-
 .../spark/sql/execution/command/cache.scala |  3 +-
 .../spark/sql/execution/command/views.scala |  2 +-
 .../spark/sql/execution/datasources/ddl.scala   |  4 +-
 .../apache/spark/sql/internal/CatalogImpl.scala | 12 ++---
 .../org/apache/spark/sql/CachedTableSuite.scala |  6 +--
 .../org/apache/spark/sql/DataFrameSuite.scala   | 19 
 .../spark/sql/DataFrameTimeWindowingSuite.scala |  2 +-
 .../org/apache/spark/sql/DatasetSuite.scala | 18 +++
 .../org/apache/spark/sql/ListTablesSuite.scala  |  2 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala|  6 +--
 .../scala/org/apache/spark/sql/UDFSuite.scala   |  4 +-
 .../columnar/InMemoryColumnarQuerySuite.scala   |  2 +-
 .../parquet/ParquetReadBenchmark.scala  |  2 +-
 .../spark/sql/internal/CatalogSuite.scala   |  2 +-
 .../apache/spark/sql/test/SQLTestUtils.scala|  2 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  2 +-
 .../spark/sql/hive/ErrorPositionSuite.scala |  6 +--
 .../spark/sql/hive/HiveDDLCommandSuite.scala|  2 +-
 .../apache/spark/sql/hive/ListTablesSuite.scala |  2 +-
 .../hive/execution/AggregationQuerySuite.scala  |  2 +-
 .../spark/sql/hive/execution/HiveUDFSuite.scala |  2 +-
 .../hive/execution/SQLWindowFunctionSuite.scala |  2 +-
 45 files changed, 197 insertions(+), 120 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9098b1a1/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
--
diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala 
b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
index 69c1154..1002631 100644
--- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
+++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
@@ -41,7 +41,7 @@ object SparkSqlExample {
 import sqlContext._
 
 val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)).toDF()
-people.registerTempTable("people")
+people.createOrReplaceTempView("people")
 val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 
19")
 val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect()
 teenagerNames.foreach(println)
@@ -52,7 +52,7 @@ object SparkSqlExample {
 System.exit(-1)
   }

spark git commit: [SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempView

2016-05-12 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 5207a005c -> 33c6eb521


[SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempView

## What changes were proposed in this pull request?

Deprecates registerTempTable and add dataset.createTempView, 
dataset.createOrReplaceTempView.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzh...@databricks.com>

Closes #12945 from clockfly/spark-15171.


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

Branch: refs/heads/master
Commit: 33c6eb5218ce3c31cc9f632a67fd2c7057569683
Parents: 5207a00
Author: Sean Zhong <seanzh...@databricks.com>
Authored: Thu May 12 15:51:53 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu May 12 15:51:53 2016 +0800

--
 .../sbt_app_sql/src/main/scala/SqlApp.scala |  4 +-
 .../apache/spark/examples/sql/JavaSparkSQL.java |  8 +--
 .../streaming/JavaSqlNetworkWordCount.java  |  2 +-
 examples/src/main/python/sql.py |  2 +-
 .../python/streaming/sql_network_wordcount.py   |  2 +-
 .../apache/spark/examples/sql/RDDRelation.scala |  4 +-
 .../streaming/SqlNetworkWordCount.scala |  2 +-
 .../spark/ml/feature/SQLTransformer.scala   |  5 +-
 python/pyspark/sql/catalog.py   | 26 +++---
 python/pyspark/sql/context.py   |  4 +-
 python/pyspark/sql/dataframe.py | 51 ++--
 python/pyspark/sql/session.py   |  6 +--
 .../analysis/DistinctAggregationRewriter.scala  |  2 +-
 .../sql/catalyst/catalog/SessionCatalog.scala   |  2 +-
 .../sql/catalyst/analysis/AnalysisTest.scala|  2 +-
 .../analysis/DecimalPrecisionSuite.scala|  2 +-
 .../catalyst/catalog/SessionCatalogSuite.scala  | 26 +-
 .../org/apache/spark/sql/DataFrameWriter.scala  |  2 +-
 .../scala/org/apache/spark/sql/Dataset.scala| 30 +++-
 .../scala/org/apache/spark/sql/SQLContext.scala |  4 +-
 .../org/apache/spark/sql/SparkSession.scala | 17 +++
 .../org/apache/spark/sql/catalog/Catalog.scala  |  8 +--
 .../spark/sql/execution/SparkSqlParser.scala|  2 +-
 .../spark/sql/execution/command/cache.scala |  3 +-
 .../spark/sql/execution/command/views.scala |  2 +-
 .../spark/sql/execution/datasources/ddl.scala   |  4 +-
 .../apache/spark/sql/internal/CatalogImpl.scala | 12 ++---
 .../org/apache/spark/sql/CachedTableSuite.scala |  6 +--
 .../org/apache/spark/sql/DataFrameSuite.scala   | 19 
 .../spark/sql/DataFrameTimeWindowingSuite.scala |  2 +-
 .../org/apache/spark/sql/DatasetSuite.scala | 18 +++
 .../org/apache/spark/sql/ListTablesSuite.scala  |  2 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala|  6 +--
 .../scala/org/apache/spark/sql/UDFSuite.scala   |  4 +-
 .../columnar/InMemoryColumnarQuerySuite.scala   |  2 +-
 .../parquet/ParquetReadBenchmark.scala  |  2 +-
 .../spark/sql/internal/CatalogSuite.scala   |  2 +-
 .../apache/spark/sql/test/SQLTestUtils.scala|  2 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  2 +-
 .../spark/sql/hive/ErrorPositionSuite.scala |  6 +--
 .../spark/sql/hive/HiveDDLCommandSuite.scala|  2 +-
 .../apache/spark/sql/hive/ListTablesSuite.scala |  2 +-
 .../hive/execution/AggregationQuerySuite.scala  |  2 +-
 .../spark/sql/hive/execution/HiveUDFSuite.scala |  2 +-
 .../hive/execution/SQLWindowFunctionSuite.scala |  2 +-
 45 files changed, 197 insertions(+), 120 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/33c6eb52/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
--
diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala 
b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
index 69c1154..1002631 100644
--- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
+++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
@@ -41,7 +41,7 @@ object SparkSqlExample {
 import sqlContext._
 
 val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)).toDF()
-people.registerTempTable("people")
+people.createOrReplaceTempView("people")
 val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 
19")
 val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect()
 teenagerNames.foreach(println)
@@ -52,7 +52,7 @@ object SparkSqlExample {
 System.exit(-1)
   }
 }
-
+
 test(teenagerNames.size == 7, "Unexpected number of selected elements: " + 
teenagerNames)
   

spark git commit: [SPARK-14933][SQL] Failed to create view out of a parquet or orc table

2016-05-11 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 3bd7a89bd -> 749c29bc0


[SPARK-14933][SQL] Failed to create view out of a parquet or orc table

## What changes were proposed in this pull request?
 Symptom
 If a table is created as parquet or ORC table with hive syntaxt DDL, such as
```SQL
create table t1 (c1 int, c2 string) stored as parquet
```
The following command will fail
```SQL
create view v1 as select * from t1
```
 Root Cause
Currently, `HiveMetaStoreCatalog` converts Paruqet/Orc tables to 
`LogicalRelation` without giving any `tableIdentifier`. `SQLBuilder` expects 
the `LogicalRelation` to have an associated `tableIdentifier`. However, the 
`LogicalRelation` created earlier does not have such a `tableIdentifier`. Thus, 
`SQLBuilder.toSQL` can not recognize this logical plan and issue an exception.

This PR is to assign a `TableIdentifier` to the `LogicalRelation` when 
resolving parquet or orc tables in `HiveMetaStoreCatalog`.

## How was this patch tested?
testcases created and dev/run-tests is run.

Author: xin Wu <xi...@us.ibm.com>

Closes #12716 from xwu0226/SPARK_14933.

(cherry picked from commit 427c20dd6d84cb9de1aac322183bc6e7b72ca25d)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 749c29bc099c20aa6156b843cf7c9216315cd5a6
Parents: 3bd7a89
Author: xin Wu <xi...@us.ibm.com>
Authored: Wed May 11 22:17:59 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 11 22:21:40 2016 +0800

--
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 10 +--
 .../sql/catalyst/LogicalPlanToSQLSuite.scala| 24 
 .../spark/sql/hive/execution/SQLViewSuite.scala | 30 
 3 files changed, 62 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/749c29bc/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 7a799b6..607f0a1 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
@@ -293,7 +293,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   fileFormat = defaultSource,
   options = options)
 
-val created = LogicalRelation(relation)
+val created = LogicalRelation(
+  relation,
+  metastoreTableIdentifier =
+Some(TableIdentifier(tableIdentifier.name, 
Some(tableIdentifier.database
 cachedDataSourceTables.put(tableIdentifier, created)
 created
   }
@@ -317,7 +320,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   userSpecifiedSchema = Some(metastoreRelation.schema),
   bucketSpec = bucketSpec,
   options = options,
-  className = fileType).resolveRelation())
+  className = fileType).resolveRelation(),
+  metastoreTableIdentifier =
+Some(TableIdentifier(tableIdentifier.name, 
Some(tableIdentifier.database
+
 
 cachedDataSourceTables.put(tableIdentifier, created)
 created

http://git-wip-us.apache.org/repos/asf/spark/blob/749c29bc/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
index 9abefa5..4315197 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
@@ -741,4 +741,28 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with 
SQLTestUtils {
   test("filter after subquery") {
 checkHiveQl("SELECT a FROM (SELECT key + 1 AS a FROM parquet_t1) t WHERE a 
> 5")
   }
+
+  test("SPARK-14933 - select parquet table") {
+withTable("parquet_t") {
+  sql(
+"""
+  |create table parquet_t (c1 int, c2 string)
+  |stored as parquet select 1, 'abc'
+""".stripMargin)
+
+  checkHiveQl

spark git commit: [SPARK-14933][SQL] Failed to create view out of a parquet or orc table

2016-05-11 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master d88afabdf -> 427c20dd6


[SPARK-14933][SQL] Failed to create view out of a parquet or orc table

## What changes were proposed in this pull request?
 Symptom
 If a table is created as parquet or ORC table with hive syntaxt DDL, such as
```SQL
create table t1 (c1 int, c2 string) stored as parquet
```
The following command will fail
```SQL
create view v1 as select * from t1
```
 Root Cause
Currently, `HiveMetaStoreCatalog` converts Paruqet/Orc tables to 
`LogicalRelation` without giving any `tableIdentifier`. `SQLBuilder` expects 
the `LogicalRelation` to have an associated `tableIdentifier`. However, the 
`LogicalRelation` created earlier does not have such a `tableIdentifier`. Thus, 
`SQLBuilder.toSQL` can not recognize this logical plan and issue an exception.

This PR is to assign a `TableIdentifier` to the `LogicalRelation` when 
resolving parquet or orc tables in `HiveMetaStoreCatalog`.

## How was this patch tested?
testcases created and dev/run-tests is run.

Author: xin Wu <xi...@us.ibm.com>

Closes #12716 from xwu0226/SPARK_14933.


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

Branch: refs/heads/master
Commit: 427c20dd6d84cb9de1aac322183bc6e7b72ca25d
Parents: d88afab
Author: xin Wu <xi...@us.ibm.com>
Authored: Wed May 11 22:17:59 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 11 22:17:59 2016 +0800

--
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 10 +--
 .../sql/catalyst/LogicalPlanToSQLSuite.scala| 24 
 .../spark/sql/hive/execution/SQLViewSuite.scala | 30 
 3 files changed, 62 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/427c20dd/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 7a799b6..607f0a1 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
@@ -293,7 +293,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   fileFormat = defaultSource,
   options = options)
 
-val created = LogicalRelation(relation)
+val created = LogicalRelation(
+  relation,
+  metastoreTableIdentifier =
+Some(TableIdentifier(tableIdentifier.name, 
Some(tableIdentifier.database
 cachedDataSourceTables.put(tableIdentifier, created)
 created
   }
@@ -317,7 +320,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   userSpecifiedSchema = Some(metastoreRelation.schema),
   bucketSpec = bucketSpec,
   options = options,
-  className = fileType).resolveRelation())
+  className = fileType).resolveRelation(),
+  metastoreTableIdentifier =
+Some(TableIdentifier(tableIdentifier.name, 
Some(tableIdentifier.database
+
 
 cachedDataSourceTables.put(tableIdentifier, created)
 created

http://git-wip-us.apache.org/repos/asf/spark/blob/427c20dd/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
index 9abefa5..4315197 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala
@@ -741,4 +741,28 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with 
SQLTestUtils {
   test("filter after subquery") {
 checkHiveQl("SELECT a FROM (SELECT key + 1 AS a FROM parquet_t1) t WHERE a 
> 5")
   }
+
+  test("SPARK-14933 - select parquet table") {
+withTable("parquet_t") {
+  sql(
+"""
+  |create table parquet_t (c1 int, c2 string)
+  |stored as parquet select 1, 'abc'
+""".stripMargin)
+
+  checkHiveQl("select * from parquet_t")
+}
+  }
+
+  test("SPARK-14933 - select orc table") {
+withTable("orc_t&q

spark git commit: [SPARK-14459][SQL] Detect relation partitioning and adjust the logical plan

2016-05-09 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 cbb4fa1b1 -> 8caaaed95


[SPARK-14459][SQL] Detect relation partitioning and adjust the logical plan

## What changes were proposed in this pull request?

This detects a relation's partitioning and adds checks to the analyzer.
If an InsertIntoTable node has no partitioning, it is replaced by the
relation's partition scheme and input columns are correctly adjusted,
placing the partition columns at the end in partition order. If an
InsertIntoTable node has partitioning, it is checked against the table's
reported partitions.

These changes required adding a PartitionedRelation trait to the catalog
interface because Hive's MetastoreRelation doesn't extend
CatalogRelation.

This commit also includes a fix to InsertIntoTable's resolved logic,
which now detects that all expected columns are present, including
dynamic partition columns. Previously, the number of expected columns
was not checked and resolved was true if there were missing columns.

## How was this patch tested?

This adds new tests to the InsertIntoTableSuite that are fixed by this PR.

Author: Ryan Blue <b...@apache.org>

Closes #12239 from rdblue/SPARK-14459-detect-hive-partitioning.

(cherry picked from commit 652bbb1bf62722b08a062c7a2bf72019f85e179e)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 8caaaed952cef533c36cf3c0020245f1f3eb62c8
Parents: cbb4fa1
Author: Ryan Blue <b...@apache.org>
Authored: Mon May 9 17:01:23 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon May 9 17:03:21 2016 +0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 41 +-
 .../plans/logical/basicLogicalOperators.scala   | 19 -
 .../hive/execution/InsertIntoHiveTable.scala| 10 ++-
 .../sql/hive/InsertIntoHiveTableSuite.scala | 83 +++-
 .../sql/hive/execution/HiveQuerySuite.scala |  2 +-
 5 files changed, 143 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8caaaed9/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 9e9a856..b7884f9 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
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, 
SimpleCatalystConf}
-import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
+import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
InMemoryCatalog, SessionCatalog}
 import org.apache.spark.sql.catalyst.encoders.OuterScopes
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate._
@@ -444,8 +444,43 @@ class Analyzer(
 }
 
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case i @ InsertIntoTable(u: UnresolvedRelation, _, _, _, _) =>
-i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) if 
child.resolved =>
+val table = lookupTableFromCatalog(u)
+// adding the table's partitions or validate the query's partition info
+table match {
+  case relation: CatalogRelation if 
relation.catalogTable.partitionColumns.nonEmpty =>
+val tablePartitionNames = 
relation.catalogTable.partitionColumns.map(_.name)
+if (parts.keys.nonEmpty) {
+  // the query's partitioning must match the table's partitioning
+  // this is set for queries like: insert into ... partition (one 
= "a", two = )
+  // TODO: add better checking to pre-inserts to avoid needing 
this here
+  if (tablePartitionNames.size != parts.keySet.size) {
+throw new AnalysisException(
+  s"""Requested partitioning does not match the 
${u.tableIdentifier} table:
+ |Requested partitions: ${parts.keys.mkString(",")}
+ |Table partitions: 
${tablePartitionNames.mkString(",")}""".stripMar

spark git commit: [SPARK-14459][SQL] Detect relation partitioning and adjust the logical plan

2016-05-09 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 16a503cf0 -> 652bbb1bf


[SPARK-14459][SQL] Detect relation partitioning and adjust the logical plan

## What changes were proposed in this pull request?

This detects a relation's partitioning and adds checks to the analyzer.
If an InsertIntoTable node has no partitioning, it is replaced by the
relation's partition scheme and input columns are correctly adjusted,
placing the partition columns at the end in partition order. If an
InsertIntoTable node has partitioning, it is checked against the table's
reported partitions.

These changes required adding a PartitionedRelation trait to the catalog
interface because Hive's MetastoreRelation doesn't extend
CatalogRelation.

This commit also includes a fix to InsertIntoTable's resolved logic,
which now detects that all expected columns are present, including
dynamic partition columns. Previously, the number of expected columns
was not checked and resolved was true if there were missing columns.

## How was this patch tested?

This adds new tests to the InsertIntoTableSuite that are fixed by this PR.

Author: Ryan Blue <b...@apache.org>

Closes #12239 from rdblue/SPARK-14459-detect-hive-partitioning.


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

Branch: refs/heads/master
Commit: 652bbb1bf62722b08a062c7a2bf72019f85e179e
Parents: 16a503c
Author: Ryan Blue <b...@apache.org>
Authored: Mon May 9 17:01:23 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon May 9 17:01:23 2016 +0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 41 +-
 .../plans/logical/basicLogicalOperators.scala   | 19 -
 .../hive/execution/InsertIntoHiveTable.scala| 10 ++-
 .../sql/hive/InsertIntoHiveTableSuite.scala | 83 +++-
 .../sql/hive/execution/HiveQuerySuite.scala |  2 +-
 5 files changed, 143 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/652bbb1b/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 9e9a856..b7884f9 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
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, 
SimpleCatalystConf}
-import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
+import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
InMemoryCatalog, SessionCatalog}
 import org.apache.spark.sql.catalyst.encoders.OuterScopes
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate._
@@ -444,8 +444,43 @@ class Analyzer(
 }
 
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case i @ InsertIntoTable(u: UnresolvedRelation, _, _, _, _) =>
-i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) if 
child.resolved =>
+val table = lookupTableFromCatalog(u)
+// adding the table's partitions or validate the query's partition info
+table match {
+  case relation: CatalogRelation if 
relation.catalogTable.partitionColumns.nonEmpty =>
+val tablePartitionNames = 
relation.catalogTable.partitionColumns.map(_.name)
+if (parts.keys.nonEmpty) {
+  // the query's partitioning must match the table's partitioning
+  // this is set for queries like: insert into ... partition (one 
= "a", two = )
+  // TODO: add better checking to pre-inserts to avoid needing 
this here
+  if (tablePartitionNames.size != parts.keySet.size) {
+throw new AnalysisException(
+  s"""Requested partitioning does not match the 
${u.tableIdentifier} table:
+ |Requested partitions: ${parts.keys.mkString(",")}
+ |Table partitions: 
${tablePartitionNames.mkString(",")}""".stripMargin)
+  }
+  // Assume partition columns are correctly placed at the end of 
the child's

spark git commit: [SPARK-15211][SQL] Select features column from LibSVMRelation causes failure

2016-05-09 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 c0c5c264f -> 238b7b416


[SPARK-15211][SQL] Select features column from LibSVMRelation causes failure

## What changes were proposed in this pull request?

We need to use `requiredSchema` in `LibSVMRelation` to project the fetch 
required columns when loading data from this data source. Otherwise, when users 
try to select `features` column, it will cause failure.

## How was this patch tested?
`LibSVMRelationSuite`.

Author: Liang-Chi Hsieh <sim...@tw.ibm.com>

Closes #12986 from viirya/fix-libsvmrelation.

(cherry picked from commit 635ef407e11dec41ae9bc428935fb8fdaa482f7e)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 238b7b4162191fc6346ab8c7af365f83f14e8946
Parents: c0c5c26
Author: Liang-Chi Hsieh <sim...@tw.ibm.com>
Authored: Mon May 9 15:05:06 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon May 9 15:05:26 2016 +0800

--
 .../apache/spark/ml/source/libsvm/LibSVMRelation.scala| 10 +-
 .../spark/ml/source/libsvm/LibSVMRelationSuite.scala  |  1 +
 2 files changed, 10 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/238b7b41/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
index 5f78fab..68a855c 100644
--- 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
@@ -203,10 +203,18 @@ class DefaultSource extends FileFormat with 
DataSourceRegister {
   }
 
   val converter = RowEncoder(dataSchema)
+  val fullOutput = dataSchema.map { f =>
+AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()
+  }
+  val requiredOutput = fullOutput.filter { a =>
+requiredSchema.fieldNames.contains(a.name)
+  }
+
+  val requiredColumns = GenerateUnsafeProjection.generate(requiredOutput, 
fullOutput)
 
   points.map { pt =>
 val features = if (sparse) pt.features.toSparse else 
pt.features.toDense
-converter.toRow(Row(pt.label, features))
+requiredColumns(converter.toRow(Row(pt.label, features)))
   }
 }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/238b7b41/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
--
diff --git 
a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
 
b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
index e52fbd7..1d7144f 100644
--- 
a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
+++ 
b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
@@ -108,5 +108,6 @@ class LibSVMRelationSuite extends SparkFunSuite with 
MLlibTestSparkContext {
   test("select features from libsvm relation") {
 val df = sqlContext.read.format("libsvm").load(path)
 df.select("features").rdd.map { case Row(d: Vector) => d }.first
+df.select("features").collect
   }
 }


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



spark git commit: [SPARK-15211][SQL] Select features column from LibSVMRelation causes failure

2016-05-09 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master a59ab594c -> 635ef407e


[SPARK-15211][SQL] Select features column from LibSVMRelation causes failure

## What changes were proposed in this pull request?

We need to use `requiredSchema` in `LibSVMRelation` to project the fetch 
required columns when loading data from this data source. Otherwise, when users 
try to select `features` column, it will cause failure.

## How was this patch tested?
`LibSVMRelationSuite`.

Author: Liang-Chi Hsieh <sim...@tw.ibm.com>

Closes #12986 from viirya/fix-libsvmrelation.


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

Branch: refs/heads/master
Commit: 635ef407e11dec41ae9bc428935fb8fdaa482f7e
Parents: a59ab59
Author: Liang-Chi Hsieh <sim...@tw.ibm.com>
Authored: Mon May 9 15:05:06 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon May 9 15:05:06 2016 +0800

--
 .../apache/spark/ml/source/libsvm/LibSVMRelation.scala| 10 +-
 .../spark/ml/source/libsvm/LibSVMRelationSuite.scala  |  1 +
 2 files changed, 10 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/635ef407/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
index 5f78fab..68a855c 100644
--- 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
@@ -203,10 +203,18 @@ class DefaultSource extends FileFormat with 
DataSourceRegister {
   }
 
   val converter = RowEncoder(dataSchema)
+  val fullOutput = dataSchema.map { f =>
+AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()
+  }
+  val requiredOutput = fullOutput.filter { a =>
+requiredSchema.fieldNames.contains(a.name)
+  }
+
+  val requiredColumns = GenerateUnsafeProjection.generate(requiredOutput, 
fullOutput)
 
   points.map { pt =>
 val features = if (sparse) pt.features.toSparse else 
pt.features.toDense
-converter.toRow(Row(pt.label, features))
+requiredColumns(converter.toRow(Row(pt.label, features)))
   }
 }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/635ef407/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
--
diff --git 
a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
 
b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
index e52fbd7..1d7144f 100644
--- 
a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
+++ 
b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
@@ -108,5 +108,6 @@ class LibSVMRelationSuite extends SparkFunSuite with 
MLlibTestSparkContext {
   test("select features from libsvm relation") {
 val df = sqlContext.read.format("libsvm").load(path)
 df.select("features").rdd.map { case Row(d: Vector) => d }.first
+df.select("features").collect
   }
 }


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



spark git commit: [SPARK-14962][SQL] Do not push down isnotnull/isnull on unsuportted types in ORC

2016-05-06 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 3f6a13c8a -> d7c755561


[SPARK-14962][SQL] Do not push down isnotnull/isnull on unsuportted types in ORC

## What changes were proposed in this pull request?

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

ORC filters were being pushed down for all types for both `IsNull` and 
`IsNotNull`.

This is apparently OK because both `IsNull` and `IsNotNull` do not take a type 
as an argument (Hive 1.2.x) during building filters (`SearchArgument`) in 
Spark-side but they do not filter correctly because stored statistics always 
produces `null` for not supported types (eg `ArrayType`) in ORC-side. So, it is 
always `true` for `IsNull` which ends up with always `false` for `IsNotNull`. 
(Please see 
[RecordReaderImpl.java#L296-L318](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L296-L318)
  and 
[RecordReaderImpl.java#L359-L365](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L359-L365)
 in Hive 1.2)

This looks prevented in Hive 1.3.x >= by forcing to give a type 
([`PredicateLeaf.Type`](https://github.com/apache/hive/blob/e085b7e9bd059d91aaf013df0db4d71dca90ec6f/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java#L50-L56))
 when building a filter 
([`SearchArgument`](https://github.com/apache/hive/blob/26b5c7b56a4f28ce3eabc0207566cce46b29b558/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java#L260))
 but Hive 1.2.x seems not doing this.

This PR prevents ORC filter creation for `IsNull` and `IsNotNull` on 
unsupported types. `OrcFilters` resembles `ParquetFilters`.

## How was this patch tested?

Unittests in `OrcQuerySuite` and `OrcFilterSuite` and `sbt scalastyle`.

Author: hyukjinkwon <gurwls...@gmail.com>
Author: Hyukjin Kwon <gurwls...@gmail.com>

Closes #12777 from HyukjinKwon/SPARK-14962.

(cherry picked from commit fa928ff9a3c1de5d5aff9d14e6bc1bd03fcca087)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: d7c755561270ee8ec1c44df2e10a8bcb4985c3de
Parents: 3f6a13c
Author: hyukjinkwon <gurwls...@gmail.com>
Authored: Sat May 7 01:46:45 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sat May 7 01:53:08 2016 +0800

--
 .../apache/spark/sql/test/SQLTestUtils.scala|  2 +-
 .../apache/spark/sql/hive/orc/OrcFilters.scala  | 63 
 .../apache/spark/sql/hive/orc/OrcRelation.scala | 19 ++---
 .../spark/sql/hive/orc/OrcFilterSuite.scala | 75 
 .../spark/sql/hive/orc/OrcQuerySuite.scala  | 14 
 .../spark/sql/hive/orc/OrcSourceSuite.scala |  9 ++-
 6 files changed, 126 insertions(+), 56 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d7c75556/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
index ffb206a..6d2b95e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
@@ -213,7 +213,7 @@ private[sql] trait SQLTestUtils
*/
   protected def stripSparkFilter(df: DataFrame): DataFrame = {
 val schema = df.schema
-val withoutFilters = df.queryExecution.sparkPlan transform {
+val withoutFilters = df.queryExecution.sparkPlan.transform {
   case FilterExec(_, child) => child
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d7c75556/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
index c025c12..c463bc8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
@@ -17,13 +17,12 @@
 
 package org.apache.spark.sql.hive.orc
 
-import org.apache.hadoop.hive.common.`type`.{HiveChar, HiveDecimal, 
HiveVarchar}
 import org.apache.hadoop.hive.ql.io.sarg.{SearchArgument, 
SearchArgumentFactory}
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder
-import org.apache.hadoop.hive.serde2.io.DateWritable
 
 import or

spark git commit: [SPARK-14962][SQL] Do not push down isnotnull/isnull on unsuportted types in ORC

2016-05-06 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master a03c5e68a -> fa928ff9a


[SPARK-14962][SQL] Do not push down isnotnull/isnull on unsuportted types in ORC

## What changes were proposed in this pull request?

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

ORC filters were being pushed down for all types for both `IsNull` and 
`IsNotNull`.

This is apparently OK because both `IsNull` and `IsNotNull` do not take a type 
as an argument (Hive 1.2.x) during building filters (`SearchArgument`) in 
Spark-side but they do not filter correctly because stored statistics always 
produces `null` for not supported types (eg `ArrayType`) in ORC-side. So, it is 
always `true` for `IsNull` which ends up with always `false` for `IsNotNull`. 
(Please see 
[RecordReaderImpl.java#L296-L318](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L296-L318)
  and 
[RecordReaderImpl.java#L359-L365](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L359-L365)
 in Hive 1.2)

This looks prevented in Hive 1.3.x >= by forcing to give a type 
([`PredicateLeaf.Type`](https://github.com/apache/hive/blob/e085b7e9bd059d91aaf013df0db4d71dca90ec6f/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java#L50-L56))
 when building a filter 
([`SearchArgument`](https://github.com/apache/hive/blob/26b5c7b56a4f28ce3eabc0207566cce46b29b558/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java#L260))
 but Hive 1.2.x seems not doing this.

This PR prevents ORC filter creation for `IsNull` and `IsNotNull` on 
unsupported types. `OrcFilters` resembles `ParquetFilters`.

## How was this patch tested?

Unittests in `OrcQuerySuite` and `OrcFilterSuite` and `sbt scalastyle`.

Author: hyukjinkwon <gurwls...@gmail.com>
Author: Hyukjin Kwon <gurwls...@gmail.com>

Closes #12777 from HyukjinKwon/SPARK-14962.


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

Branch: refs/heads/master
Commit: fa928ff9a3c1de5d5aff9d14e6bc1bd03fcca087
Parents: a03c5e6
Author: hyukjinkwon <gurwls...@gmail.com>
Authored: Sat May 7 01:46:45 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sat May 7 01:46:45 2016 +0800

--
 .../apache/spark/sql/test/SQLTestUtils.scala|  2 +-
 .../apache/spark/sql/hive/orc/OrcFilters.scala  | 63 
 .../apache/spark/sql/hive/orc/OrcRelation.scala | 19 ++---
 .../spark/sql/hive/orc/OrcFilterSuite.scala | 75 
 .../spark/sql/hive/orc/OrcQuerySuite.scala  | 14 
 .../spark/sql/hive/orc/OrcSourceSuite.scala |  9 ++-
 6 files changed, 126 insertions(+), 56 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fa928ff9/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
index ffb206a..6d2b95e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
@@ -213,7 +213,7 @@ private[sql] trait SQLTestUtils
*/
   protected def stripSparkFilter(df: DataFrame): DataFrame = {
 val schema = df.schema
-val withoutFilters = df.queryExecution.sparkPlan transform {
+val withoutFilters = df.queryExecution.sparkPlan.transform {
   case FilterExec(_, child) => child
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fa928ff9/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
index c025c12..c463bc8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
@@ -17,13 +17,12 @@
 
 package org.apache.spark.sql.hive.orc
 
-import org.apache.hadoop.hive.common.`type`.{HiveChar, HiveDecimal, 
HiveVarchar}
 import org.apache.hadoop.hive.ql.io.sarg.{SearchArgument, 
SearchArgumentFactory}
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder
-import org.apache.hadoop.hive.serde2.io.DateWritable
 
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types._
 
 /**
  * Helper object f

spark git commit: [SPARK-14139][SQL] RowEncoder should preserve schema nullability

2016-05-05 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 77361a433 -> 55cc1c991


[SPARK-14139][SQL] RowEncoder should preserve schema nullability

## What changes were proposed in this pull request?

The problem is: In `RowEncoder`, we use `Invoke` to get the field of an 
external row, which lose the nullability information. This PR creates a 
`GetExternalRowField` expression, so that we can preserve the nullability info.

TODO: simplify the null handling logic in `RowEncoder`, to remove so many if 
branches, in follow-up PR.

## How was this patch tested?

new tests in `RowEncoderSuite`

Note that, This PR takes over https://github.com/apache/spark/pull/11980, with 
a little simplification, so all credits should go to koertkuipers

Author: Wenchen Fan <wenc...@databricks.com>
Author: Koert Kuipers <ko...@tresata.com>

Closes #12364 from cloud-fan/nullable.


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

Branch: refs/heads/master
Commit: 55cc1c991a9e39efb14177a948b09b7909e53e25
Parents: 77361a4
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri May 6 01:08:04 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri May 6 01:08:04 2016 +0800

--
 .../sql/catalyst/encoders/RowEncoder.scala  | 36 ++---
 .../sql/catalyst/expressions/objects.scala  | 42 
 .../sql/catalyst/encoders/RowEncoderSuite.scala |  8 
 .../org/apache/spark/sql/DatasetSuite.scala | 18 -
 4 files changed, 88 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/55cc1c99/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
index 44e135c..cfde3bf 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
@@ -35,9 +35,8 @@ import org.apache.spark.unsafe.types.UTF8String
 object RowEncoder {
   def apply(schema: StructType): ExpressionEncoder[Row] = {
 val cls = classOf[Row]
-val inputObject = BoundReference(0, ObjectType(cls), nullable = true)
-// We use an If expression to wrap extractorsFor result of StructType
-val serializer = serializerFor(inputObject, 
schema).asInstanceOf[If].falseValue
+val inputObject = BoundReference(0, ObjectType(cls), nullable = false)
+val serializer = serializerFor(inputObject, schema)
 val deserializer = deserializerFor(schema)
 new ExpressionEncoder[Row](
   schema,
@@ -130,21 +129,28 @@ object RowEncoder {
 
 case StructType(fields) =>
   val convertedFields = fields.zipWithIndex.map { case (f, i) =>
-val method = if (f.dataType.isInstanceOf[StructType]) {
-  "getStruct"
+val fieldValue = serializerFor(
+  GetExternalRowField(inputObject, i, 
externalDataTypeForInput(f.dataType)),
+  f.dataType
+)
+if (f.nullable) {
+  If(
+Invoke(inputObject, "isNullAt", BooleanType, Literal(i) :: Nil),
+Literal.create(null, f.dataType),
+fieldValue
+  )
 } else {
-  "get"
+  fieldValue
 }
-If(
-  Invoke(inputObject, "isNullAt", BooleanType, Literal(i) :: Nil),
-  Literal.create(null, f.dataType),
-  serializerFor(
-Invoke(inputObject, method, externalDataTypeForInput(f.dataType), 
Literal(i) :: Nil),
-f.dataType))
   }
-  If(IsNull(inputObject),
-Literal.create(null, inputType),
-CreateStruct(convertedFields))
+
+  if (inputObject.nullable) {
+If(IsNull(inputObject),
+  Literal.create(null, inputType),
+  CreateStruct(convertedFields))
+  } else {
+CreateStruct(convertedFields)
+  }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/55cc1c99/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
index 523eed8..dbaff16 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expr

spark git commit: [SPARK-14127][SQL] Native "DESC [EXTENDED | FORMATTED] " DDL command

2016-05-04 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 b99f715e8 -> 1fba7595f


[SPARK-14127][SQL] Native "DESC [EXTENDED | FORMATTED] " DDL command

## What changes were proposed in this pull request?

This PR implements native `DESC [EXTENDED | FORMATTED] ` DDL command. 
Sample output:

```
scala> spark.sql("desc extended src").show(100, truncate = false)
++-+---+
|col_name|data_type|comment|
++-+---+
|key |int  |   |
|value   |string   |   |
|| |   |
|# Detailed Table Information|CatalogTable(`default`.`src`, ...|   |
++-+---+

scala> spark.sql("desc formatted src").show(100, truncate = false)
++--+---+
|col_name|data_type 
|comment|
++--+---+
|key |int   
|   |
|value   |string
|   |
||  
|   |
|# Detailed Table Information|  
|   |
|Database:   |default   
|   |
|Owner:  |lian  
|   |
|Create Time:|Mon Jan 04 17:06:00 CST 2016  
|   |
|Last Access Time:   |Thu Jan 01 08:00:00 CST 1970  
|   |
|Location:   
|hdfs://localhost:9000/user/hive/warehouse_hive121/src |   |
|Table Type: |MANAGED   
|   |
|Table Parameters:   |  
|   |
|  transient_lastDdlTime |1451898360
|   |
||  
|   |
|# Storage Information   |  
|   |
|SerDe Library:  
|org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe|   |
|InputFormat:|org.apache.hadoop.mapred.TextInputFormat  
|   |
|OutputFormat:   
|org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat|   |
|Num Buckets:|-1
|   |
|Bucket Columns: |[]
|   |
|Sort Columns:   |[]
|   |
|Storage Desc Parameters:|  
|   |
|  serialization.format  |1 
|   |
++--+---+
```

## How was this patch tested?

A test case is added to `HiveDDLSuite` to check command output.

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

Closes #12844 from liancheng/spark-14127-desc-table.

(cherry picked from commit f152fae306dc75565cb4648ee1211416d7c0bb23)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 1fba7595f2e9cac984692238c6698bd5cfb0a453
Parents: b99f715
Author: Cheng Lian <l...@databricks.com>
Authored: Wed May 4 16:44:09 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 4 16:44:39 2016 +0800

--
 .../spark/sql/catalyst/catalog/interface.scala  |   5 +-
 .../catalyst/catalog/ExternalCatalogSuite.scala |   1 +
 .../spark/sql/execution/SparkSqlParser.scala|  14 ++-
 .../command/createDataSourceTables.scala|   2 +
 .../spark/sql/execution/command/tables.scala| 104 +++
 .../spark/sql/execution/command/DDLSuite.scala  |   6 +-
 .../spark/sql/hive/client/HiveClientImpl.scala  |   4 +-
 .../hive/exec

spark git commit: [SPARK-14127][SQL] Native "DESC [EXTENDED | FORMATTED] " DDL command

2016-05-04 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 6c12e801e -> f152fae30


[SPARK-14127][SQL] Native "DESC [EXTENDED | FORMATTED] " DDL command

## What changes were proposed in this pull request?

This PR implements native `DESC [EXTENDED | FORMATTED] ` DDL command. 
Sample output:

```
scala> spark.sql("desc extended src").show(100, truncate = false)
++-+---+
|col_name|data_type|comment|
++-+---+
|key |int  |   |
|value   |string   |   |
|| |   |
|# Detailed Table Information|CatalogTable(`default`.`src`, ...|   |
++-+---+

scala> spark.sql("desc formatted src").show(100, truncate = false)
++--+---+
|col_name|data_type 
|comment|
++--+---+
|key |int   
|   |
|value   |string
|   |
||  
|   |
|# Detailed Table Information|  
|   |
|Database:   |default   
|   |
|Owner:  |lian  
|   |
|Create Time:|Mon Jan 04 17:06:00 CST 2016  
|   |
|Last Access Time:   |Thu Jan 01 08:00:00 CST 1970  
|   |
|Location:   
|hdfs://localhost:9000/user/hive/warehouse_hive121/src |   |
|Table Type: |MANAGED   
|   |
|Table Parameters:   |  
|   |
|  transient_lastDdlTime |1451898360
|   |
||  
|   |
|# Storage Information   |  
|   |
|SerDe Library:  
|org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe|   |
|InputFormat:|org.apache.hadoop.mapred.TextInputFormat  
|   |
|OutputFormat:   
|org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat|   |
|Num Buckets:|-1
|   |
|Bucket Columns: |[]
|   |
|Sort Columns:   |[]
|   |
|Storage Desc Parameters:|  
|   |
|  serialization.format  |1 
|   |
++--+---+
```

## How was this patch tested?

A test case is added to `HiveDDLSuite` to check command output.

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

Closes #12844 from liancheng/spark-14127-desc-table.


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

Branch: refs/heads/master
Commit: f152fae306dc75565cb4648ee1211416d7c0bb23
Parents: 6c12e80
Author: Cheng Lian <l...@databricks.com>
Authored: Wed May 4 16:44:09 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 4 16:44:09 2016 +0800

--
 .../spark/sql/catalyst/catalog/interface.scala  |   5 +-
 .../catalyst/catalog/ExternalCatalogSuite.scala |   1 +
 .../spark/sql/execution/SparkSqlParser.scala|  14 ++-
 .../command/createDataSourceTables.scala|   2 +
 .../spark/sql/execution/command/tables.scala| 104 +++
 .../spark/sql/execution/command/DDLSuite.scala  |   6 +-
 .../spark/sql/hive/client/HiveClientImpl.scala  |   4 +-
 .../hive/execution/CreateTableAsSelect.scala|   3 +-
 .../spark/sql/hive/HiveDDLCommandSuite.scala|   2 +-
 .../sql/hive/MetastoreDat

spark git commit: [SPARK-14237][SQL] De-duplicate partition value appending logic in various buildReader() implementations

2016-05-04 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 2ff667606 -> 64ad9ba27


[SPARK-14237][SQL] De-duplicate partition value appending logic in various 
buildReader() implementations

## What changes were proposed in this pull request?

Currently, various `FileFormat` data sources share approximately the same code 
for partition value appending. This PR tries to eliminate this duplication.

A new method `buildReaderWithPartitionValues()` is added to `FileFormat` with a 
default implementation that appends partition values to `InternalRow`s produced 
by the reader function returned by `buildReader()`.

Special data sources like Parquet, which implements partition value appending 
inside `buildReader()` because of the vectorized reader, and the Text data 
source, which doesn't support partitioning, override 
`buildReaderWithPartitionValues()` and simply delegate to `buildReader()`.

This PR brings two benefits:

1. Apparently, it de-duplicates partition value appending logic

2. Now the reader function returned by `buildReader()` is only required to 
produce `InternalRow`s rather than `UnsafeRow`s if the data source doesn't 
override `buildReaderWithPartitionValues()`.

   Because the safe-to-unsafe conversion is also performed while appending 
partition values. This makes 3rd-party data sources (e.g. spark-avro) easier to 
implement since they no longer need to access private APIs involving 
`UnsafeRow`.

## How was this patch tested?

Existing tests should do the work.

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

Closes #12866 from liancheng/spark-14237-simplify-partition-values-appending.

(cherry picked from commit bc3760d405cc8c3ffcd957b188afa8b7e3b1f824)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-2.0
Commit: 64ad9ba27aafa333a6897861fe2aeb4bb83506fc
Parents: 2ff6676
Author: Cheng Lian <l...@databricks.com>
Authored: Wed May 4 14:16:57 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 4 14:17:26 2016 +0800

--
 .../spark/ml/source/libsvm/LibSVMRelation.scala | 17 +
 .../datasources/FileSourceStrategy.scala|  2 +-
 .../datasources/csv/DefaultSource.scala | 17 ++---
 .../datasources/fileSourceInterfaces.scala  | 40 
 .../datasources/json/JSONRelation.scala | 10 +
 .../datasources/parquet/ParquetRelation.scala   | 14 +++
 .../datasources/text/DefaultSource.scala| 13 +++
 .../execution/datasources/csv/CSVSuite.scala|  3 --
 .../apache/spark/sql/hive/orc/OrcRelation.scala | 11 +-
 9 files changed, 74 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/64ad9ba2/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
index ba2e1e2..5f78fab 100644
--- 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
@@ -204,25 +204,10 @@ class DefaultSource extends FileFormat with 
DataSourceRegister {
 
   val converter = RowEncoder(dataSchema)
 
-  val unsafeRowIterator = points.map { pt =>
+  points.map { pt =>
 val features = if (sparse) pt.features.toSparse else 
pt.features.toDense
 converter.toRow(Row(pt.label, features))
   }
-
-  def toAttribute(f: StructField): AttributeReference =
-AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()
-
-  // Appends partition values
-  val fullOutput = (dataSchema ++ partitionSchema).map(toAttribute)
-  val requiredOutput = fullOutput.filter { a =>
-requiredSchema.fieldNames.contains(a.name) || 
partitionSchema.fieldNames.contains(a.name)
-  }
-  val joinedRow = new JoinedRow()
-  val appendPartitionColumns = 
GenerateUnsafeProjection.generate(requiredOutput, fullOutput)
-
-  unsafeRowIterator.map { dataRow =>
-appendPartitionColumns(joinedRow(dataRow, file.partitionValues))
-  }
 }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/64ad9ba2/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStr

spark git commit: [SPARK-14237][SQL] De-duplicate partition value appending logic in various buildReader() implementations

2016-05-04 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 695f0e919 -> bc3760d40


[SPARK-14237][SQL] De-duplicate partition value appending logic in various 
buildReader() implementations

## What changes were proposed in this pull request?

Currently, various `FileFormat` data sources share approximately the same code 
for partition value appending. This PR tries to eliminate this duplication.

A new method `buildReaderWithPartitionValues()` is added to `FileFormat` with a 
default implementation that appends partition values to `InternalRow`s produced 
by the reader function returned by `buildReader()`.

Special data sources like Parquet, which implements partition value appending 
inside `buildReader()` because of the vectorized reader, and the Text data 
source, which doesn't support partitioning, override 
`buildReaderWithPartitionValues()` and simply delegate to `buildReader()`.

This PR brings two benefits:

1. Apparently, it de-duplicates partition value appending logic

2. Now the reader function returned by `buildReader()` is only required to 
produce `InternalRow`s rather than `UnsafeRow`s if the data source doesn't 
override `buildReaderWithPartitionValues()`.

   Because the safe-to-unsafe conversion is also performed while appending 
partition values. This makes 3rd-party data sources (e.g. spark-avro) easier to 
implement since they no longer need to access private APIs involving 
`UnsafeRow`.

## How was this patch tested?

Existing tests should do the work.

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

Closes #12866 from liancheng/spark-14237-simplify-partition-values-appending.


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

Branch: refs/heads/master
Commit: bc3760d405cc8c3ffcd957b188afa8b7e3b1f824
Parents: 695f0e9
Author: Cheng Lian <l...@databricks.com>
Authored: Wed May 4 14:16:57 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed May 4 14:16:57 2016 +0800

--
 .../spark/ml/source/libsvm/LibSVMRelation.scala | 17 +
 .../datasources/FileSourceStrategy.scala|  2 +-
 .../datasources/csv/DefaultSource.scala | 17 ++---
 .../datasources/fileSourceInterfaces.scala  | 40 
 .../datasources/json/JSONRelation.scala | 10 +
 .../datasources/parquet/ParquetRelation.scala   | 14 +++
 .../datasources/text/DefaultSource.scala| 13 +++
 .../execution/datasources/csv/CSVSuite.scala|  3 --
 .../apache/spark/sql/hive/orc/OrcRelation.scala | 11 +-
 9 files changed, 74 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/bc3760d4/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
index ba2e1e2..5f78fab 100644
--- 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
@@ -204,25 +204,10 @@ class DefaultSource extends FileFormat with 
DataSourceRegister {
 
   val converter = RowEncoder(dataSchema)
 
-  val unsafeRowIterator = points.map { pt =>
+  points.map { pt =>
 val features = if (sparse) pt.features.toSparse else 
pt.features.toDense
 converter.toRow(Row(pt.label, features))
   }
-
-  def toAttribute(f: StructField): AttributeReference =
-AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()
-
-  // Appends partition values
-  val fullOutput = (dataSchema ++ partitionSchema).map(toAttribute)
-  val requiredOutput = fullOutput.filter { a =>
-requiredSchema.fieldNames.contains(a.name) || 
partitionSchema.fieldNames.contains(a.name)
-  }
-  val joinedRow = new JoinedRow()
-  val appendPartitionColumns = 
GenerateUnsafeProjection.generate(requiredOutput, fullOutput)
-
-  unsafeRowIterator.map { dataRow =>
-appendPartitionColumns(joinedRow(dataRow, file.partitionValues))
-  }
 }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/bc3760d4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
index 615906a..8a93c6f 1006

spark git commit: [SPARK-14875][SQL] Makes OutputWriterFactory.newInstance public

2016-04-25 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master cef77d1f6 -> e66afd5c6


[SPARK-14875][SQL] Makes OutputWriterFactory.newInstance public

## What changes were proposed in this pull request?

This method was accidentally made `private[sql]` in Spark 2.0. This PR makes it 
public again, since 3rd party data sources like spark-avro depend on it.

## How was this patch tested?

N/A

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

Closes #12652 from liancheng/spark-14875.


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

Branch: refs/heads/master
Commit: e66afd5c66a6d8f33c90ce08c35c3823735bde83
Parents: cef77d1
Author: Cheng Lian <l...@databricks.com>
Authored: Mon Apr 25 20:42:49 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon Apr 25 20:42:49 2016 +0800

--
 .../spark/sql/execution/datasources/fileSourceInterfaces.scala | 2 +-
 .../scala/org/apache/spark/sql/sources/SimpleTextRelation.scala| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e66afd5c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
index ed24bdd..4063c6e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
@@ -59,7 +59,7 @@ abstract class OutputWriterFactory extends Serializable {
* @param context The Hadoop MapReduce task context.
* @since 1.4.0
*/
-  private[sql] def newInstance(
+  def newInstance(
   path: String,
   bucketId: Option[Int], // TODO: This doesn't belong here...
   dataSchema: StructType,

http://git-wip-us.apache.org/repos/asf/spark/blob/e66afd5c/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
index 16996ae..dad4f87 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
@@ -48,7 +48,7 @@ class SimpleTextSource extends FileFormat with 
DataSourceRegister {
   job: Job,
   options: Map[String, String],
   dataSchema: StructType): OutputWriterFactory = new OutputWriterFactory {
-override private[sql] def newInstance(
+override def newInstance(
 path: String,
 bucketId: Option[Int],
 dataSchema: StructType,


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



spark git commit: [SPARK-14843][ML] Fix encoding error in LibSVMRelation

2016-04-22 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master c089c6f4e -> 8098f1585


[SPARK-14843][ML] Fix encoding error in LibSVMRelation

## What changes were proposed in this pull request?

We use `RowEncoder` in libsvm data source to serialize the label and features 
read from libsvm files. However, the schema passed in this encoder is not 
correct. As the result, we can't correctly select `features` column from the 
DataFrame. We should use full data schema instead of `requiredSchema` to 
serialize the data read in. Then do projection to select required columns later.

## How was this patch tested?
`LibSVMRelationSuite`.

Author: Liang-Chi Hsieh <sim...@tw.ibm.com>

Closes #12611 from viirya/fix-libsvm.


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

Branch: refs/heads/master
Commit: 8098f158576b07343f74e2061d217b106c71b62d
Parents: c089c6f
Author: Liang-Chi Hsieh <sim...@tw.ibm.com>
Authored: Sat Apr 23 01:11:36 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sat Apr 23 01:11:36 2016 +0800

--
 .../org/apache/spark/ml/source/libsvm/LibSVMRelation.scala  | 9 ++---
 .../apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala | 9 +++--
 2 files changed, 13 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8098f158/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
index e8b0dd6..dc2a6f5 100644
--- 
a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala
@@ -202,7 +202,7 @@ class DefaultSource extends FileFormat with 
DataSourceRegister {
 LabeledPoint(label, Vectors.sparse(numFeatures, indices, values))
   }
 
-  val converter = RowEncoder(requiredSchema)
+  val converter = RowEncoder(dataSchema)
 
   val unsafeRowIterator = points.map { pt =>
 val features = if (sparse) pt.features.toSparse else 
pt.features.toDense
@@ -213,9 +213,12 @@ class DefaultSource extends FileFormat with 
DataSourceRegister {
 AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()
 
   // Appends partition values
-  val fullOutput = (requiredSchema ++ partitionSchema).map(toAttribute)
+  val fullOutput = (dataSchema ++ partitionSchema).map(toAttribute)
+  val requiredOutput = fullOutput.filter { a =>
+requiredSchema.fieldNames.contains(a.name) || 
partitionSchema.fieldNames.contains(a.name)
+  }
   val joinedRow = new JoinedRow()
-  val appendPartitionColumns = 
GenerateUnsafeProjection.generate(fullOutput, fullOutput)
+  val appendPartitionColumns = 
GenerateUnsafeProjection.generate(requiredOutput, fullOutput)
 
   unsafeRowIterator.map { dataRow =>
 appendPartitionColumns(joinedRow(dataRow, file.partitionValues))

http://git-wip-us.apache.org/repos/asf/spark/blob/8098f158/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
--
diff --git 
a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
 
b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
index 0bd1497..e52fbd7 100644
--- 
a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
+++ 
b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala
@@ -23,9 +23,9 @@ import java.nio.charset.StandardCharsets
 import com.google.common.io.Files
 
 import org.apache.spark.{SparkException, SparkFunSuite}
-import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors}
+import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, 
Vectors}
 import org.apache.spark.mllib.util.MLlibTestSparkContext
-import org.apache.spark.sql.SaveMode
+import org.apache.spark.sql.{Row, SaveMode}
 import org.apache.spark.util.Utils
 
 
@@ -104,4 +104,9 @@ class LibSVMRelationSuite extends SparkFunSuite with 
MLlibTestSparkContext {
   df.write.format("libsvm").save(path + "_2")
 }
   }
+
+  test("select features from libsvm relation") {
+val df = sqlContext.read.format("libsvm").load(path)
+df.select("features").rdd.map { case Row(d: Vector) => d }.first
+  }
 }


---

spark git commit: [SPARK-14488][SPARK-14493][SQL] "CREATE TEMPORARY TABLE ... USING ... AS SELECT" shouldn't create persisted table

2016-04-12 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master b0f5497e9 -> 124cbfb68


[SPARK-14488][SPARK-14493][SQL] "CREATE TEMPORARY TABLE ... USING ... AS 
SELECT" shouldn't create persisted table

## What changes were proposed in this pull request?

When planning logical plan node `CreateTableUsingAsSelect`, we neglected its 
`temporary` field and always generates a `CreateMetastoreDataSourceAsSelect`. 
This PR fixes this issue generating `CreateTempTableUsingAsSelect` when 
`temporary` is true.

This PR also fixes SPARK-14493 since the root cause of SPARK-14493 is that we 
were `CreateMetastoreDataSourceAsSelect` uses default Hive warehouse location 
when `PATH` data source option is absent.

## How was this patch tested?

Added a test case to create a temporary table using the target syntax and check 
whether it's indeed a temporary table.

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

Closes #12303 from liancheng/spark-14488-fix-ctas-using.


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

Branch: refs/heads/master
Commit: 124cbfb683a5e959e1b5181d4d0cc56956b50385
Parents: b0f5497
Author: Cheng Lian <l...@databricks.com>
Authored: Tue Apr 12 22:28:57 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Apr 12 22:28:57 2016 +0800

--
 .../apache/spark/sql/hive/HiveStrategies.scala  | 10 ++--
 .../sql/hive/execution/SQLQuerySuite.scala  | 49 ++--
 2 files changed, 53 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/124cbfb6/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index f44937e..010361a 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -23,9 +23,8 @@ import org.apache.spark.sql.catalyst.planning._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.command.{DescribeCommand => 
RunnableDescribeCommand, _}
-import org.apache.spark.sql.execution.datasources.{CreateTableUsing, 
CreateTableUsingAsSelect,
-  DescribeCommand}
+import org.apache.spark.sql.execution.command.{DescribeCommand => _, _}
+import org.apache.spark.sql.execution.datasources.{CreateTableUsing, 
CreateTableUsingAsSelect, CreateTempTableUsingAsSelect, DescribeCommand}
 import org.apache.spark.sql.hive.execution._
 
 private[hive] trait HiveStrategies {
@@ -90,6 +89,11 @@ private[hive] trait HiveStrategies {
 tableIdent, userSpecifiedSchema, provider, opts, allowExisting, 
managedIfNoPath)
 ExecutedCommand(cmd) :: Nil
 
+  case c: CreateTableUsingAsSelect if c.temporary =>
+val cmd = CreateTempTableUsingAsSelect(
+  c.tableIdent, c.provider, c.partitionColumns, c.mode, c.options, 
c.child)
+ExecutedCommand(cmd) :: Nil
+
   case c: CreateTableUsingAsSelect =>
 val cmd = CreateMetastoreDataSourceAsSelect(c.tableIdent, c.provider, 
c.partitionColumns,
   c.bucketSpec, c.mode, c.options, c.child)

http://git-wip-us.apache.org/repos/asf/spark/blob/124cbfb6/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 b4886eb..7eaf19d 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
@@ -19,12 +19,9 @@ package org.apache.spark.sql.hive.execution
 
 import java.sql.{Date, Timestamp}
 
-import scala.collection.JavaConverters._
-
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, 
FunctionRegistry}
-import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation}
@@ -1852,4 +1849,50 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiv

spark git commit: [SPARK-14295][MLLIB][HOTFIX] Fixes Scala 2.10 compilation failure

2016-04-01 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 22249afb4 -> 3715ecdf4


[SPARK-14295][MLLIB][HOTFIX] Fixes Scala 2.10 compilation failure

## What changes were proposed in this pull request?

Fixes a compilation failure introduced in PR #12088 under Scala 2.10.

## How was this patch tested?

Compilation.

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

Closes #12107 from liancheng/spark-14295-hotfix.


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

Branch: refs/heads/master
Commit: 3715ecdf417b47423ff07145a5623d8d817c45ef
Parents: 22249af
Author: Cheng Lian <l...@databricks.com>
Authored: Fri Apr 1 17:02:48 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Apr 1 17:02:48 2016 +0800

--
 mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3715ecdf/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
--
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
index 4b9d779..774170f 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
@@ -120,7 +120,7 @@ object MLUtils {
   i += 1
 }
 
-(label, indices, values)
+(label, indices.toArray, values.toArray)
   }
 
   /**


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



spark git commit: [SPARK-14114][SQL] implement buildReader for text data source

2016-03-30 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 7320f9bd1 -> 816f359cf


[SPARK-14114][SQL] implement buildReader for text data source

## What changes were proposed in this pull request?

This PR implements buildReader for text data source and enable it in the new 
data source code path.

## How was this patch tested?

Existing tests.

Author: Wenchen Fan <wenc...@databricks.com>

Closes #11934 from cloud-fan/text.


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

Branch: refs/heads/master
Commit: 816f359cf043ef719a0bc7df0506a3a830fff70d
Parents: 7320f9b
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Wed Mar 30 17:32:53 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Wed Mar 30 17:32:53 2016 +0800

--
 .../datasources/FileSourceStrategy.scala|  3 ++-
 .../datasources/text/DefaultSource.scala| 28 +++-
 2 files changed, 29 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/816f359c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
index 20fda95..4448796 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
@@ -59,7 +59,8 @@ private[sql] object FileSourceStrategy extends Strategy with 
Logging {
   if (files.fileFormat.toString == "TestFileFormat" ||
  files.fileFormat.isInstanceOf[parquet.DefaultSource] ||
  files.fileFormat.toString == "ORC" ||
- files.fileFormat.isInstanceOf[json.DefaultSource]) &&
+ files.fileFormat.isInstanceOf[json.DefaultSource] ||
+ files.fileFormat.isInstanceOf[text.DefaultSource]) &&
  files.sqlContext.conf.useFileScan =>
   // Filters on this relation fall into four categories based on where we 
can use them to avoid
   // reading unneeded data:

http://git-wip-us.apache.org/repos/asf/spark/blob/816f359c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala
index 5cfc9e9..d6ab5fc 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.execution.datasources.text
 
+import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileStatus, Path}
 import org.apache.hadoop.io.{LongWritable, NullWritable, Text}
 import org.apache.hadoop.mapred.{JobConf, TextInputFormat}
@@ -30,7 +31,7 @@ import org.apache.spark.sql.{AnalysisException, Row, 
SQLContext}
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, 
UnsafeRowWriter}
-import org.apache.spark.sql.execution.datasources.CompressionCodecs
+import org.apache.spark.sql.execution.datasources.{CompressionCodecs, 
HadoopFileLinesReader, PartitionedFile}
 import org.apache.spark.sql.sources._
 import org.apache.spark.sql.types.{StringType, StructType}
 import org.apache.spark.util.SerializableConfiguration
@@ -125,6 +126,31 @@ class DefaultSource extends FileFormat with 
DataSourceRegister {
   }
 }
   }
+
+  override def buildReader(
+  sqlContext: SQLContext,
+  partitionSchema: StructType,
+  dataSchema: StructType,
+  filters: Seq[Filter],
+  options: Map[String, String]): PartitionedFile => Iterator[InternalRow] 
= {
+val conf = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
+val broadcastedConf =
+  sqlContext.sparkContext.broadcast(new SerializableConfiguration(conf))
+
+file => {
+  val unsafeRow = new UnsafeRow(1)
+  val bufferHolder = new BufferHolder(unsafeRow)
+  val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1)
+
+  new HadoopFileLinesReader(file, broadcastedConf.value

spark git commit: [SPARK-14208][SQL] Renames spark.sql.parquet.fileScan

2016-03-29 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 425bcf6d6 -> a632bb56f


[SPARK-14208][SQL] Renames spark.sql.parquet.fileScan

## What changes were proposed in this pull request?

Renames SQL option `spark.sql.parquet.fileScan` since now all 
`HadoopFsRelation` based data sources are being migrated to `FileScanRDD` code 
path.

## How was this patch tested?

None.

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

Closes #12003 from liancheng/spark-14208-option-renaming.


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

Branch: refs/heads/master
Commit: a632bb56f8867df39a78d7f01fb870f548b09815
Parents: 425bcf6
Author: Cheng Lian <l...@databricks.com>
Authored: Tue Mar 29 20:56:01 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Mar 29 20:56:01 2016 +0800

--
 .../spark/sql/execution/datasources/FileSourceStrategy.scala | 2 +-
 .../main/scala/org/apache/spark/sql/internal/SQLConf.scala   | 8 
 2 files changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a632bb56/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
index 76a724e..20fda95 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
@@ -60,7 +60,7 @@ private[sql] object FileSourceStrategy extends Strategy with 
Logging {
  files.fileFormat.isInstanceOf[parquet.DefaultSource] ||
  files.fileFormat.toString == "ORC" ||
  files.fileFormat.isInstanceOf[json.DefaultSource]) &&
- files.sqlContext.conf.parquetFileScan =>
+ files.sqlContext.conf.useFileScan =>
   // Filters on this relation fall into four categories based on where we 
can use them to avoid
   // reading unneeded data:
   //  - partition keys only - used to prune directories to read

http://git-wip-us.apache.org/repos/asf/spark/blob/a632bb56/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 77af0e0..ca6ba4c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -288,9 +288,9 @@ object SQLConf {
 defaultValue = Some(true),
 doc = "Whether the query analyzer should be case sensitive or not.")
 
-  val PARQUET_FILE_SCAN = booleanConf("spark.sql.parquet.fileScan",
+  val USE_FILE_SCAN = booleanConf("spark.sql.sources.fileScan",
 defaultValue = Some(true),
-doc = "Use the new FileScanRDD path for reading parquet data.",
+doc = "Use the new FileScanRDD path for reading HDSF based data sources.",
 isPublic = false)
 
   val PARQUET_SCHEMA_MERGING_ENABLED = 
booleanConf("spark.sql.parquet.mergeSchema",
@@ -583,9 +583,9 @@ class SQLConf extends Serializable with CatalystConf with 
ParserConf with Loggin
 
   def useCompression: Boolean = getConf(COMPRESS_CACHED)
 
-  def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION)
+  def useFileScan: Boolean = getConf(USE_FILE_SCAN)
 
-  def parquetFileScan: Boolean = getConf(PARQUET_FILE_SCAN)
+  def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION)
 
   def parquetCacheMetadata: Boolean = getConf(PARQUET_CACHE_METADATA)
 


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



spark git commit: [SPARK-13456][SQL][FOLLOW-UP] lazily generate the outer pointer for case class defined in REPL

2016-03-25 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 70a6f0bb5 -> e9b6e7d85


[SPARK-13456][SQL][FOLLOW-UP] lazily generate the outer pointer for case class 
defined in REPL

## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/11410, we missed a corner case: define 
the inner class and use it in `Dataset` at the same time by using paste mode. 
For this case, the inner class and the `Dataset` are inside same line object, 
when we build the `Dataset`, we try to get outer pointer from line object, and 
it will fail because the line object is not initialized yet.

https://issues.apache.org/jira/browse/SPARK-13456?focusedCommentId=15209174=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15209174
 is an example for this corner case.

This PR make the process of getting outer pointer from line object lazy, so 
that we can successfully build the `Dataset` and finish initializing the line 
object.

## How was this patch tested?

new test in repl suite.

Author: Wenchen Fan <wenc...@databricks.com>

Closes #11931 from cloud-fan/repl.


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

Branch: refs/heads/master
Commit: e9b6e7d8577cd721a433130f29e8b112d98768b9
Parents: 70a6f0b
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri Mar 25 20:19:04 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Mar 25 20:19:04 2016 +0800

--
 .../scala/org/apache/spark/repl/ReplSuite.scala | 15 
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  2 +-
 .../sql/catalyst/encoders/OuterScopes.scala | 39 +---
 .../sql/catalyst/expressions/objects.scala  | 12 +++---
 4 files changed, 48 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e9b6e7d8/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala
--
diff --git 
a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala 
b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index f148a6d..dbfacba 100644
--- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -59,6 +59,10 @@ class ReplSuite extends SparkFunSuite {
 return out.toString
   }
 
+  // Simulate the paste mode in Scala REPL.
+  def runInterpreterInPasteMode(master: String, input: String): String =
+runInterpreter(master, ":paste\n" + input + 4.toChar) // 4 is the ascii 
code of CTRL + D
+
   def assertContains(message: String, output: String) {
 val isContain = output.contains(message)
 assert(isContain,
@@ -381,4 +385,15 @@ class ReplSuite extends SparkFunSuite {
 assertDoesNotContain("error:", output)
 assertDoesNotContain("Exception", output)
   }
+
+  test("define case class and create Dataset together with paste mode") {
+val output = runInterpreterInPasteMode("local-cluster[1,1,1024]",
+  """
+|import sqlContext.implicits._
+|case class TestClass(value: Int)
+|Seq(TestClass(1)).toDS()
+  """.stripMargin)
+assertDoesNotContain("error:", output)
+assertDoesNotContain("Exception", output)
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/e9b6e7d8/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 b344e04..89b18af 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
@@ -605,7 +605,7 @@ class Analyzer(
 "access to the scope that this class was defined in.\n" +
 "Try moving this class out of its parent class.")
   }
-  n.copy(outerPointer = Some(Literal.fromObject(outer)))
+  n.copy(outerPointer = Some(outer))
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e9b6e7d8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala
 
b/sq

spark git commit: [SPARK-13549][SQL] Refactor the Optimizer Rule CollapseProject

2016-03-23 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master cde086cb2 -> 6ce008ba4


[SPARK-13549][SQL] Refactor the Optimizer Rule CollapseProject

 What changes were proposed in this pull request?

The PR https://github.com/apache/spark/pull/10541 changed the rule 
`CollapseProject` by enabling collapsing `Project` into `Aggregate`. It leaves 
a to-do item to remove the duplicate code. This PR is to finish this to-do 
item. Also added a test case for covering this change.

 How was this patch tested?

Added a new test case.

liancheng Could you check if the code refactoring is fine? Thanks!

Author: gatorsmile <gatorsm...@gmail.com>

Closes #11427 from gatorsmile/collapseProjectRefactor.


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

Branch: refs/heads/master
Commit: 6ce008ba46aa1fc8a5c222ce0f25a6d81f53588e
Parents: cde086c
Author: gatorsmile <gatorsm...@gmail.com>
Authored: Thu Mar 24 00:51:31 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Mar 24 00:51:31 2016 +0800

--
 .../sql/catalyst/optimizer/Optimizer.scala  | 101 +--
 .../optimizer/CollapseProjectSuite.scala|  26 -
 2 files changed, 70 insertions(+), 57 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6ce008ba/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 0840d46..4cfdcf9 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
@@ -417,68 +417,57 @@ object ColumnPruning extends Rule[LogicalPlan] {
 object CollapseProject extends Rule[LogicalPlan] {
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-case p @ Project(projectList1, Project(projectList2, child)) =>
-  // Create a map of Aliases to their values from the child projection.
-  // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> 
Alias(a + b, c)).
-  val aliasMap = AttributeMap(projectList2.collect {
-case a: Alias => (a.toAttribute, a)
-  })
-
-  // We only collapse these two Projects if their overlapped expressions 
are all
-  // deterministic.
-  val hasNondeterministic = projectList1.exists(_.collect {
-case a: Attribute if aliasMap.contains(a) => aliasMap(a).child
-  }.exists(!_.deterministic))
-
-  if (hasNondeterministic) {
+case p1 @ Project(_, p2: Project) =>
+  if (haveCommonNonDeterministicOutput(p1.projectList, p2.projectList)) {
+p1
+  } else {
+p2.copy(projectList = buildCleanedProjectList(p1.projectList, 
p2.projectList))
+  }
+case p @ Project(_, agg: Aggregate) =>
+  if (haveCommonNonDeterministicOutput(p.projectList, 
agg.aggregateExpressions)) {
 p
   } else {
-// Substitute any attributes that are produced by the child 
projection, so that we safely
-// eliminate it.
-// e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a 
+ b + 1 ...'
-// TODO: Fix TransformBase to avoid the cast below.
-val substitutedProjection = projectList1.map(_.transform {
-  case a: Attribute => aliasMap.getOrElse(a, a)
-}).asInstanceOf[Seq[NamedExpression]]
-// collapse 2 projects may introduce unnecessary Aliases, trim them 
here.
-val cleanedProjection = substitutedProjection.map(p =>
-  
CleanupAliases.trimNonTopLevelAliases(p).asInstanceOf[NamedExpression]
-)
-Project(cleanedProjection, child)
+agg.copy(aggregateExpressions = buildCleanedProjectList(
+  p.projectList, agg.aggregateExpressions))
   }
+  }
 
-// TODO Eliminate duplicate code
-// This clause is identical to the one above except that the inner 
operator is an `Aggregate`
-// rather than a `Project`.
-case p @ Project(projectList1, agg @ Aggregate(_, projectList2, child)) =>
-  // Create a map of Aliases to their values from the child projection.
-  // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> 
Alias(a + b, c)).
-  val aliasMap = AttributeMap(projectList2.collect {
-case a: Alias => (a.toAttribute, a)
-  })
+  private def collectAliases(projectList: Seq[NamedExpression]): 
Attribu

spark git commit: [SPARK-13817][SQL][MINOR] Renames Dataset.newDataFrame to Dataset.ofRows

2016-03-23 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 7d1175011 -> cde086cb2


[SPARK-13817][SQL][MINOR] Renames Dataset.newDataFrame to Dataset.ofRows

## What changes were proposed in this pull request?

This PR does the renaming as suggested by marmbrus in [this comment][1].

## How was this patch tested?

Existing tests.

[1]: 
https://github.com/apache/spark/commit/6d37e1eb90054cdb6323b75fb202f78ece604b15#commitcomment-16654694

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

Closes #11889 from liancheng/spark-13817-follow-up.


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

Branch: refs/heads/master
Commit: cde086cb2a9a85406fc18d8e63e46425f614c15f
Parents: 7d11750
Author: Cheng Lian <l...@databricks.com>
Authored: Thu Mar 24 00:42:13 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Mar 24 00:42:13 2016 +0800

--
 .../org/apache/spark/sql/DataFrameReader.scala  |  8 +++---
 .../scala/org/apache/spark/sql/Dataset.scala|  4 +--
 .../spark/sql/KeyValueGroupedDataset.scala  |  2 +-
 .../spark/sql/RelationalGroupedDataset.scala|  8 +++---
 .../scala/org/apache/spark/sql/SQLContext.scala | 26 ++--
 .../spark/sql/execution/command/commands.scala  |  2 +-
 .../sql/execution/datasources/DataSource.scala  |  2 +-
 .../datasources/InsertIntoDataSource.scala  |  2 +-
 .../InsertIntoHadoopFsRelation.scala|  2 +-
 .../spark/sql/execution/datasources/ddl.scala   |  8 +++---
 .../sql/execution/stat/FrequentItems.scala  |  2 +-
 .../sql/execution/stat/StatFunctions.scala  |  2 +-
 .../execution/streaming/StreamExecution.scala   |  2 +-
 .../spark/sql/execution/streaming/memory.scala  |  2 +-
 .../scala/org/apache/spark/sql/functions.scala  |  2 +-
 .../org/apache/spark/sql/DataFrameSuite.scala   |  2 +-
 .../scala/org/apache/spark/sql/StreamTest.scala |  2 +-
 .../datasources/FileSourceStrategySuite.scala   |  2 +-
 .../apache/spark/sql/test/SQLTestUtils.scala|  2 +-
 .../spark/sql/hive/execution/commands.scala |  2 +-
 .../apache/spark/sql/hive/SQLBuilderTest.scala  |  2 +-
 .../hive/execution/AggregationQuerySuite.scala  |  2 +-
 22 files changed, 44 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cde086cb/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
index 1d4693f..704535a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -129,7 +129,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) 
extends Logging {
 userSpecifiedSchema = userSpecifiedSchema,
 className = source,
 options = extraOptions.toMap)
-Dataset.newDataFrame(sqlContext, 
LogicalRelation(dataSource.resolveRelation()))
+Dataset.ofRows(sqlContext, LogicalRelation(dataSource.resolveRelation()))
   }
 
   /**
@@ -176,7 +176,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) 
extends Logging {
 userSpecifiedSchema = userSpecifiedSchema,
 className = source,
 options = extraOptions.toMap)
-Dataset.newDataFrame(sqlContext, 
StreamingRelation(dataSource.createSource()))
+Dataset.ofRows(sqlContext, StreamingRelation(dataSource.createSource()))
   }
 
   /**
@@ -376,7 +376,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) 
extends Logging {
 parsedOptions)
 }
 
-Dataset.newDataFrame(
+Dataset.ofRows(
   sqlContext,
   LogicalRDD(
 schema.toAttributes,
@@ -424,7 +424,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) 
extends Logging {
* @since 1.4.0
*/
   def table(tableName: String): DataFrame = {
-Dataset.newDataFrame(sqlContext,
+Dataset.ofRows(sqlContext,
   sqlContext.sessionState.catalog.lookupRelation(
 sqlContext.sessionState.sqlParser.parseTableIdentifier(tableName)))
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/cde086cb/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 31864d6..ec0b3c7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -53,7

spark git commit: [SPARK-13772][SQL] Fix data type mismatch for decimal

2016-03-22 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 f9221ad79 -> 3243d46ab


[SPARK-13772][SQL] Fix data type mismatch for decimal

Fix data type mismatch for decimal, patch for branch-1.6.

Author: cenyuhai <cenyu...@didichuxing.com>

Closes #11605 from cenyuhai/SPARK-13772.


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

Branch: refs/heads/branch-1.6
Commit: 3243d46ab8c3f76c1a7ee2cc5588f08ec7b51cfe
Parents: f9221ad
Author: cenyuhai <cenyu...@didichuxing.com>
Authored: Tue Mar 22 20:53:11 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Mar 22 20:53:18 2016 +0800

--
 .../sql/catalyst/analysis/HiveTypeCoercion.scala  |  2 +-
 .../sql/catalyst/analysis/HiveTypeCoercionSuite.scala | 14 ++
 2 files changed, 15 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3243d46a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index dbcbd68..470af35 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -677,7 +677,7 @@ object HiveTypeCoercion {
   case e if !e.childrenResolved => e
   // Find tightest common type for If, if the true value and false value 
have different types.
   case i @ If(pred, left, right) if left.dataType != right.dataType =>
-findTightestCommonTypeToString(left.dataType, right.dataType).map { 
widestType =>
+findWiderTypeForTwo(left.dataType, right.dataType).map { widestType =>
   val newLeft = if (left.dataType == widestType) left else Cast(left, 
widestType)
   val newRight = if (right.dataType == widestType) right else 
Cast(right, widestType)
   If(pred, newLeft, newRight)

http://git-wip-us.apache.org/repos/asf/spark/blob/3243d46a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
index 1429150..3b5bd77 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
@@ -299,6 +299,20 @@ class HiveTypeCoercionSuite extends PlanTest {
 )
   }
 
+  test("test for SPARK-13772") {
+val rule = HiveTypeCoercion.IfCoercion
+ruleTest(rule,
+  If(Literal(true), Literal(1.0), Cast(Literal(1.0), DecimalType(19, 0))),
+  If(Literal(true), Literal(1.0), Cast(Cast(Literal(1.0), DecimalType(19, 
0)), DoubleType))
+)
+
+ruleTest(rule,
+  If(Literal(true), Literal(Decimal(1)), Cast(Literal(1.0), 
DecimalType(19, 9))),
+  If(Literal(true), Cast(Literal(Decimal(1)), DecimalType(19, 9)),
+Cast(Literal(1.0), DecimalType(19, 9)))
+)
+  }
+
   test("type coercion for CaseKeyWhen") {
 ruleTest(HiveTypeCoercion.CaseWhenCoercion,
   CaseKeyWhen(Literal(1.toShort), Seq(Literal(1), Literal("a"))),


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



spark git commit: [SPARK-13774][SQL] - Improve error message for non-existent paths and add tests

2016-03-22 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 4e09a0d5e -> 0ce01635c


[SPARK-13774][SQL] - Improve error message for non-existent paths and add tests

SPARK-13774: IllegalArgumentException: Can not create a Path from an empty 
string for incorrect file path

**Overview:**
-   If a non-existent path is given in this call
``
scala> sqlContext.read.format("csv").load("file-path-is-incorrect.csv")
``
it throws the following error:
`java.lang.IllegalArgumentException: Can not create a Path from an empty 
string` …..
`It gets called from inferSchema call in 
org.apache.spark.sql.execution.datasources.DataSource.resolveRelation`

-   The purpose of this JIRA is to throw a better error message.
-   With the fix, you will now get a _Path does not exist_ error message.
```
scala> sqlContext.read.format("csv").load("file-path-is-incorrect.csv")
org.apache.spark.sql.AnalysisException: Path does not exist: 
file:/Users/ksunitha/trunk/spark/file-path-is-incorrect.csv;
  at 
org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:215)
  at 
org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:204)
  ...
  at 
org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:204)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:131)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:141)
  ... 49 elided
```

**Details**
_Changes include:_
-   Check if path exists or not in resolveRelation in DataSource, and throw 
an AnalysisException with message like “Path does not exist: $path”
-   AnalysisException is thrown similar to the exceptions thrown in 
resolveRelation.
-   The glob path and the non glob path is checked with minimal calls to 
path exists. If the globPath is empty, then it is a nonexistent glob pattern 
and an error will be thrown. In the scenario that it is not globPath, it is 
necessary to only check if the first element in the Seq is valid or not.

_Test modifications:_
-   Changes went in for 3 tests to account for this error checking.
-   SQLQuerySuite:test("run sql directly on files") – Error message 
needed to be updated.
-   2 tests failed in MetastoreDataSourcesSuite because they had a dummy 
path and so test is modified to give a tempdir and allow it to move past so it 
can continue to test the codepath it meant to test

_New Tests:_
2 new tests are added to DataFrameSuite to validate that glob and non-glob path 
will throw the new error message.

_Testing:_
Unit tests were run with the fix.

**Notes/Questions to reviewers:**
-   There is some code duplication in DataSource.scala in resolveRelation 
method and also createSource with respect to getting the paths.  I have not 
made any changes to the createSource codepath.  Should we make the change there 
as well ?

-   From other JIRAs, I know there is restructuring and changes going on in 
this area, not sure how that will affect these changes, but since this seemed 
like a starter issue, I looked into it.  If we prefer not to add the overhead 
of the checks, or if there is a better place to do so, let me know.

I would appreciate your review. Thanks for your time and comments.

Author: Sunitha Kambhampati <skam...@us.ibm.com>

Closes #11775 from skambha/improve_errmsg.


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

Branch: refs/heads/master
Commit: 0ce01635cc66ca5f9d8962235054335b16f7507e
Parents: 4e09a0d
Author: Sunitha Kambhampati <skam...@us.ibm.com>
Authored: Tue Mar 22 20:47:57 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Mar 22 20:47:57 2016 +0800

--
 .../sql/execution/datasources/DataSource.scala  | 11 ++-
 .../org/apache/spark/sql/DataFrameSuite.scala   | 16 
 .../org/apache/spark/sql/SQLQuerySuite.scala|  2 +-
 .../sql/hive/MetastoreDataSourcesSuite.scala| 97 ++--
 4 files changed, 78 insertions(+), 48 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0ce01635/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
index e2d5f42..e2a14ed 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spa

spark git commit: [SPARK-13473][SQL] Simplifies PushPredicateThroughProject

2016-03-22 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 14464cadb -> f2e855fba


[SPARK-13473][SQL] Simplifies PushPredicateThroughProject

## What changes were proposed in this pull request?

This is a follow-up of PR #11348.

After PR #11348, a predicate is never pushed through a project as long as the 
project contains any non-deterministic fields. Thus, it's impossible that the 
candidate filter condition can reference any non-deterministic projected 
fields, and related logic can be safely cleaned up.

To be more specific, the following optimization is allowed:

```scala
// From:
df.select('a, 'b).filter('c > rand(42))
// To:
df.filter('c > rand(42)).select('a, 'b)
```

while this isn't:

```scala
// From:
df.select('a, rand('b) as 'rb, 'c).filter('c > 'rb)
// To:
df.filter('c > rand('b)).select('a, rand('b) as 'rb, 'c)
```

## How was this patch tested?

Existing test cases should do the work.

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

Closes #11864 from liancheng/spark-13473-cleanup.


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

Branch: refs/heads/master
Commit: f2e855fba8eb73475cf312cdf880c1297d4323bb
Parents: 14464ca
Author: Cheng Lian <l...@databricks.com>
Authored: Tue Mar 22 19:20:56 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Mar 22 19:20:56 2016 +0800

--
 .../sql/catalyst/optimizer/Optimizer.scala  | 24 +---
 1 file changed, 1 insertion(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f2e855fb/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 41e8dc0..0840d46 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
@@ -891,29 +891,7 @@ object PushPredicateThroughProject extends 
Rule[LogicalPlan] with PredicateHelpe
 case a: Alias => (a.toAttribute, a.child)
   })
 
-  // Split the condition into small conditions by `And`, so that we can 
push down part of this
-  // condition without nondeterministic expressions.
-  val andConditions = splitConjunctivePredicates(condition)
-
-  val (deterministic, nondeterministic) = 
andConditions.partition(_.collect {
-case a: Attribute if aliasMap.contains(a) => aliasMap(a)
-  }.forall(_.deterministic))
-
-  // If there is no nondeterministic conditions, push down the whole 
condition.
-  if (nondeterministic.isEmpty) {
-project.copy(child = Filter(replaceAlias(condition, aliasMap), 
grandChild))
-  } else {
-// If they are all nondeterministic conditions, leave it un-changed.
-if (deterministic.isEmpty) {
-  filter
-} else {
-  // Push down the small conditions without nondeterministic 
expressions.
-  val pushedCondition =
-deterministic.map(replaceAlias(_, aliasMap)).reduce(And)
-  Filter(nondeterministic.reduce(And),
-project.copy(child = Filter(pushedCondition, grandChild)))
-}
-  }
+  project.copy(child = Filter(replaceAlias(condition, aliasMap), 
grandChild))
   }
 
 }


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



spark git commit: [SPARK-14000][SQL] case class with a tuple field can't work in Dataset

2016-03-21 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 2c5b18fb0 -> 17a3f0067


[SPARK-14000][SQL] case class with a tuple field can't work in Dataset

## What changes were proposed in this pull request?

When we validate an encoder, we may call `dataType` on unresolved expressions. 
This PR fix the validation so that we will resolve attributes first.

## How was this patch tested?

a new test in `DatasetSuite`

Author: Wenchen Fan <wenc...@databricks.com>

Closes #11816 from cloud-fan/encoder.


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

Branch: refs/heads/master
Commit: 17a3f00676ca0217f6ee55a1565e96893792
Parents: 2c5b18f
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Mon Mar 21 22:22:15 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon Mar 21 22:22:15 2016 +0800

--
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala  |  5 -
 .../sql/catalyst/encoders/ExpressionEncoder.scala  | 10 --
 .../catalyst/expressions/complexTypeExtractors.scala   |  7 ++-
 .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 13 +++--
 4 files changed, 29 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/17a3f006/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 333a54e..ccc65b4 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
@@ -600,7 +600,10 @@ class Analyzer(
   exprs.exists(_.collect { case _: Star => true }.nonEmpty)
   }
 
-  private def resolveExpression(expr: Expression, plan: LogicalPlan, throws: 
Boolean = false) = {
+  protected[sql] def resolveExpression(
+  expr: Expression,
+  plan: LogicalPlan,
+  throws: Boolean = false) = {
 // Resolve expression in one round.
 // If throws == false or the desired attribute doesn't exist
 // (like try to resolve `a.b` but `a` doesn't exist), fail and return the 
origin one.

http://git-wip-us.apache.org/repos/asf/spark/blob/17a3f006/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
index 58f6d0e..918233d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
@@ -282,8 +282,14 @@ case class ExpressionEncoder[T](
 // If we have nested tuple, the `fromRowExpression` will contains 
`GetStructField` instead of
 // `UnresolvedExtractValue`, so we need to check if their ordinals are all 
valid.
 // Note that, `BoundReference` contains the expected type, but here we 
need the actual type, so
-// we unbound it by the given `schema` and propagate the actual type to 
`GetStructField`.
-val unbound = fromRowExpression transform {
+// we unbound it by the given `schema` and propagate the actual type to 
`GetStructField`, after
+// we resolve the `fromRowExpression`.
+val resolved = SimpleAnalyzer.resolveExpression(
+  fromRowExpression,
+  LocalRelation(schema),
+  throws = true)
+
+val unbound = resolved transform {
   case b: BoundReference => schema(b.ordinal)
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/17a3f006/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
index 44cdc8d..c06dcc9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
@@ -110,7 +110,12 @@ case class GetStructField(child: Expression, ordinal: Int, 
name: Option[String]
 
   override def dataT

spark git commit: [SPARK-14004][SQL][MINOR] AttributeReference and Alias should only use the first qualifier to generate SQL strings

2016-03-19 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 0acb32a3f -> 14c7236dc


[SPARK-14004][SQL][MINOR] AttributeReference and Alias should only use the 
first qualifier to generate SQL strings

## What changes were proposed in this pull request?

Current implementations of `AttributeReference.sql` and `Alias.sql` joins all 
available qualifiers, which is logically wrong. But this implementation mistake 
doesn't cause any real SQL generation bugs though, since there is always at 
most one qualifier for any given `AttributeReference` or `Alias`.

This PR fixes this issue by only picking the first qualifiers.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)

Existing tests should be enough.

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

Closes #11820 from liancheng/spark-14004-single-qualifier.


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

Branch: refs/heads/master
Commit: 14c7236dc63fe362f052175886e9ad700419bc63
Parents: 0acb32a
Author: Cheng Lian <l...@databricks.com>
Authored: Sat Mar 19 00:22:17 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sat Mar 19 00:22:17 2016 +0800

--
 .../spark/sql/catalyst/expressions/namedExpressions.scala  | 6 ++
 1 file changed, 2 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/14c7236d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
index 271ef33..8c63cce 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -183,8 +183,7 @@ case class Alias(child: Expression, name: String)(
   }
 
   override def sql: String = {
-val qualifiersString =
-  if (qualifiers.isEmpty) "" else 
qualifiers.map(quoteIdentifier).mkString("", ".", ".")
+val qualifiersString = if (qualifiers.isEmpty) "" else qualifiers.head + 
"."
 s"${child.sql} AS $qualifiersString${quoteIdentifier(name)}"
   }
 }
@@ -299,8 +298,7 @@ case class AttributeReference(
   override def simpleString: String = s"$name#${exprId.id}: 
${dataType.simpleString}"
 
   override def sql: String = {
-val qualifiersString =
-  if (qualifiers.isEmpty) "" else 
qualifiers.map(quoteIdentifier).mkString("", ".", ".")
+val qualifiersString = if (qualifiers.isEmpty) "" else qualifiers.head + 
"."
 s"$qualifiersString${quoteIdentifier(name)}"
   }
 }


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



spark git commit: [SPARK-14001][SQL] support multi-children Union in SQLBuilder

2016-03-18 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 7783b6f38 -> 0f1015ffd


[SPARK-14001][SQL] support multi-children Union in SQLBuilder

## What changes were proposed in this pull request?

The fix is simple, use the existing `CombineUnions` rule to combine adjacent 
Unions before build SQL string.

## How was this patch tested?

The re-enabled test

Author: Wenchen Fan <wenc...@databricks.com>

Closes #11818 from cloud-fan/bug-fix.


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

Branch: refs/heads/master
Commit: 0f1015ffdd40cd8647f6acdd5cdd717b883e4875
Parents: 7783b6f
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri Mar 18 19:42:33 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Mar 18 19:42:33 2016 +0800

--
 .../scala/org/apache/spark/sql/hive/SQLBuilder.scala  | 14 ++
 .../apache/spark/sql/hive/LogicalPlanToSQLSuite.scala |  7 +--
 2 files changed, 11 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0f1015ff/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
--
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
index 249a685..5175bd4 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
@@ -24,9 +24,8 @@ import scala.util.control.NonFatal
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.{DataFrame, SQLContext}
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.optimizer.CollapseProject
+import org.apache.spark.sql.catalyst.optimizer.{CollapseProject, CombineUnions}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
 import org.apache.spark.sql.catalyst.util.quoteIdentifier
@@ -384,11 +383,18 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: 
SQLContext) extends Loggi
 
   object Canonicalizer extends RuleExecutor[LogicalPlan] {
 override protected def batches: Seq[Batch] = Seq(
-  Batch("Collapse Project", FixedPoint(100),
+  Batch("Prepare", FixedPoint(100),
 // The `WidenSetOperationTypes` analysis rule may introduce extra 
`Project`s over
 // `Aggregate`s to perform type casting.  This rule merges these 
`Project`s into
 // `Aggregate`s.
-CollapseProject),
+CollapseProject,
+// Parser is unable to parse the following query:
+// SELECT  `u_1`.`id`
+// FROM (((SELECT  `t0`.`id` FROM `default`.`t0`)
+// UNION ALL (SELECT  `t0`.`id` FROM `default`.`t0`))
+// UNION ALL (SELECT  `t0`.`id` FROM `default`.`t0`)) AS u_1
+// This rule combine adjacent Unions together so we can generate flat 
UNION ALL SQL string.
+CombineUnions),
   Batch("Recover Scoping Info", Once,
 // A logical plan is allowed to have same-name outputs with different 
qualifiers(e.g. the
 // `Join` operator). However, this kind of plan can't be put under a 
sub query as we will

http://git-wip-us.apache.org/repos/asf/spark/blob/0f1015ff/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
index f86eba6..f6b9072 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
@@ -141,12 +141,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with 
SQLTestUtils {
 checkHiveQl("SELECT * FROM t0 UNION SELECT * FROM t0")
   }
 
-  // Parser is unable to parse the following query:
-  // SELECT  `u_1`.`id`
-  // FROM (((SELECT  `t0`.`id` FROM `default`.`t0`)
-  // UNION ALL (SELECT  `t0`.`id` FROM `default`.`t0`))
-  // UNION ALL (SELECT  `t0`.`id` FROM `default`.`t0`)) AS u_1
-  ignore("three-child union") {
+  test("three-child union") {
 checkHiveQl(
   """
 |SELECT id FROM parquet_t0


--

spark git commit: [SPARK-13972][SQ] hive tests should fail if SQL generation failed

2016-03-18 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 53f32a22d -> 0acb32a3f


[SPARK-13972][SQ] hive tests should fail if SQL generation failed

## What changes were proposed in this pull request?

Now we should be able to convert all logical plans to SQL string, if they are 
parsed from hive query. This PR changes the error handling to throw exceptions 
instead of just log.

We will send new PRs for spotted bugs, and merge this one after all bugs are 
fixed.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenc...@databricks.com>

Closes #11782 from cloud-fan/test.


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

Branch: refs/heads/master
Commit: 0acb32a3f19484d3ea3b60fccef596025c8a8f83
Parents: 53f32a2
Author: Wenchen Fan <wenc...@databricks.com>
Authored: Fri Mar 18 23:16:14 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Mar 18 23:16:14 2016 +0800

--
 .../sql/hive/execution/HiveComparisonTest.scala | 71 
 1 file changed, 28 insertions(+), 43 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0acb32a3/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index d21bb57..019d752 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -132,25 +132,7 @@ abstract class HiveComparisonTest
 new java.math.BigInteger(1, digest.digest).toString(16)
   }
 
-  /** Used for testing [[SQLBuilder]] */
-  private var numConvertibleQueries: Int = 0
-  private var numTotalQueries: Int = 0
-
   override protected def afterAll(): Unit = {
-logInfo({
-  val percentage = if (numTotalQueries > 0) {
-numConvertibleQueries.toDouble / numTotalQueries * 100
-  } else {
-0D
-  }
-
-  s"""SQLBuilder statistics:
- |- Total query number:$numTotalQueries
- |- Number of convertible queries: $numConvertibleQueries
- |- Percentage of convertible queries: $percentage%
-   """.stripMargin
-})
-
 try {
   TestHive.reset()
 } finally {
@@ -412,32 +394,35 @@ abstract class HiveComparisonTest
   if (containsCommands) {
 originalQuery
   } else {
-numTotalQueries += 1
+val convertedSQL = try {
+  new SQLBuilder(originalQuery.analyzed, TestHive).toSQL
+} catch {
+  case NonFatal(e) => fail(
+s"""Cannot convert the following HiveQL query plan back to 
SQL query string:
+|
+|# Original HiveQL query string:
+|$queryString
+|
+|# Resolved query plan:
+|${originalQuery.analyzed.treeString}
+ """.stripMargin, e)
+}
+
 try {
-  val sql = new SQLBuilder(originalQuery.analyzed, 
TestHive).toSQL
-  numConvertibleQueries += 1
-  logInfo(
-s"""
-  |### Running SQL generation round-trip test {{{
-  |${originalQuery.analyzed.treeString}
-  |Original SQL:
-  |$queryString
-  |
-  |Generated SQL:
-  |$sql
-  |}}}
-   """.stripMargin.trim)
-  new TestHive.QueryExecution(sql)
-} catch { case NonFatal(e) =>
-  logInfo(
-s"""
-   |### Cannot convert the following logical plan back to 
SQL {{{
-   |${originalQuery.analyzed.treeString}
-   |Original SQL:
-   |$queryString
-   |}}}
-   """.stripMargin.trim)
-  originalQuery
+  new TestHive.QueryExecution(convertedSQL)
+} catch {
+  case NonFatal(e) => fail(
+s"""

spark git commit: [MINOR][DOCS] Replace `DataFrame` with `Dataset` in Javadoc.

2016-03-12 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master c079420d7 -> db88d0204


[MINOR][DOCS] Replace `DataFrame` with `Dataset` in Javadoc.

## What changes were proposed in this pull request?

SPARK-13817 (PR #11656) replaces `DataFrame` with `Dataset` from Java. This PR 
fixes the remaining broken links and sample Java code in `package-info.java`. 
As a result, it will update the following Javadoc.

* 
http://spark.apache.org/docs/latest/api/java/org/apache/spark/ml/attribute/package-summary.html
* 
http://spark.apache.org/docs/latest/api/java/org/apache/spark/ml/feature/package-summary.html

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongj...@apache.org>

Closes #11675 from dongjoon-hyun/replace_dataframe_with_dataset_in_javadoc.


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

Branch: refs/heads/master
Commit: db88d0204e3a9a05dbe6e67e1abb942639c50a06
Parents: c079420
Author: Dongjoon Hyun <dongj...@apache.org>
Authored: Sun Mar 13 12:11:18 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sun Mar 13 12:11:18 2016 +0800

--
 .../org/apache/spark/ml/attribute/package-info.java |  2 +-
 .../scala/org/apache/spark/ml/feature/package-info.java | 12 ++--
 2 files changed, 7 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/db88d020/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java 
b/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java
index e3474f3..464ed12 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java
+++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java
@@ -20,7 +20,7 @@
 /**
  * ML attributes
  *
- * The ML pipeline API uses {@link org.apache.spark.sql.DataFrame}s as ML 
datasets.
+ * The ML pipeline API uses {@link org.apache.spark.sql.Dataset}s as ML 
datasets.
  * Each dataset consists of typed columns, e.g., string, double, vector, etc.
  * However, knowing only the column type may not be sufficient to handle the 
data properly.
  * For instance, a double column with values 0.0, 1.0, 2.0, ... may represent 
some label indices,

http://git-wip-us.apache.org/repos/asf/spark/blob/db88d020/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java
--
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java 
b/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java
index 7a35f2d..dcff424 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java
@@ -22,7 +22,7 @@
  * The `ml.feature` package provides common feature transformers that help 
convert raw data or
  * features into more suitable forms for model fitting.
  * Most feature transformers are implemented as {@link 
org.apache.spark.ml.Transformer}s, which
- * transforms one {@link org.apache.spark.sql.DataFrame} into another, e.g.,
+ * transforms one {@link org.apache.spark.sql.Dataset} into another, e.g.,
  * {@link org.apache.spark.ml.feature.HashingTF}.
  * Some feature transformers are implemented as {@link 
org.apache.spark.ml.Estimator}}s, because the
  * transformation requires some aggregated information of the dataset, e.g., 
document
@@ -31,7 +31,7 @@
  * obtain the model first, e.g., {@link org.apache.spark.ml.feature.IDFModel}, 
in order to apply
  * transformation.
  * The transformation is usually done by appending new columns to the input
- * {@link org.apache.spark.sql.DataFrame}, so all input columns are carried 
over.
+ * {@link org.apache.spark.sql.Dataset}, so all input columns are carried over.
  *
  * We try to make each transformer minimal, so it becomes flexible to assemble 
feature
  * transformation pipelines.
@@ -46,7 +46,7 @@
  *   import org.apache.spark.api.java.JavaRDD;
  *   import static org.apache.spark.sql.types.DataTypes.*;
  *   import org.apache.spark.sql.types.StructType;
- *   import org.apache.spark.sql.DataFrame;
+ *   import org.apache.spark.sql.Dataset;
  *   import org.apache.spark.sql.RowFactory;
  *   import org.apache.spark.sql.Row;
  *
@@ -66,7 +66,7 @@
  *  RowFactory.create(0, "Hi I heard about Spark", 3.0),
  *  RowFactory.create(1, "I wish Java could use case classes", 4.0),
  *  RowFactory.create(2, "Logistic regression models are neat", 4.0)));
- *  DataFrame df =

spark git commit: [SPARK-13817][BUILD][SQL] Re-enable MiMA and removes object DataFrame

2016-03-11 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 07f1c5447 -> 6d37e1eb9


[SPARK-13817][BUILD][SQL] Re-enable MiMA and removes object DataFrame

## What changes were proposed in this pull request?

PR #11443 temporarily disabled MiMA check, this PR re-enables it.

One extra change is that `object DataFrame` is also removed. The only purpose 
of introducing `object DataFrame` was to use it as an internal factory for 
creating `Dataset[Row]`. By replacing this internal factory with 
`Dataset.newDataFrame`, both `DataFrame` and `DataFrame$` are entirely removed 
from the API, so that we can simply put a `MissingClassProblem` filter in 
`MimaExcludes.scala` for most DataFrame API  changes.

## How was this patch tested?

Tested by MiMA check triggered by Jenkins.

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

Closes #11656 from liancheng/re-enable-mima.


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

Branch: refs/heads/master
Commit: 6d37e1eb90054cdb6323b75fb202f78ece604b15
Parents: 07f1c54
Author: Cheng Lian <l...@databricks.com>
Authored: Fri Mar 11 22:17:50 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Mar 11 22:17:50 2016 +0800

--
 dev/run-tests.py|  3 +--
 .../ml/JavaAFTSurvivalRegressionExample.java|  1 -
 .../spark/examples/ml/JavaALSExample.java   |  1 -
 .../spark/examples/ml/JavaBinarizerExample.java |  1 -
 .../examples/ml/JavaCrossValidatorExample.java  |  1 -
 .../examples/ml/JavaDeveloperApiExample.java|  1 -
 .../JavaEstimatorTransformerParamExample.java   |  1 -
 .../spark/examples/ml/JavaKMeansExample.java|  1 -
 .../spark/examples/ml/JavaLDAExample.java   |  1 -
 .../spark/examples/ml/JavaOneVsRestExample.java |  1 -
 .../examples/ml/JavaSQLTransformerExample.java  |  1 -
 project/MimaExcludes.scala  | 22 +++
 .../scala/org/apache/spark/sql/DataFrame.scala  | 16 +--
 .../org/apache/spark/sql/DataFrameReader.scala  |  8 +++---
 .../org/apache/spark/sql/GroupedData.scala  |  8 +++---
 .../org/apache/spark/sql/GroupedDataset.scala   |  2 +-
 .../scala/org/apache/spark/sql/SQLContext.scala | 28 ++--
 .../spark/sql/execution/command/commands.scala  |  4 +--
 .../sql/execution/datasources/DataSource.scala  |  4 +--
 .../datasources/InsertIntoDataSource.scala  |  2 +-
 .../InsertIntoHadoopFsRelation.scala|  2 +-
 .../spark/sql/execution/datasources/ddl.scala   | 10 +++
 .../sql/execution/stat/FrequentItems.scala  |  4 +--
 .../sql/execution/stat/StatFunctions.scala  |  4 +--
 .../execution/streaming/StreamExecution.scala   |  2 +-
 .../spark/sql/execution/streaming/memory.scala  |  2 +-
 .../scala/org/apache/spark/sql/functions.scala  |  2 +-
 .../org/apache/spark/sql/DataFrameSuite.scala   |  2 +-
 .../scala/org/apache/spark/sql/StreamTest.scala |  2 +-
 .../apache/spark/sql/test/SQLTestUtils.scala|  2 +-
 .../spark/sql/hive/execution/commands.scala |  2 +-
 .../apache/spark/sql/hive/SQLBuilderTest.scala  |  4 +--
 .../hive/execution/AggregationQuerySuite.scala  |  2 +-
 33 files changed, 78 insertions(+), 69 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6d37e1eb/dev/run-tests.py
--
diff --git a/dev/run-tests.py b/dev/run-tests.py
index ebeede5..a1e6f1b 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -573,8 +573,7 @@ def main():
 # backwards compatibility checks
 if build_tool == "sbt":
 # Note: compatibility tests only supported in sbt for now
-# TODO Temporarily disable MiMA check for DF-to-DS migration 
prototyping
-# detect_binary_inop_with_mima()
+detect_binary_inop_with_mima()
 # Since we did not build assembly/assembly before running dev/mima, we 
need to
 # do it here because the tests still rely on it; see SPARK-13294 for 
details.
 build_spark_assembly_sbt(hadoop_version)

http://git-wip-us.apache.org/repos/asf/spark/blob/6d37e1eb/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java
--
diff --git 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java
 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java
index 3905310..22b93a3 100644
--- 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java
+++ 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvival

spark git commit: Revert "[SPARK-13616][SQL] Let SQLBuilder convert logical plan without a project on top of it"

2016-03-05 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 8290004d9 -> 8ff88094d


Revert "[SPARK-13616][SQL] Let SQLBuilder convert logical plan without a 
project on top of it"

This reverts commit f87ce0504ea0697969ac3e67690c78697b76e94a.

According to discussion in #11466, let's revert PR #11466 for safe.

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

Closes #11539 from liancheng/revert-pr-11466.


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

Branch: refs/heads/master
Commit: 8ff88094daa4945e7d718baa7b20703fd8087ab0
Parents: 8290004
Author: Cheng Lian <l...@databricks.com>
Authored: Sun Mar 6 12:54:04 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sun Mar 6 12:54:04 2016 +0800

--
 .../org/apache/spark/sql/hive/SQLBuilder.scala  | 23 +--
 .../spark/sql/hive/LogicalPlanToSQLSuite.scala  | 41 
 2 files changed, 1 insertion(+), 63 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8ff88094/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
--
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
index 8d411a9..683f738 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
@@ -65,7 +65,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: 
SQLContext) extends Loggi
 case e => e
   }
 
-  val generatedSQL = toSQL(replaced, true)
+  val generatedSQL = toSQL(replaced)
   logDebug(
 s"""Built SQL query string successfully from given logical plan:
|
@@ -90,27 +90,6 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: 
SQLContext) extends Loggi
 }
   }
 
-  private def toSQL(node: LogicalPlan, topNode: Boolean): String = {
-if (topNode) {
-  node match {
-case d: Distinct => toSQL(node)
-case p: Project => toSQL(node)
-case a: Aggregate => toSQL(node)
-case s: Sort => toSQL(node)
-case r: RepartitionByExpression => toSQL(node)
-case _ =>
-  build(
-"SELECT",
-node.output.map(_.sql).mkString(", "),
-"FROM",
-toSQL(node)
-  )
-  }
-} else {
-  toSQL(node)
-}
-  }
-
   private def toSQL(node: LogicalPlan): String = node match {
 case Distinct(p: Project) =>
   projectToSQL(p, isDistinct = true)

http://git-wip-us.apache.org/repos/asf/spark/blob/8ff88094/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
index f457d43..ed85856 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
@@ -19,8 +19,6 @@ package org.apache.spark.sql.hive
 
 import scala.util.control.NonFatal
 
-import org.apache.spark.sql.{DataFrame, SQLContext}
-import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.test.SQLTestUtils
 
@@ -56,33 +54,6 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with 
SQLTestUtils {
 sql("DROP TABLE IF EXISTS t0")
   }
 
-  private def checkPlan(plan: LogicalPlan, sqlContext: SQLContext, expected: 
String): Unit = {
-val convertedSQL = try new SQLBuilder(plan, sqlContext).toSQL catch {
-  case NonFatal(e) =>
-fail(
-  s"""Cannot convert the following logical query plan back to SQL 
query string:
- |
- |# Original logical query plan:
- |${plan.treeString}
-   """.stripMargin, e)
-}
-
-try {
-  checkAnswer(sql(convertedSQL), DataFrame(sqlContext, plan))
-} catch { case cause: Throwable =>
-  fail(
-s"""Failed to execute converted SQL string or got wrong answer:
-   |
-   |# Converted SQL query string:
-   |$convertedSQL
-   |
-   |# Original logical query plan:
-   |${plan.treeString}
- """.stripMargin,
-cause)
-}
-  }
-
   private def checkHiveQ

spark git commit: [SPARK-12720][SQL] SQL Generation Support for Cube, Rollup, and Grouping Sets

2016-03-05 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master f19228eed -> adce5ee72


[SPARK-12720][SQL] SQL Generation Support for Cube, Rollup, and Grouping Sets

 What changes were proposed in this pull request?

This PR is for supporting SQL generation for cube, rollup and grouping sets.

For example, a query using rollup:
```SQL
SELECT count(*) as cnt, key % 5, grouping_id() FROM t1 GROUP BY key % 5 WITH 
ROLLUP
```
Original logical plan:
```
  Aggregate [(key#17L % cast(5 as bigint))#47L,grouping__id#46],
[(count(1),mode=Complete,isDistinct=false) AS cnt#43L,
 (key#17L % cast(5 as bigint))#47L AS _c1#45L,
 grouping__id#46 AS _c2#44]
  +- Expand [List(key#17L, value#18, (key#17L % cast(5 as bigint))#47L, 0),
 List(key#17L, value#18, null, 1)],
[key#17L,value#18,(key#17L % cast(5 as bigint))#47L,grouping__id#46]
 +- Project [key#17L,
 value#18,
 (key#17L % cast(5 as bigint)) AS (key#17L % cast(5 as 
bigint))#47L]
+- Subquery t1
   +- Relation[key#17L,value#18] ParquetRelation
```
Converted SQL:
```SQL
  SELECT count( 1) AS `cnt`,
 (`t1`.`key` % CAST(5 AS BIGINT)),
 grouping_id() AS `_c2`
  FROM `default`.`t1`
  GROUP BY (`t1`.`key` % CAST(5 AS BIGINT))
  GROUPING SETS (((`t1`.`key` % CAST(5 AS BIGINT))), ())
```

 How was the this patch tested?

Added eight test cases in `LogicalPlanToSQLSuite`.

Author: gatorsmile <gatorsm...@gmail.com>
Author: xiaoli <lixiao1...@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #11283 from gatorsmile/groupingSetsToSQL.


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

Branch: refs/heads/master
Commit: adce5ee721c6a844ff21dfcd8515859458fe611d
Parents: f19228e
Author: gatorsmile <gatorsm...@gmail.com>
Authored: Sat Mar 5 19:25:03 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sat Mar 5 19:25:03 2016 +0800

--
 python/pyspark/sql/functions.py |  14 +-
 .../sql/catalyst/expressions/grouping.scala |   1 +
 .../org/apache/spark/sql/hive/SQLBuilder.scala  |  76 +-
 .../spark/sql/hive/LogicalPlanToSQLSuite.scala  | 143 +++
 4 files changed, 226 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/adce5ee7/python/pyspark/sql/functions.py
--
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 92e724f..88924e2 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -348,13 +348,13 @@ def grouping_id(*cols):
 grouping columns).
 
 >>> df.cube("name").agg(grouping_id(), sum("age")).orderBy("name").show()
-+-+++
-| name|groupingid()|sum(age)|
-+-+++
-| null|   1|   7|
-|Alice|   0|   2|
-|  Bob|   0|   5|
-+-+++
++-+-++
+| name|grouping_id()|sum(age)|
++-+-++
+| null|1|   7|
+|Alice|0|   2|
+|  Bob|0|   5|
++-+-++
 """
 sc = SparkContext._active_spark_context
 jc = sc._jvm.functions.grouping_id(_to_seq(sc, cols, _to_java_column))

http://git-wip-us.apache.org/repos/asf/spark/blob/adce5ee7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala
index a204060..437e417 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala
@@ -63,4 +63,5 @@ case class GroupingID(groupByExprs: Seq[Expression]) extends 
Expression with Une
   override def children: Seq[Expression] = groupByExprs
   override def dataType: DataType = IntegerType
   override def nullable: Boolean = false
+  override def prettyName: String = "grouping_id"
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/adce5ee7/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
--
diff --git a/sql/hive/src/main/scala/org/apache/spar

spark git commit: [SPARK-13540][SQL] Supports using nested classes within Scala objects as Dataset element type

2016-02-29 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master ac5c63528 -> 916fc34f9


[SPARK-13540][SQL] Supports using nested classes within Scala objects as 
Dataset element type

## What changes were proposed in this pull request?

Nested classes defined within Scala objects are translated into Java static 
nested classes. Unlike inner classes, they don't need outer scopes. But the 
analyzer still thinks that an outer scope is required.

This PR fixes this issue simply by checking whether a nested class is static 
before looking up its outer scope.

## How was this patch tested?

A test case is added to `DatasetSuite`. It checks contents of a Dataset whose 
element type is a nested class declared in a Scala object.

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

Closes #11421 from liancheng/spark-13540-object-as-outer-scope.


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

Branch: refs/heads/master
Commit: 916fc34f98dd731f607d9b3ed657bad6cc30df2c
Parents: ac5c635
Author: Cheng Lian <l...@databricks.com>
Authored: Tue Mar 1 01:07:45 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Mar 1 01:07:45 2016 +0800

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


http://git-wip-us.apache.org/repos/asf/spark/blob/916fc34f/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 23e4709..876aa0e 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
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.catalyst.analysis
 
+import java.lang.reflect.Modifier
+
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.sql.AnalysisException
@@ -559,7 +561,13 @@ class Analyzer(
   }
 
   resolveExpression(unbound, LocalRelation(attributes), throws = true) 
transform {
-case n: NewInstance if n.outerPointer.isEmpty && n.cls.isMemberClass =>
+case n: NewInstance
+  // If this is an inner class of another class, register the outer 
object in `OuterScopes`.
+  // Note that static inner classes (e.g., inner classes within Scala 
objects) don't need
+  // outer pointer registration.
+  if n.outerPointer.isEmpty &&
+ n.cls.isMemberClass &&
+ !Modifier.isStatic(n.cls.getModifiers) =>
   val outer = 
OuterScopes.outerScopes.get(n.cls.getDeclaringClass.getName)
   if (outer == null) {
 throw new AnalysisException(

http://git-wip-us.apache.org/repos/asf/spark/blob/916fc34f/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 14fc37b..33df637 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -621,12 +621,22 @@ class DatasetSuite extends QueryTest with 
SharedSQLContext {
   ds.filter(_ => true),
   Some(1), Some(2), Some(3))
   }
+
+  test("SPARK-13540 Dataset of nested class defined in Scala object") {
+checkAnswer(
+  Seq(OuterObject.InnerClass("foo")).toDS(),
+  OuterObject.InnerClass("foo"))
+  }
 }
 
 class OuterClass extends Serializable {
   case class InnerClass(a: String)
 }
 
+object OuterObject {
+  case class InnerClass(a: String)
+}
+
 case class ClassData(a: String, b: Int)
 case class ClassData2(c: String, d: Int)
 case class ClassNullableData(a: String, b: Integer)


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



spark git commit: [SPARK-13457][SQL] Removes DataFrame RDD operations

2016-02-26 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 5c3912e5c -> 99dfcedbf


[SPARK-13457][SQL] Removes DataFrame RDD operations

## What changes were proposed in this pull request?

This is another try of PR #11323.

This PR removes DataFrame RDD operations except for `foreach` and 
`foreachPartitions` (they are actions rather than transformations). Original 
calls are now replaced by calls to methods of `DataFrame.rdd`.

PR #11323 was reverted because it introduced a regression: both 
`DataFrame.foreach` and `DataFrame.foreachPartitions` wrap underlying RDD 
operations with `withNewExecutionId` to track Spark jobs. But they are removed 
in #11323.

## How was the this patch tested?

No extra tests are added. Existing tests should do the work.

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

Closes #11388 from liancheng/remove-df-rdd-ops.


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

Branch: refs/heads/master
Commit: 99dfcedbfd4c83c7b6a343456f03e8c6e29968c5
Parents: 5c3912e
Author: Cheng Lian <l...@databricks.com>
Authored: Sat Feb 27 00:28:30 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sat Feb 27 00:28:30 2016 +0800

--
 .../spark/examples/ml/DataFrameExample.scala|  2 +-
 .../spark/examples/ml/DecisionTreeExample.scala |  8 +++
 .../spark/examples/ml/OneVsRestExample.scala|  2 +-
 .../spark/examples/mllib/LDAExample.scala   |  1 +
 .../apache/spark/examples/sql/RDDRelation.scala |  2 +-
 .../spark/examples/sql/hive/HiveFromSpark.scala |  2 +-
 .../scala/org/apache/spark/ml/Predictor.scala   |  6 +++--
 .../ml/classification/LogisticRegression.scala  | 13 +++
 .../spark/ml/clustering/BisectingKMeans.scala   |  4 ++--
 .../org/apache/spark/ml/clustering/KMeans.scala |  6 ++---
 .../org/apache/spark/ml/clustering/LDA.scala|  1 +
 .../BinaryClassificationEvaluator.scala |  9 
 .../MulticlassClassificationEvaluator.scala |  6 ++---
 .../ml/evaluation/RegressionEvaluator.scala |  3 ++-
 .../apache/spark/ml/feature/ChiSqSelector.scala |  2 +-
 .../spark/ml/feature/CountVectorizer.scala  |  2 +-
 .../scala/org/apache/spark/ml/feature/IDF.scala |  2 +-
 .../apache/spark/ml/feature/MaxAbsScaler.scala  |  2 +-
 .../apache/spark/ml/feature/MinMaxScaler.scala  |  2 +-
 .../apache/spark/ml/feature/OneHotEncoder.scala |  2 +-
 .../scala/org/apache/spark/ml/feature/PCA.scala |  2 +-
 .../spark/ml/feature/StandardScaler.scala   |  2 +-
 .../apache/spark/ml/feature/StringIndexer.scala |  1 +
 .../apache/spark/ml/feature/VectorIndexer.scala |  2 +-
 .../org/apache/spark/ml/feature/Word2Vec.scala  |  2 +-
 .../apache/spark/ml/recommendation/ALS.scala|  1 +
 .../ml/regression/AFTSurvivalRegression.scala   |  2 +-
 .../ml/regression/IsotonicRegression.scala  |  6 ++---
 .../spark/ml/regression/LinearRegression.scala  | 16 -
 .../spark/mllib/api/python/PythonMLLibAPI.scala |  8 +++
 .../spark/mllib/clustering/KMeansModel.scala|  2 +-
 .../spark/mllib/clustering/LDAModel.scala   |  4 ++--
 .../clustering/PowerIterationClustering.scala   |  2 +-
 .../BinaryClassificationMetrics.scala   |  2 +-
 .../mllib/evaluation/MulticlassMetrics.scala|  2 +-
 .../mllib/evaluation/MultilabelMetrics.scala|  4 +++-
 .../mllib/evaluation/RegressionMetrics.scala|  2 +-
 .../spark/mllib/feature/ChiSqSelector.scala |  2 +-
 .../org/apache/spark/mllib/fpm/FPGrowth.scala   |  2 +-
 .../MatrixFactorizationModel.scala  | 12 +-
 .../mllib/tree/model/DecisionTreeModel.scala|  2 +-
 .../mllib/tree/model/treeEnsembleModels.scala   |  2 +-
 .../LogisticRegressionSuite.scala   |  2 +-
 .../MultilayerPerceptronClassifierSuite.scala   |  5 ++--
 .../ml/classification/OneVsRestSuite.scala  |  6 ++---
 .../ml/clustering/BisectingKMeansSuite.scala|  3 ++-
 .../spark/ml/clustering/KMeansSuite.scala   |  3 ++-
 .../apache/spark/ml/clustering/LDASuite.scala   |  2 +-
 .../spark/ml/feature/OneHotEncoderSuite.scala   |  4 ++--
 .../spark/ml/feature/StringIndexerSuite.scala   |  6 ++---
 .../spark/ml/feature/VectorIndexerSuite.scala   |  5 ++--
 .../apache/spark/ml/feature/Word2VecSuite.scala |  8 +++
 .../spark/ml/recommendation/ALSSuite.scala  |  7 +++---
 .../spark/ml/regression/GBTRegressorSuite.scala |  2 +-
 .../ml/regression/IsotonicRegressionSuite.scala |  6 ++---
 .../ml/regression/LinearRegressionSuite.scala   | 17 +++---
 .../scala/org/apache/spark/sql/DataFrame.scala  | 24 
 .../org/apache/spark/sql/GroupedData.scala  |  1 +
 .../org/apache/spark/sql/api/r/SQLUtils.scala   |  2 +-
 .../spark/sql/DataFrameAggregateSuite.scala

spark git commit: [SPARK-13457][SQL] Removes DataFrame RDD operations

2016-02-25 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 4460113d4 -> 157fe64f3


[SPARK-13457][SQL] Removes DataFrame RDD operations

## What changes were proposed in this pull request?

This PR removes DataFrame RDD operations. Original calls are now replaced by 
calls to methods of `DataFrame.rdd`.

## How was the this patch tested?

No extra tests are added. Existing tests should do the work.

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

Closes #11323 from liancheng/remove-df-rdd-ops.


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

Branch: refs/heads/master
Commit: 157fe64f3ecbd13b7286560286e50235eecfe30e
Parents: 4460113
Author: Cheng Lian <l...@databricks.com>
Authored: Thu Feb 25 23:07:59 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Feb 25 23:07:59 2016 +0800

--
 .../spark/examples/ml/DataFrameExample.scala|  2 +-
 .../spark/examples/ml/DecisionTreeExample.scala |  8 ++--
 .../spark/examples/ml/OneVsRestExample.scala|  2 +-
 .../spark/examples/mllib/LDAExample.scala   |  1 +
 .../apache/spark/examples/sql/RDDRelation.scala |  2 +-
 .../spark/examples/sql/hive/HiveFromSpark.scala |  2 +-
 .../scala/org/apache/spark/ml/Predictor.scala   |  6 ++-
 .../ml/classification/LogisticRegression.scala  | 13 +++---
 .../spark/ml/clustering/BisectingKMeans.scala   |  4 +-
 .../org/apache/spark/ml/clustering/KMeans.scala |  6 +--
 .../org/apache/spark/ml/clustering/LDA.scala|  1 +
 .../BinaryClassificationEvaluator.scala |  9 ++---
 .../MulticlassClassificationEvaluator.scala |  6 +--
 .../ml/evaluation/RegressionEvaluator.scala |  3 +-
 .../apache/spark/ml/feature/ChiSqSelector.scala |  2 +-
 .../spark/ml/feature/CountVectorizer.scala  |  2 +-
 .../scala/org/apache/spark/ml/feature/IDF.scala |  2 +-
 .../apache/spark/ml/feature/MinMaxScaler.scala  |  2 +-
 .../apache/spark/ml/feature/OneHotEncoder.scala |  2 +-
 .../scala/org/apache/spark/ml/feature/PCA.scala |  2 +-
 .../spark/ml/feature/StandardScaler.scala   |  2 +-
 .../apache/spark/ml/feature/StringIndexer.scala |  1 +
 .../apache/spark/ml/feature/VectorIndexer.scala |  2 +-
 .../org/apache/spark/ml/feature/Word2Vec.scala  |  2 +-
 .../apache/spark/ml/recommendation/ALS.scala|  1 +
 .../ml/regression/AFTSurvivalRegression.scala   |  2 +-
 .../ml/regression/IsotonicRegression.scala  |  6 +--
 .../spark/ml/regression/LinearRegression.scala  | 16 +---
 .../spark/mllib/api/python/PythonMLLibAPI.scala |  8 ++--
 .../spark/mllib/clustering/KMeansModel.scala|  2 +-
 .../spark/mllib/clustering/LDAModel.scala   |  4 +-
 .../clustering/PowerIterationClustering.scala   |  2 +-
 .../BinaryClassificationMetrics.scala   |  2 +-
 .../mllib/evaluation/MulticlassMetrics.scala|  2 +-
 .../mllib/evaluation/MultilabelMetrics.scala|  4 +-
 .../mllib/evaluation/RegressionMetrics.scala|  2 +-
 .../spark/mllib/feature/ChiSqSelector.scala |  2 +-
 .../org/apache/spark/mllib/fpm/FPGrowth.scala   |  2 +-
 .../MatrixFactorizationModel.scala  | 12 +++---
 .../mllib/tree/model/DecisionTreeModel.scala|  2 +-
 .../mllib/tree/model/treeEnsembleModels.scala   |  2 +-
 .../LogisticRegressionSuite.scala   |  2 +-
 .../MultilayerPerceptronClassifierSuite.scala   |  5 ++-
 .../ml/classification/OneVsRestSuite.scala  |  6 +--
 .../ml/clustering/BisectingKMeansSuite.scala|  3 +-
 .../spark/ml/clustering/KMeansSuite.scala   |  3 +-
 .../apache/spark/ml/clustering/LDASuite.scala   |  2 +-
 .../spark/ml/feature/OneHotEncoderSuite.scala   |  4 +-
 .../spark/ml/feature/StringIndexerSuite.scala   |  6 +--
 .../spark/ml/feature/VectorIndexerSuite.scala   |  5 ++-
 .../apache/spark/ml/feature/Word2VecSuite.scala |  8 ++--
 .../spark/ml/recommendation/ALSSuite.scala  |  7 ++--
 .../spark/ml/regression/GBTRegressorSuite.scala |  2 +-
 .../ml/regression/IsotonicRegressionSuite.scala |  6 +--
 .../ml/regression/LinearRegressionSuite.scala   | 17 
 .../scala/org/apache/spark/sql/DataFrame.scala  | 42 
 .../org/apache/spark/sql/GroupedData.scala  |  1 +
 .../org/apache/spark/sql/api/r/SQLUtils.scala   |  2 +-
 .../execution/datasources/jdbc/JdbcUtils.scala  |  2 +-
 .../spark/sql/DataFrameAggregateSuite.scala |  2 +-
 .../parquet/ParquetFilterSuite.scala|  2 +-
 .../datasources/parquet/ParquetIOSuite.scala|  2 +-
 .../sql/execution/ui/SQLListenerSuite.scala |  4 +-
 .../spark/sql/hive/HiveSparkSubmitSuite.scala   |  2 +-
 .../sql/hive/execution/HiveQuerySuite.scala |  2 +
 .../spark/sql/hive/orc/OrcQuerySuite.scala  |  5 ++-
 .../apache/spark/sql/hive/parquetSuites.scala   |  2 +-
 67 files changed, 1

[3/3] spark git commit: [SPARK-13486][SQL] Move SQLConf into an internal package

2016-02-25 Thread lian
[SPARK-13486][SQL] Move SQLConf into an internal package

## What changes were proposed in this pull request?
This patch moves SQLConf into org.apache.spark.sql.internal package to make it 
very explicit that it is internal. Soon I will also submit more API work that 
creates implementations of interfaces in this internal package.

## How was this patch tested?
If it compiles, then the refactoring should work.

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

Closes #11363 from rxin/SPARK-13486.


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

Branch: refs/heads/master
Commit: 2b2c8c33236677c916541f956f7b94bba014a9ce
Parents: 07f92ef
Author: Reynold Xin <r...@databricks.com>
Authored: Thu Feb 25 17:49:50 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Feb 25 17:49:50 2016 +0800

--
 project/MimaExcludes.scala  |   6 +
 .../scala/org/apache/spark/sql/DataFrame.scala  |   1 +
 .../org/apache/spark/sql/GroupedData.scala  |   1 +
 .../scala/org/apache/spark/sql/SQLConf.scala| 730 ---
 .../scala/org/apache/spark/sql/SQLContext.scala |   3 +-
 .../spark/sql/execution/ExistingRDD.scala   |   3 +-
 .../spark/sql/execution/SparkStrategies.scala   |   3 +-
 .../apache/spark/sql/execution/commands.scala   |   3 +-
 .../InsertIntoHadoopFsRelation.scala|   1 +
 .../execution/datasources/SqlNewHadoopRDD.scala |   3 +-
 .../execution/datasources/WriterContainer.scala |   1 +
 .../parquet/CatalystSchemaConverter.scala   |   4 +-
 .../parquet/CatalystWriteSupport.scala  |   2 +-
 .../datasources/parquet/ParquetRelation.scala   |   1 +
 .../spark/sql/execution/debug/package.scala |   1 +
 .../execution/local/BinaryHashJoinNode.scala|   2 +-
 .../execution/local/BroadcastHashJoinNode.scala |   2 +-
 .../sql/execution/local/ConvertToSafeNode.scala |   2 +-
 .../execution/local/ConvertToUnsafeNode.scala   |   2 +-
 .../spark/sql/execution/local/ExpandNode.scala  |   2 +-
 .../spark/sql/execution/local/FilterNode.scala  |   2 +-
 .../sql/execution/local/IntersectNode.scala |   4 +-
 .../spark/sql/execution/local/LimitNode.scala   |   2 +-
 .../spark/sql/execution/local/LocalNode.scala   |   3 +-
 .../execution/local/NestedLoopJoinNode.scala|   2 +-
 .../spark/sql/execution/local/ProjectNode.scala |   2 +-
 .../spark/sql/execution/local/SampleNode.scala  |   2 +-
 .../spark/sql/execution/local/SeqScanNode.scala |   2 +-
 .../local/TakeOrderedAndProjectNode.scala   |   2 +-
 .../spark/sql/execution/local/UnionNode.scala   |   2 +-
 .../org/apache/spark/sql/internal/SQLConf.scala | 730 +++
 .../apache/spark/sql/internal/package-info.java |  22 +
 .../org/apache/spark/sql/internal/package.scala |  24 +
 .../spark/sql/DataFrameAggregateSuite.scala |   1 +
 .../apache/spark/sql/DataFramePivotSuite.scala  |   1 +
 .../org/apache/spark/sql/DataFrameSuite.scala   |   1 +
 .../scala/org/apache/spark/sql/JoinSuite.scala  |   1 +
 .../spark/sql/MultiSQLContextsSuite.scala   |   1 +
 .../scala/org/apache/spark/sql/QueryTest.scala  |   1 +
 .../apache/spark/sql/SQLConfEntrySuite.scala| 150 
 .../org/apache/spark/sql/SQLConfSuite.scala | 132 
 .../org/apache/spark/sql/SQLContextSuite.scala  |   1 +
 .../org/apache/spark/sql/SQLQuerySuite.scala|   1 +
 .../execution/ExchangeCoordinatorSuite.scala|   1 +
 .../spark/sql/execution/PlannerSuite.scala  |   3 +-
 .../columnar/PartitionBatchPruningSuite.scala   |   1 +
 .../execution/datasources/json/JsonSuite.scala  |   1 +
 .../parquet/ParquetFilterSuite.scala|   1 +
 .../datasources/parquet/ParquetIOSuite.scala|   1 +
 .../ParquetPartitionDiscoverySuite.scala|   1 +
 .../datasources/parquet/ParquetQuerySuite.scala |   1 +
 .../parquet/ParquetReadBenchmark.scala  |   3 +-
 .../datasources/parquet/ParquetTest.scala   |   3 +-
 .../sql/execution/joins/InnerJoinSuite.scala|   3 +-
 .../sql/execution/joins/OuterJoinSuite.scala|   3 +-
 .../sql/execution/joins/SemiJoinSuite.scala |   3 +-
 .../spark/sql/execution/local/DummyNode.scala   |   2 +-
 .../sql/execution/local/HashJoinNodeSuite.scala |   2 +-
 .../sql/execution/local/LocalNodeTest.scala |   2 +-
 .../local/NestedLoopJoinNodeSuite.scala |   2 +-
 .../sql/execution/metric/SQLMetricsSuite.scala  |   1 +
 .../spark/sql/internal/SQLConfEntrySuite.scala  | 150 
 .../spark/sql/internal/SQLConfSuite.scala   | 133 
 .../spark/sql/sources/DataSourceTest.scala  |   1 +
 .../spark/sql/sources/FilteredScanSuite.scala   |   1 +
 .../spark/sql/sources/PrunedScanSuite.scala |   1 +
 .../spark/sql/sources/SaveLoadSuite

[1/3] spark git commit: [SPARK-13486][SQL] Move SQLConf into an internal package

2016-02-25 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 07f92ef1f -> 2b2c8c332


http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
index c89a151..28ad7ae 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala
@@ -18,8 +18,8 @@
 package org.apache.spark.sql.test
 
 import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.sql.{SQLConf, SQLContext}
-
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * A special [[SQLContext]] prepared for testing.
@@ -39,7 +39,7 @@ private[sql] class TestSQLContext(sc: SparkContext) extends 
SQLContext(sc) { sel
   super.clear()
 
   // Make sure we start with the default test configs even after clear
-  TestSQLContext.overrideConfs.map {
+  TestSQLContext.overrideConfs.foreach {
 case (key, value) => setConfString(key, value)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
index 66eaa3e..f32ba5f 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
@@ -32,10 +32,10 @@ import org.apache.hive.service.server.{HiveServer2, 
HiveServerServerOptionsProce
 import org.apache.spark.{Logging, SparkContext}
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, 
SparkListenerJobStart}
-import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
 import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.util.{ShutdownHookManager, Utils}
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index 8fef22c..458d4f2 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -33,9 +33,10 @@ import 
org.apache.hive.service.cli.operation.ExecuteStatementOperation
 import org.apache.hive.service.cli.session.HiveSession
 
 import org.apache.spark.Logging
-import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf}
+import org.apache.spark.sql.{DataFrame, Row => SparkRow}
 import org.apache.spark.sql.execution.SetCommand
 import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 import org.apache.spark.util.{Utils => SparkUtils}
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2b2c8c33/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
--
diff --git 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index 5f9952a..c05527b 100644
--- 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -202,7 +202,7 @@ class HiveThriftBinaryServerSuite extends 
HiveThriftJdbcTest {
   }
 
   test("test multiple session") {
-import org.apache.spark.sql.SQLConf
+import org.apache.spark.sql.internal.SQLConf
 var defaultV1: String = null
 var defaultV2: String = null
 var data: ArrayBuffer[Int] = null


spark git commit: [SPARK-13263][SQL] SQL Generation Support for Tablesample

2016-02-23 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 5cd3e6f60 -> 87250580f


[SPARK-13263][SQL] SQL Generation Support for Tablesample

In the parser, tableSample clause is part of tableSource.
```
tableSource
init { gParent.pushMsg("table source", state); }
after { gParent.popMsg(state); }
: tabname=tableName
((tableProperties) => props=tableProperties)?
((tableSample) => ts=tableSample)?
((KW_AS) => (KW_AS alias=Identifier)
|
(Identifier) => (alias=Identifier))?
-> ^(TOK_TABREF $tabname $props? $ts? $alias?)
;
```

Two typical query samples using TABLESAMPLE are:
```
"SELECT s.id FROM t0 TABLESAMPLE(10 PERCENT) s"
"SELECT * FROM t0 TABLESAMPLE(0.1 PERCENT)"
```

FYI, the logical plan of a TABLESAMPLE query:
```
sql("SELECT * FROM t0 TABLESAMPLE(0.1 PERCENT)").explain(true)

== Analyzed Logical Plan ==
id: bigint
Project [id#16L]
+- Sample 0.0, 0.001, false, 381
   +- Subquery t0
  +- Relation[id#16L] ParquetRelation
```

Thanks! cc liancheng

Author: gatorsmile <gatorsm...@gmail.com>
Author: xiaoli <lixiao1...@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

This patch had conflicts when merged, resolved by
Committer: Cheng Lian <l...@databricks.com>

Closes #11148 from gatorsmile/tablesplitsample.


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

Branch: refs/heads/master
Commit: 87250580f214cb7c4dff01c5a3498ea6cb79a27e
Parents: 5cd3e6f
Author: gatorsmile <gatorsm...@gmail.com>
Authored: Tue Feb 23 16:13:09 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Feb 23 16:13:09 2016 +0800

--
 .../apache/spark/sql/catalyst/CatalystQl.scala  |  6 +-
 .../sql/catalyst/optimizer/Optimizer.scala  |  2 +-
 .../catalyst/plans/logical/basicOperators.scala |  5 +-
 .../optimizer/FilterPushdownSuite.scala |  4 +-
 .../scala/org/apache/spark/sql/DataFrame.scala  |  4 +-
 .../scala/org/apache/spark/sql/Dataset.scala|  2 +-
 .../org/apache/spark/sql/hive/SQLBuilder.scala  | 18 
 .../spark/sql/hive/LogicalPlanToSQLSuite.scala  | 98 ++--
 8 files changed, 103 insertions(+), 36 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/87250580/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala
index 069c665..a0a56d7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala
@@ -499,12 +499,14 @@ 
https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
   s"Sampling fraction ($fraction) must be on interval [0, 100]")
 Sample(0.0, fraction.toDouble / 100, withReplacement = false,
   (math.random * 1000).toInt,
-  relation)
+  relation)(
+  isTableSample = true)
   case Token("TOK_TABLEBUCKETSAMPLE",
   Token(numerator, Nil) ::
 Token(denominator, Nil) :: Nil) =>
 val fraction = numerator.toDouble / denominator.toDouble
-Sample(0.0, fraction, withReplacement = false, (math.random * 
1000).toInt, relation)
+Sample(0.0, fraction, withReplacement = false, (math.random * 
1000).toInt, relation)(
+  isTableSample = true)
   case a =>
 noParseRule("Sampling", a)
 }.getOrElse(relation)

http://git-wip-us.apache.org/repos/asf/spark/blob/87250580/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 1554382..1f05f20 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
@@ -123,7 +123,7 @@ object SamplePushDown extends Rule[LogicalPlan] {
 // Push down projection into sample
 case Project(projectList, s @ Sample(lb, up, replace, seed, child)) =>
   Sample(lb, up, replace, seed,
-Project(projectList, child))
+Project(projectL

[1/2] spark git commit: [SPARK-12799] Simplify various string output for expressions

2016-02-21 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master d806ed343 -> d9efe63ec


http://git-wip-us.apache.org/repos/asf/spark/blob/d9efe63e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala
index 1c0d53f..54dda0c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala
@@ -544,7 +544,7 @@ private[parquet] object CatalystSchemaConverter {
   !name.matches(".*[ ,;{}()\n\t=].*"),
   s"""Attribute name "$name" contains invalid character(s) among " 
,;{}()\\n\\t=".
  |Please use alias to rename it.
-   """.stripMargin.split("\n").mkString(" "))
+   """.stripMargin.split("\n").mkString(" ").trim)
   }
 
   def checkFieldNames(schema: StructType): StructType = {

http://git-wip-us.apache.org/repos/asf/spark/blob/d9efe63e/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala
index 0e53a0c..9aff0be 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.python
 import org.apache.spark.{Accumulator, Logging}
 import org.apache.spark.api.python.PythonBroadcast
 import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.sql.catalyst.expressions.{Expression, Unevaluable}
+import org.apache.spark.sql.catalyst.expressions.{Expression, 
NonSQLExpression, Unevaluable}
 import org.apache.spark.sql.types.DataType
 
 /**
@@ -36,9 +36,12 @@ case class PythonUDF(
 broadcastVars: java.util.List[Broadcast[PythonBroadcast]],
 accumulator: Accumulator[java.util.List[Array[Byte]]],
 dataType: DataType,
-children: Seq[Expression]) extends Expression with Unevaluable with 
Logging {
+children: Seq[Expression])
+  extends Expression with Unevaluable with NonSQLExpression with Logging {
 
-  override def toString: String = s"PythonUDF#$name(${children.mkString(",")})"
+  override def toString: String = s"PythonUDF#$name(${children.mkString(", 
")})"
 
   override def nullable: Boolean = true
+
+  override def sql: String = s"$name(${children.mkString(", ")})"
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d9efe63e/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index f9ba607..498f007 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -525,7 +525,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
 val e = intercept[AnalysisException] {
   ds.as[ClassData2]
 }
-assert(e.getMessage.contains("cannot resolve 'c' given input columns: [a, 
b]"), e.getMessage)
+assert(e.getMessage.contains("cannot resolve '`c`' given input columns: 
[a, b]"), e.getMessage)
   }
 
   test("runtime nullability check") {

http://git-wip-us.apache.org/repos/asf/spark/blob/d9efe63e/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
index bd87449..41a9404 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
@@ -455,7 +455,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest 
with SharedSQLContext {
   sqlContext.udf.register("div0", (x: Int) => x / 0)
   withTempPath { dir =>
 intercept[org.apache.spark.SparkException] {
-  sqlContext.sql("select div0(1)").write.parquet(dir.getCanonicalPath)
+  sqlContext.sql("select div0(1) as 
div0").write.parquet(dir.getCanonicalPath)
 }
 val path = new Path(dir.getCanonicalPath, "_temporary")
 val fs = 

spark git commit: [SPARK-13205][SQL] SQL Generation Support for Self Join

2016-02-10 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 663cc400f -> 0f09f0226


[SPARK-13205][SQL] SQL Generation Support for Self Join

This PR addresses two issues:
  - Self join does not work in SQL Generation
  - When creating new instances for `LogicalRelation`, 
`metastoreTableIdentifier` is lost.

liancheng Could you please review the code changes? Thank you!

Author: gatorsmile <gatorsm...@gmail.com>

Closes #11084 from gatorsmile/selfJoinInSQLGen.


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

Branch: refs/heads/master
Commit: 0f09f0226983cdc409ef504dff48395787dc844f
Parents: 663cc40
Author: gatorsmile <gatorsm...@gmail.com>
Authored: Thu Feb 11 11:08:21 2016 +0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Feb 11 11:08:21 2016 +0800

--
 .../spark/sql/execution/datasources/LogicalRelation.scala |  6 +-
 .../main/scala/org/apache/spark/sql/hive/SQLBuilder.scala | 10 +-
 .../org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala |  8 
 3 files changed, 22 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0f09f022/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
index fa97f3d..0e0748f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
@@ -76,7 +76,11 @@ case class LogicalRelation(
   /** Used to lookup original attribute capitalization */
   val attributeMap: AttributeMap[AttributeReference] = 
AttributeMap(output.map(o => (o, o)))
 
-  def newInstance(): this.type = 
LogicalRelation(relation).asInstanceOf[this.type]
+  def newInstance(): this.type =
+LogicalRelation(
+  relation,
+  expectedOutputAttributes,
+  metastoreTableIdentifier).asInstanceOf[this.type]
 
   override def simpleString: String = s"Relation[${output.mkString(",")}] 
$relation"
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0f09f022/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
--
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
index fc5725d..4b75e60 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala
@@ -142,7 +142,15 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: 
SQLContext) extends Loggi
   Some(s"`$database`.`$table`")
 
 case Subquery(alias, child) =>
-  toSQL(child).map(childSQL => s"($childSQL) AS $alias")
+  toSQL(child).map( childSQL =>
+child match {
+  // Parentheses is not used for persisted data source relations
+  // e.g., select x.c1 from (t1) as x inner join (t1) as y on x.c1 = 
y.c1
+  case Subquery(_, _: LogicalRelation | _: MetastoreRelation) =>
+s"$childSQL AS $alias"
+  case _ =>
+s"($childSQL) AS $alias"
+})
 
 case Join(left, right, joinType, condition) =>
   for {

http://git-wip-us.apache.org/repos/asf/spark/blob/0f09f022/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
index 129bfe0..80ae312 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
@@ -104,6 +104,14 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with 
SQLTestUtils {
 checkHiveQl("SELECT id FROM t0 UNION ALL SELECT CAST(id AS INT) AS id FROM 
t0")
   }
 
+  test("self join") {
+checkHiveQl("SELECT x.key FROM t1 x JOIN t1 y ON x.key = y.key")
+  }
+
+  test("self join with group by") {
+checkHiveQl("SELECT x.key, COUNT(*) FROM t1 x JOIN t1 y ON x.key = y.key 

spark git commit: [SPARK-12231][SQL] create a combineFilters' projection when we call buildPartitionedTableScan

2016-01-31 Thread lian
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 bb01cbe9b -> ddb963304


[SPARK-12231][SQL] create a combineFilters' projection when we call 
buildPartitionedTableScan

Hello Michael & All:

We have some issues to submit the new codes in the other PR(#10299), so we 
closed that PR and open this one with the fix.

The reason for the previous failure is that the projection for the scan when 
there is a filter that is not pushed down (the "left-over" filter) could be 
different, in elements or ordering, from the original projection.

With this new codes, the approach to solve this problem is:

Insert a new Project if the "left-over" filter is nonempty and (the original 
projection is not empty and the projection for the scan has more than one 
elements which could otherwise cause different ordering in projection).

We create 3 test cases to cover the otherwise failure cases.

Author: Kevin Yu <q...@us.ibm.com>

Closes #10388 from kevinyu98/spark-12231.

(cherry picked from commit fd50df413fbb3b7528cdff311cc040a6212340b9)
Signed-off-by: Cheng Lian <l...@databricks.com>


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

Branch: refs/heads/branch-1.6
Commit: ddb9633043e82fb2a34c7e0e29b487f635c3c744
Parents: bb01cbe
Author: Kevin Yu <q...@us.ibm.com>
Authored: Mon Dec 28 11:58:33 2015 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Sun Jan 31 16:16:34 2016 -0800

--
 .../datasources/DataSourceStrategy.scala| 28 ++---
 .../parquet/ParquetFilterSuite.scala| 41 
 2 files changed, 64 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ddb96330/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
index 8a15a51..3741a9c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
@@ -77,7 +77,8 @@ private[sql] object DataSourceStrategy extends Strategy with 
Logging {
   val pushedFilters = 
filters.filter(_.references.intersect(partitionColumns).isEmpty)
 
   // Predicates with both partition keys and attributes
-  val combineFilters = filters.toSet -- partitionFilters.toSet -- 
pushedFilters.toSet
+  val partitionAndNormalColumnFilters =
+filters.toSet -- partitionFilters.toSet -- pushedFilters.toSet
 
   val selectedPartitions = prunePartitions(partitionFilters, 
t.partitionSpec).toArray
 
@@ -88,16 +89,33 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
 s"Selected $selected partitions out of $total, pruned $percentPruned% 
partitions."
   }
 
+  // need to add projections from "partitionAndNormalColumnAttrs" in if it 
is not empty
+  val partitionAndNormalColumnAttrs = 
AttributeSet(partitionAndNormalColumnFilters)
+  val partitionAndNormalColumnProjs = if 
(partitionAndNormalColumnAttrs.isEmpty) {
+projects
+  } else {
+(partitionAndNormalColumnAttrs ++ projects).toSeq
+  }
+
   val scan = buildPartitionedTableScan(
 l,
-projects,
+partitionAndNormalColumnProjs,
 pushedFilters,
 t.partitionSpec.partitionColumns,
 selectedPartitions)
 
-  combineFilters
-.reduceLeftOption(expressions.And)
-.map(execution.Filter(_, scan)).getOrElse(scan) :: Nil
+  // Add a Projection to guarantee the original projection:
+  // this is because "partitionAndNormalColumnAttrs" may be different
+  // from the original "projects", in elements or their ordering
+
+  partitionAndNormalColumnFilters.reduceLeftOption(expressions.And).map(cf 
=>
+if (projects.isEmpty || projects == partitionAndNormalColumnProjs) {
+  // if the original projection is empty, no need for the additional 
Project either
+  execution.Filter(cf, scan)
+} else {
+  execution.Project(projects, execution.Filter(cf, scan))
+}
+  ).getOrElse(scan) :: Nil
 
 // Scanning non-partitioned HadoopFsRelation
 case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
HadoopFsRelation, _)) =>

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

spark git commit: [SPARK-12818] Polishes spark-sketch module

2016-01-29 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 5f686cc8b -> 2b027e9a3


[SPARK-12818] Polishes spark-sketch module

Fixes various minor code and Javadoc styling issues.

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

Closes #10985 from liancheng/sketch-polishing.


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

Branch: refs/heads/master
Commit: 2b027e9a386fe4009f61ad03b169335af5a9a5c6
Parents: 5f686cc
Author: Cheng Lian <l...@databricks.com>
Authored: Fri Jan 29 12:01:13 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Fri Jan 29 12:01:13 2016 -0800

--
 .../org/apache/spark/util/sketch/BitArray.java  |   2 +-
 .../apache/spark/util/sketch/BloomFilter.java   | 111 ++-
 .../spark/util/sketch/BloomFilterImpl.java  |  40 +++
 .../spark/util/sketch/CountMinSketch.java   |  26 +++--
 .../spark/util/sketch/CountMinSketchImpl.java   |  12 ++
 .../org/apache/spark/util/sketch/Utils.java |   2 +-
 6 files changed, 110 insertions(+), 83 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2b027e9a/common/sketch/src/main/java/org/apache/spark/util/sketch/BitArray.java
--
diff --git 
a/common/sketch/src/main/java/org/apache/spark/util/sketch/BitArray.java 
b/common/sketch/src/main/java/org/apache/spark/util/sketch/BitArray.java
index 2a0484e..480a0a7 100644
--- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BitArray.java
+++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BitArray.java
@@ -22,7 +22,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 
-public final class BitArray {
+final class BitArray {
   private final long[] data;
   private long bitCount;
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2b027e9a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
--
diff --git 
a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java 
b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
index 81772fc..c0b425e 100644
--- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
+++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
@@ -22,16 +22,10 @@ import java.io.InputStream;
 import java.io.OutputStream;
 
 /**
- * A Bloom filter is a space-efficient probabilistic data structure, that is 
used to test whether
- * an element is a member of a set. It returns false when the element is 
definitely not in the
- * set, returns true when the element is probably in the set.
- *
- * Internally a Bloom filter is initialized with 2 information: how many space 
to use(number of
- * bits) and how many hash values to calculate for each record.  To get as 
lower false positive
- * probability as possible, user should call {@link BloomFilter#create} to 
automatically pick a
- * best combination of these 2 parameters.
- *
- * Currently the following data types are supported:
+ * A Bloom filter is a space-efficient probabilistic data structure that 
offers an approximate
+ * containment test with one-sided error: if it claims that an item is 
contained in it, this
+ * might be in error, but if it claims that an item is not contained in 
it, then this is
+ * definitely true. Currently supported data types include:
  * 
  *   {@link Byte}
  *   {@link Short}
@@ -39,14 +33,17 @@ import java.io.OutputStream;
  *   {@link Long}
  *   {@link String}
  * 
+ * The false positive probability ({@code FPP}) of a Bloom filter is defined 
as the probability that
+ * {@linkplain #mightContain(Object)} will erroneously return {@code true} for 
an object that hasu
+ * not actually been put in the {@code BloomFilter}.
  *
- * The implementation is largely based on the {@code BloomFilter} class from 
guava.
+ * The implementation is largely based on the {@code BloomFilter} class from 
Guava.
  */
 public abstract class BloomFilter {
 
   public enum Version {
 /**
- * {@code BloomFilter} binary format version 1 (all values written in 
big-endian order):
+ * {@code BloomFilter} binary format version 1. All values written in 
big-endian order:
  * 
  *   Version number, always 1 (32 bit)
  *   Number of hash functions (32 bit)
@@ -68,14 +65,13 @@ public abstract class BloomFilter {
   }
 
   /**
-   * Returns the false positive probability, i.e. the probability that
-   * {@linkplain #mightContain(Object)} will erroneously return {@code true} 
for an object that
-   * has not actually b

spark git commit: [SPARK-13050][BUILD] Scalatest tags fail build with the addition of the sketch module

2016-01-28 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 721ced28b -> 8d3cc3de7


[SPARK-13050][BUILD] Scalatest tags fail build with the addition of the sketch 
module

A dependency on the spark test tags was left out of the sketch module pom file 
causing builds to fail when test tags were used. This dependency is found in 
the pom file for every other module in spark.

Author: Alex Bozarth <ajboz...@us.ibm.com>

Closes #10954 from ajbozarth/spark13050.


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

Branch: refs/heads/master
Commit: 8d3cc3de7d116190911e7943ef3233fe3b7db1bf
Parents: 721ced2
Author: Alex Bozarth <ajboz...@us.ibm.com>
Authored: Thu Jan 28 23:34:50 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jan 28 23:34:50 2016 -0800

--
 common/sketch/pom.xml | 7 +++
 1 file changed, 7 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8d3cc3de/common/sketch/pom.xml
--
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 67723fa..2cafe8c 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -35,6 +35,13 @@
 sketch
   
 
+  
+
+  org.apache.spark
+  spark-test-tags_${scala.binary.version}
+
+  
+
   
 
target/scala-${scala.binary.version}/classes
 
target/scala-${scala.binary.version}/test-classes


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



spark git commit: [SPARK-12401][SQL] Add integration tests for postgres enum types

2016-01-28 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master df78a934a -> abae889f0


[SPARK-12401][SQL] Add integration tests for postgres enum types

We can handle posgresql-specific enum types as strings in jdbc.
So, we should just add tests and close the corresponding JIRA ticket.

Author: Takeshi YAMAMURO <linguin@gmail.com>

Closes #10596 from maropu/AddTestsInIntegration.


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

Branch: refs/heads/master
Commit: abae889f08eb412cb897e4e63614ec2c93885ffd
Parents: df78a93
Author: Takeshi YAMAMURO <linguin@gmail.com>
Authored: Thu Jan 28 15:20:16 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jan 28 15:20:16 2016 -0800

--
 .../spark/sql/jdbc/PostgresIntegrationSuite.scala| 15 +--
 1 file changed, 9 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/abae889f/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala
--
diff --git 
a/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala
 
b/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala
index 7d011be..72bda8f 100644
--- 
a/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala
+++ 
b/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala
@@ -21,7 +21,7 @@ import java.sql.Connection
 import java.util.Properties
 
 import org.apache.spark.sql.Column
-import org.apache.spark.sql.catalyst.expressions.{If, Literal}
+import org.apache.spark.sql.catalyst.expressions.Literal
 import org.apache.spark.tags.DockerTest
 
 @DockerTest
@@ -39,12 +39,13 @@ class PostgresIntegrationSuite extends 
DockerJDBCIntegrationSuite {
   override def dataPreparation(conn: Connection): Unit = {
 conn.prepareStatement("CREATE DATABASE foo").executeUpdate()
 conn.setCatalog("foo")
+conn.prepareStatement("CREATE TYPE enum_type AS ENUM ('d1', 
'd2')").executeUpdate()
 conn.prepareStatement("CREATE TABLE bar (c0 text, c1 integer, c2 double 
precision, c3 bigint, "
   + "c4 bit(1), c5 bit(10), c6 bytea, c7 boolean, c8 inet, c9 cidr, "
-  + "c10 integer[], c11 text[], c12 real[])").executeUpdate()
+  + "c10 integer[], c11 text[], c12 real[], c13 
enum_type)").executeUpdate()
 conn.prepareStatement("INSERT INTO bar VALUES ('hello', 42, 1.25, 
123456789012345, B'0', "
   + "B'1000100101', E'xDEADBEEF', true, '172.16.0.42', 
'192.168.0.0/16', "
-  + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}')""").executeUpdate()
+  + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', 
'd1')""").executeUpdate()
   }
 
   test("Type mapping for various types") {
@@ -52,7 +53,7 @@ class PostgresIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 val rows = df.collect()
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass)
-assert(types.length == 13)
+assert(types.length == 14)
 assert(classOf[String].isAssignableFrom(types(0)))
 assert(classOf[java.lang.Integer].isAssignableFrom(types(1)))
 assert(classOf[java.lang.Double].isAssignableFrom(types(2)))
@@ -66,22 +67,24 @@ class PostgresIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 assert(classOf[Seq[Int]].isAssignableFrom(types(10)))
 assert(classOf[Seq[String]].isAssignableFrom(types(11)))
 assert(classOf[Seq[Double]].isAssignableFrom(types(12)))
+assert(classOf[String].isAssignableFrom(types(13)))
 assert(rows(0).getString(0).equals("hello"))
 assert(rows(0).getInt(1) == 42)
 assert(rows(0).getDouble(2) == 1.25)
 assert(rows(0).getLong(3) == 123456789012345L)
-assert(rows(0).getBoolean(4) == false)
+assert(!rows(0).getBoolean(4))
 // BIT(10)'s come back as ASCII strings of ten ASCII 0's and 1's...
 assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](5),
   Array[Byte](49, 48, 48, 48, 49, 48, 48, 49, 48, 49)))
 assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6),
   Array[Byte](0xDE.toByte, 0xAD.toByte, 0xBE.toByte, 0xEF.toByte)))
-assert(rows(0).getBoolean(7) == true)
+assert(rows(0).getBoolean(7))
 assert(rows(0).getString(8) == "172.16.0.42")
 assert(rows(0).getString(9) == &qu

spark git commit: [SPARK-11955][SQL] Mark optional fields in merging schema for safely pushdowning filters in Parquet

2016-01-28 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 3a40c0e57 -> 4637fc08a


[SPARK-11955][SQL] Mark optional fields in merging schema for safely 
pushdowning filters in Parquet

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

Currently we simply skip pushdowning filters in parquet if we enable schema 
merging.

However, we can actually mark particular fields in merging schema for safely 
pushdowning filters in parquet.

Author: Liang-Chi Hsieh <vii...@appier.com>
Author: Liang-Chi Hsieh <vii...@gmail.com>

Closes #9940 from viirya/safe-pushdown-parquet-filters.


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

Branch: refs/heads/master
Commit: 4637fc08a3733ec313218fb7e4d05064d9a6262d
Parents: 3a40c0e
Author: Liang-Chi Hsieh <vii...@appier.com>
Authored: Thu Jan 28 16:25:21 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Thu Jan 28 16:25:21 2016 -0800

--
 .../org/apache/spark/sql/types/Metadata.scala   |  5 +++
 .../org/apache/spark/sql/types/StructType.scala | 34 +---
 .../apache/spark/sql/types/DataTypeSuite.scala  | 14 +--
 .../datasources/parquet/ParquetFilters.scala| 37 -
 .../datasources/parquet/ParquetRelation.scala   | 13 +++---
 .../parquet/ParquetFilterSuite.scala| 43 +++-
 6 files changed, 117 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4637fc08/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala
index 9e0f994..66f1236 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala
@@ -273,4 +273,9 @@ class MetadataBuilder {
 map.put(key, value)
 this
   }
+
+  def remove(key: String): this.type = {
+map.remove(key)
+this
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4637fc08/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
index 3bd733f..da0c928 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
@@ -334,6 +334,8 @@ case class StructType(fields: Array[StructField]) extends 
DataType with Seq[Stru
 
 object StructType extends AbstractDataType {
 
+  private[sql] val metadataKeyForOptionalField = "_OPTIONAL_"
+
   override private[sql] def defaultConcreteType: DataType = new StructType
 
   override private[sql] def acceptsType(other: DataType): Boolean = {
@@ -359,6 +361,18 @@ object StructType extends AbstractDataType {
   protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType =
 StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, 
a.metadata)))
 
+  def removeMetadata(key: String, dt: DataType): DataType =
+dt match {
+  case StructType(fields) =>
+val newFields = fields.map { f =>
+  val mb = new MetadataBuilder()
+  f.copy(dataType = removeMetadata(key, f.dataType),
+metadata = mb.withMetadata(f.metadata).remove(key).build())
+}
+StructType(newFields)
+  case _ => dt
+}
+
   private[sql] def merge(left: DataType, right: DataType): DataType =
 (left, right) match {
   case (ArrayType(leftElementType, leftContainsNull),
@@ -376,24 +390,32 @@ object StructType extends AbstractDataType {
 
   case (StructType(leftFields), StructType(rightFields)) =>
 val newFields = ArrayBuffer.empty[StructField]
+// This metadata will record the fields that only exist in one of two 
StructTypes
+val optionalMeta = new MetadataBuilder()
 
 val rightMapped = fieldsMap(rightFields)
 leftFields.foreach {
   case leftField @ StructField(leftName, leftType, leftNullable, _) =>
 rightMapped.get(leftName)
   .map { case rightField @ StructField(_, rightType, 
rightNullable, _) =>
-  leftField.copy(
-dataType = merge(leftType, rightType),
-nullable = leftNullable || rightNullable)
-}
-

spark git commit: [SQL] Minor Scaladoc format fix

2016-01-26 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master ee74498de -> 83507fea9


[SQL] Minor Scaladoc format fix

Otherwise the `^` character is always marked as error in IntelliJ since it 
represents an unclosed superscript markup tag.

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

Closes #10926 from liancheng/agg-doc-fix.


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

Branch: refs/heads/master
Commit: 83507fea9f45c336d73dd4795b8cb37bcd63e31d
Parents: ee74498
Author: Cheng Lian <l...@databricks.com>
Authored: Tue Jan 26 14:29:29 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Jan 26 14:29:29 2016 -0800

--
 .../sql/catalyst/expressions/aggregate/interfaces.scala  | 8 
 1 file changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/83507fea/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
index ddd99c5..561fa33 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
@@ -200,7 +200,7 @@ abstract class ImperativeAggregate extends 
AggregateFunction with CodegenFallbac
* For example, we have two aggregate functions `avg(x)` and `avg(y)`, which 
share the same
* aggregation buffer. In this shared buffer, the position of the first 
buffer value of `avg(x)`
* will be 0 and the position of the first buffer value of `avg(y)` will be 
2:
-   *
+   * {{{
*  avg(x) mutableAggBufferOffset = 0
*  |
*  v
@@ -210,7 +210,7 @@ abstract class ImperativeAggregate extends 
AggregateFunction with CodegenFallbac
*^
*|
* avg(y) mutableAggBufferOffset = 2
-   *
+   * }}}
*/
   protected val mutableAggBufferOffset: Int
 
@@ -233,7 +233,7 @@ abstract class ImperativeAggregate extends 
AggregateFunction with CodegenFallbac
* `avg(x)` and `avg(y)`. In the shared input aggregation buffer, the 
position of the first
* buffer value of `avg(x)` will be 1 and the position of the first buffer 
value of `avg(y)`
* will be 3 (position 0 is used for the value of `key`):
-   *
+   * {{{
*  avg(x) inputAggBufferOffset = 1
*   |
*   v
@@ -243,7 +243,7 @@ abstract class ImperativeAggregate extends 
AggregateFunction with CodegenFallbac
* ^
* |
*   avg(y) inputAggBufferOffset = 3
-   *
+   * }}}
*/
   protected val inputAggBufferOffset: Int
 


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



spark git commit: [SPARK-12934] use try-with-resources for streams

2016-01-25 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 109061f7a -> fdcc3512f


[SPARK-12934] use try-with-resources for streams

liancheng please take a look

Author: tedyu <yuzhih...@gmail.com>

Closes #10906 from tedyu/master.


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

Branch: refs/heads/master
Commit: fdcc3512f7b45e5b067fc26cb05146f79c4a5177
Parents: 109061f
Author: tedyu <yuzhih...@gmail.com>
Authored: Mon Jan 25 18:23:47 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon Jan 25 18:23:47 2016 -0800

--
 .../src/main/java/org/apache/spark/util/sketch/CountMinSketch.java | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fdcc3512/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java
--
diff --git 
a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java 
b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java
index 6793864..9f4ff42 100644
--- 
a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java
+++ 
b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java
@@ -128,11 +128,13 @@ abstract public class CountMinSketch {
 
   /**
* Writes out this {@link CountMinSketch} to an output stream in binary 
format.
+   * It is the caller's responsibility to close the stream
*/
   public abstract void writeTo(OutputStream out) throws IOException;
 
   /**
* Reads in a {@link CountMinSketch} from an input stream.
+   * It is the caller's responsibility to close the stream
*/
   public static CountMinSketch readFrom(InputStream in) throws IOException {
 return CountMinSketchImpl.readFrom(in);


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



spark git commit: [SPARK-12560][SQL] SqlTestUtils.stripSparkFilter needs to copy utf8strings

2016-01-19 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master b72e01e82 -> 4dbd31612


[SPARK-12560][SQL] SqlTestUtils.stripSparkFilter needs to copy utf8strings

See https://issues.apache.org/jira/browse/SPARK-12560

This isn't causing any problems currently because the tests for string 
predicate pushdown are currently disabled.  I ran into this while trying to 
turn them back on with a different version of parquet.  Figure it was good to 
fix now in any case.

Author: Imran Rashid <iras...@cloudera.com>

Closes #10510 from squito/SPARK-12560.


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

Branch: refs/heads/master
Commit: 4dbd3161227a32736105cef624f9df21650a359c
Parents: b72e01e
Author: Imran Rashid <iras...@cloudera.com>
Authored: Tue Jan 19 12:24:21 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Jan 19 12:24:21 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/spark/blob/4dbd3161/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
index 7df344e..5f73d71 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
@@ -189,7 +189,7 @@ private[sql] trait SQLTestUtils
   .executedPlan.asInstanceOf[org.apache.spark.sql.execution.Filter]
   .child
   .execute()
-  .map(row => Row.fromSeq(row.toSeq(schema)))
+  .map(row => Row.fromSeq(row.copy().toSeq(schema)))
 
 sqlContext.createDataFrame(childRDD, schema)
   }


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



spark git commit: [SPARK-12867][SQL] Nullability of Intersect can be stricter

2016-01-19 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 2388de519 -> b72e01e82


[SPARK-12867][SQL] Nullability of Intersect can be stricter

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

When intersecting one nullable column with one non-nullable column, the result 
will not contain any null. Thus, we can make nullability of `intersect` 
stricter.

liancheng Could you please check if the code changes are appropriate? Also 
added test cases to verify the results. Thanks!

Author: gatorsmile <gatorsm...@gmail.com>

Closes #10812 from gatorsmile/nullabilityIntersect.


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

Branch: refs/heads/master
Commit: b72e01e82148a908eb19bb3f526f9777bfe27dde
Parents: 2388de5
Author: gatorsmile <gatorsm...@gmail.com>
Authored: Tue Jan 19 11:35:58 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Jan 19 11:35:58 2016 -0800

--
 .../catalyst/plans/logical/basicOperators.scala | 18 +++--
 .../org/apache/spark/sql/DataFrameSuite.scala   | 21 
 2 files changed, 33 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b72e01e8/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 2a1b1b1..f4a3d85 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
@@ -91,11 +91,6 @@ case class Filter(condition: Expression, child: LogicalPlan) 
extends UnaryNode {
 }
 
 abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends 
BinaryNode {
-  override def output: Seq[Attribute] =
-left.output.zip(right.output).map { case (leftAttr, rightAttr) =>
-  leftAttr.withNullability(leftAttr.nullable || rightAttr.nullable)
-}
-
   final override lazy val resolved: Boolean =
 childrenResolved &&
   left.output.length == right.output.length &&
@@ -108,13 +103,24 @@ private[sql] object SetOperation {
 
 case class Union(left: LogicalPlan, right: LogicalPlan) extends 
SetOperation(left, right) {
 
+  override def output: Seq[Attribute] =
+left.output.zip(right.output).map { case (leftAttr, rightAttr) =>
+  leftAttr.withNullability(leftAttr.nullable || rightAttr.nullable)
+}
+
   override def statistics: Statistics = {
 val sizeInBytes = left.statistics.sizeInBytes + 
right.statistics.sizeInBytes
 Statistics(sizeInBytes = sizeInBytes)
   }
 }
 
-case class Intersect(left: LogicalPlan, right: LogicalPlan) extends 
SetOperation(left, right)
+case class Intersect(left: LogicalPlan, right: LogicalPlan) extends 
SetOperation(left, right) {
+
+  override def output: Seq[Attribute] =
+left.output.zip(right.output).map { case (leftAttr, rightAttr) =>
+  leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable)
+}
+}
 
 case class Except(left: LogicalPlan, right: LogicalPlan) extends 
SetOperation(left, right) {
   /** We don't use right.output because those rows get excluded from the set. 
*/

http://git-wip-us.apache.org/repos/asf/spark/blob/b72e01e8/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 afc8df0..bd11a38 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
@@ -337,6 +337,27 @@ class DataFrameSuite extends QueryTest with 
SharedSQLContext {
 checkAnswer(lowerCaseData.intersect(upperCaseData), Nil)
   }
 
+  test("intersect - nullability") {
+val nonNullableInts = Seq(Tuple1(1), Tuple1(3)).toDF()
+assert(nonNullableInts.schema.forall(_.nullable == false))
+
+val df1 = nonNullableInts.intersect(nullInts)
+checkAnswer(df1, Row(1) :: Row(3) :: Nil)
+assert(df1.schema.forall(_.nullable == false))
+
+val df2 = nullInts.intersect(nonNullableInts)
+checkAnswer(df2, Row(1) :: Row(3) :: Nil)
+assert(df2.schema.forall(_.nullable == false))
+
+val df3 = nullInts.intersect(nullInts)
+checkAnswer(df3,

spark git commit: [SPARK-12724] SQL generation support for persisted data source tables

2016-01-12 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master 0d543b98f -> 8ed5f12d2


[SPARK-12724] SQL generation support for persisted data source tables

This PR implements SQL generation support for persisted data source tables.  A 
new field `metastoreTableIdentifier: Option[TableIdentifier]` is added to 
`LogicalRelation`.  When a `LogicalRelation` representing a persisted data 
source relation is created, this field holds the database name and table name 
of the relation.

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

Closes #10712 from liancheng/spark-12724-datasources-sql-gen.


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

Branch: refs/heads/master
Commit: 8ed5f12d2bb408bd37e4156b5f1bad9a6b8c3cb5
Parents: 0d543b9
Author: Cheng Lian <l...@databricks.com>
Authored: Tue Jan 12 14:19:53 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Tue Jan 12 14:19:53 2016 -0800

--
 .../main/scala/org/apache/spark/sql/DataFrame.scala |  2 +-
 .../execution/datasources/DataSourceStrategy.scala  | 16 
 .../sql/execution/datasources/LogicalRelation.scala |  8 +---
 .../datasources/parquet/ParquetRelation.scala   | 10 ++
 .../spark/sql/execution/datasources/rules.scala | 16 
 .../datasources/parquet/ParquetFilterSuite.scala|  2 +-
 .../parquet/ParquetPartitionDiscoverySuite.scala|  2 +-
 .../spark/sql/sources/FilteredScanSuite.scala   |  2 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |  6 --
 .../org/apache/spark/sql/hive/SQLBuilder.scala  | 14 +-
 .../apache/spark/sql/hive/execution/commands.scala  |  2 +-
 .../spark/sql/hive/LogicalPlanToSQLSuite.scala  | 10 ++
 .../spark/sql/hive/MetastoreDataSourcesSuite.scala  |  2 +-
 .../spark/sql/hive/execution/SQLQuerySuite.scala|  2 +-
 .../apache/spark/sql/hive/orc/OrcFilterSuite.scala  |  2 +-
 .../org/apache/spark/sql/hive/parquetSuites.scala   |  8 
 .../spark/sql/sources/hadoopFsRelationSuites.scala  |  2 +-
 17 files changed, 55 insertions(+), 51 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8ed5f12d/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 60d2f05..91bf2f8 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
@@ -1728,7 +1728,7 @@ class DataFrame private[sql](
*/
   def inputFiles: Array[String] = {
 val files: Seq[String] = logicalPlan.collect {
-  case LogicalRelation(fsBasedRelation: FileRelation, _) =>
+  case LogicalRelation(fsBasedRelation: FileRelation, _, _) =>
 fsBasedRelation.inputFiles
   case fr: FileRelation =>
 fr.inputFiles

http://git-wip-us.apache.org/repos/asf/spark/blob/8ed5f12d/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
index 1d6290e..da9320f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
@@ -41,7 +41,7 @@ import org.apache.spark.util.{SerializableConfiguration, 
Utils}
  */
 private[sql] object DataSourceStrategy extends Strategy with Logging {
   def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match {
-case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
CatalystScan, _)) =>
+case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
CatalystScan, _, _)) =>
   pruneFilterProjectRaw(
 l,
 projects,
@@ -49,14 +49,14 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
 (requestedColumns, allPredicates, _) =>
   toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, 
allPredicates))) :: Nil
 
-case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
PrunedFilteredScan, _)) =>
+case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
PrunedFilteredScan, _, _)) =>
   pruneFilterProject(
 l,
 projects,
 filters,
 (a, f) =&

spark git commit: [SPARK-12742][SQL] org.apache.spark.sql.hive.LogicalPlanToSQLSuite failure due to Table already exists exception

2016-01-11 Thread lian
Repository: spark
Updated Branches:
  refs/heads/master fe9eb0b0c -> 473907adf


[SPARK-12742][SQL] org.apache.spark.sql.hive.LogicalPlanToSQLSuite failure due 
to Table already exists exception

```
[info] Exception encountered when attempting to run a suite with class name:
org.apache.spark.sql.hive.LogicalPlanToSQLSuite *** ABORTED *** (325 
milliseconds)
[info]   org.apache.spark.sql.AnalysisException: Table `t1` already exists.;
[info]   at 
org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:296)
[info]   at 
org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:285)
[info]   at 
org.apache.spark.sql.hive.LogicalPlanToSQLSuite.beforeAll(LogicalPlanToSQLSuite.scala:33)
[info]   at 
org.scalatest.BeforeAndAfterAll$class.beforeAll(BeforeAndAfterAll.scala:187)
[info]   at 
org.apache.spark.sql.hive.LogicalPlanToSQLSuite.beforeAll(LogicalPlanToSQLSuite.scala:23)
[info]   at 
org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:253)
[info]   at 
org.apache.spark.sql.hive.LogicalPlanToSQLSuite.run(LogicalPlanToSQLSuite.scala:23)
[info]   at 
org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
[info]   at 
org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
[info]   at sbt.ForkMain$Run$2.call(ForkMain.java:296)
[info]   at sbt.ForkMain$Run$2.call(ForkMain.java:286)
[info]   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
[info]   at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
[info]   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[info]   at java.lang.Thread.run(Thread.java:745)
```

/cc liancheng

Author: wangfei <wangfei_he...@126.com>

Closes #10682 from scwf/fix-test.


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

Branch: refs/heads/master
Commit: 473907adf6e37855ee31d0703b43d7170e26b4b9
Parents: fe9eb0b
Author: wangfei <wangfei_he...@126.com>
Authored: Mon Jan 11 18:18:44 2016 -0800
Committer: Cheng Lian <l...@databricks.com>
Committed: Mon Jan 11 18:18:44 2016 -0800

--
 .../scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala   | 3 +++
 1 file changed, 3 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/473907ad/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
index 9a8a9c5..2ee8150 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala
@@ -24,6 +24,9 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with 
SQLTestUtils {
   import testImplicits._
 
   protected override def beforeAll(): Unit = {
+sql("DROP TABLE IF EXISTS t0")
+sql("DROP TABLE IF EXISTS t1")
+sql("DROP TABLE IF EXISTS t2")
 sqlContext.range(10).write.saveAsTable("t0")
 
 sqlContext


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



<    1   2   3   4   5   >