Commit graph

1177 commits

Author SHA1 Message Date
Kazuaki Ishizaki 8e629b10cb [SPARK-12530][BUILD] Fix build break at Spark-Master-Maven-Snapshots from #1293
Compilation error caused due to string concatenations that are not a constant
Use raw string literal to avoid string concatenations

https://amplab.cs.berkeley.edu/jenkins/view/Spark-Packaging/job/Spark-Master-Maven-Snapshots/1293/

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #10488 from kiszk/SPARK-12530.
2015-12-29 10:35:23 -08:00
gatorsmile 01ba95d8bf [SPARK-12441][SQL] Fixing missingInput in Generate/MapPartitions/AppendColumns/MapGroups/CoGroup
When explain any plan with Generate, we will see an exclamation mark in the plan. Normally, when we see this mark, it means the plan has an error. This PR is to correct the `missingInput` in `Generate`.

For example,
```scala
val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDF("number", "letters")
val df2 =
  df.explode('letters) {
    case Row(letters: String) => letters.split(" ").map(Tuple1(_)).toSeq
  }

df2.explain(true)
```
Before the fix, the plan is like
```
== Parsed Logical Plan ==
'Generate UserDefinedGenerator('letters), true, false, None
+- Project [_1#0 AS number#2,_2#1 AS letters#3]
   +- LocalRelation [_1#0,_2#1], [[1,a b c],[2,a b],[3,a]]

== Analyzed Logical Plan ==
number: int, letters: string, _1: string
Generate UserDefinedGenerator(letters#3), true, false, None, [_1#8]
+- Project [_1#0 AS number#2,_2#1 AS letters#3]
   +- LocalRelation [_1#0,_2#1], [[1,a b c],[2,a b],[3,a]]

== Optimized Logical Plan ==
Generate UserDefinedGenerator(letters#3), true, false, None, [_1#8]
+- LocalRelation [number#2,letters#3], [[1,a b c],[2,a b],[3,a]]

== Physical Plan ==
!Generate UserDefinedGenerator(letters#3), true, false, [number#2,letters#3,_1#8]
+- LocalTableScan [number#2,letters#3], [[1,a b c],[2,a b],[3,a]]
```

**Updates**: The same issues are also found in the other four Dataset operators: `MapPartitions`/`AppendColumns`/`MapGroups`/`CoGroup`. Fixed all these four.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10393 from gatorsmile/generateExplain.
2015-12-28 12:48:30 -08:00
Stephan Kessler a6a4812434 [SPARK-7727][SQL] Avoid inner classes in RuleExecutor
Moved (case) classes Strategy, Once, FixedPoint and Batch to the companion object. This is necessary if we want to have the Optimizer easily extendable in the following sense: Usually a user wants to add additional rules, and just take the ones that are already there. However, inner classes made that impossible since the code did not compile

This allows easy extension of existing Optimizers see the DefaultOptimizerExtendableSuite for a corresponding test case.

Author: Stephan Kessler <stephan.kessler@sap.com>

Closes #10174 from stephankessler/SPARK-7727.
2015-12-28 12:46:20 -08:00
pierre-borckmans 43b2a63900 [SPARK-12477][SQL] - Tungsten projection fails for null values in array fields
Accessing null elements in an array field fails when tungsten is enabled.
It works in Spark 1.3.1, and in Spark > 1.5 with Tungsten disabled.

This PR solves this by checking if the accessed element in the array field is null, in the generated code.

Example:
```
// Array of String
case class AS( as: Seq[String] )
val dfAS = sc.parallelize( Seq( AS ( Seq("a",null,"b") ) ) ).toDF
dfAS.registerTempTable("T_AS")
for (i <- 0 to 2) { println(i + " = " + sqlContext.sql(s"select as[$i] from T_AS").collect.mkString(","))}
```

With Tungsten disabled:
```
0 = [a]
1 = [null]
2 = [b]
```

With Tungsten enabled:
```
0 = [a]
15/12/22 09:32:50 ERROR Executor: Exception in task 7.0 in stage 1.0 (TID 15)
java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.UnsafeRowWriters$UTF8StringWriter.getSize(UnsafeRowWriters.java:90)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.execution.TungstenProject$$anonfun$3$$anonfun$apply$3.apply(basicOperators.scala:90)
	at org.apache.spark.sql.execution.TungstenProject$$anonfun$3$$anonfun$apply$3.apply(basicOperators.scala:88)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
```

Author: pierre-borckmans <pierre.borckmans@realimpactanalytics.com>

Closes #10429 from pierre-borckmans/SPARK-12477_Tungsten-Projection-Null-Element-In-Array.
2015-12-22 23:00:42 -08:00
Liang-Chi Hsieh 50301c0a28 [SPARK-11164][SQL] Add InSet pushdown filter back for Parquet
When the filter is ```"b in ('1', '2')"```, the filter is not pushed down to Parquet. Thanks!

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10278 from gatorsmile/parquetFilterNot.
2015-12-23 14:08:29 +08:00
Cheng Lian 86761e10e1 [SPARK-12478][SQL] Bugfix: Dataset fields of product types can't be null
When creating extractors for product types (i.e. case classes and tuples), a null check is missing, thus we always assume input product values are non-null.

This PR adds a null check in the extractor expression for product types. The null check is stripped off for top level product fields, which are mapped to the outermost `Row`s, since they can't be null.

Thanks cloud-fan for helping investigating this issue!

Author: Cheng Lian <lian@databricks.com>

Closes #10431 from liancheng/spark-12478.top-level-null-field.
2015-12-23 10:21:00 +08:00
Dilip Biswal b374a25831 [SPARK-12102][SQL] Cast a non-nullable struct field to a nullable field during analysis
Compare both left and right side of the case expression ignoring nullablity when checking for type equality.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #10156 from dilipbiswal/spark-12102.
2015-12-22 15:21:49 -08:00
Xiu Guo b5ce84a1bb [SPARK-12456][SQL] Add ExpressionDescription to misc functions
First try, not sure how much information we need to provide in the usage part.

Author: Xiu Guo <xguo27@gmail.com>

Closes #10423 from xguo27/SPARK-12456.
2015-12-22 10:44:01 -08:00
Cheng Lian 42bfde2983 [SPARK-12371][SQL] Runtime nullability check for NewInstance
This PR adds a new expression `AssertNotNull` to ensure non-nullable fields of products and case classes don't receive null values at runtime.

Author: Cheng Lian <lian@databricks.com>

Closes #10331 from liancheng/dataset-nullability-check.
2015-12-22 19:41:44 +08:00
gatorsmile 4883a5087d [SPARK-12374][SPARK-12150][SQL] Adding logical/physical operators for Range
Based on the suggestions from marmbrus , added logical/physical operators for Range for improving the performance.

Also added another API for resolving the JIRA Spark-12150.

Could you take a look at my implementation, marmbrus ? If not good, I can rework it. : )

Thank you very much!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10335 from gatorsmile/rangeOperators.
2015-12-21 13:46:58 -08:00
Wenchen Fan 7634fe9511 [SPARK-12321][SQL] JSON format for TreeNode (use reflection)
An alternative solution for https://github.com/apache/spark/pull/10295 , instead of implementing json format for all logical/physical plans and expressions, use reflection to implement it in `TreeNode`.

Here I use pre-order traversal to flattern a plan tree to a plan list, and add an extra field `num-children` to each plan node, so that we can reconstruct the tree from the list.

example json:

logical plan tree:
```
[ {
  "class" : "org.apache.spark.sql.catalyst.plans.logical.Sort",
  "num-children" : 1,
  "order" : [ [ {
    "class" : "org.apache.spark.sql.catalyst.expressions.SortOrder",
    "num-children" : 1,
    "child" : 0,
    "direction" : "Ascending"
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
    "num-children" : 0,
    "name" : "i",
    "dataType" : "integer",
    "nullable" : true,
    "metadata" : { },
    "exprId" : {
      "id" : 10,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  } ] ],
  "global" : false,
  "child" : 0
}, {
  "class" : "org.apache.spark.sql.catalyst.plans.logical.Project",
  "num-children" : 1,
  "projectList" : [ [ {
    "class" : "org.apache.spark.sql.catalyst.expressions.Alias",
    "num-children" : 1,
    "child" : 0,
    "name" : "i",
    "exprId" : {
      "id" : 10,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.Add",
    "num-children" : 2,
    "left" : 0,
    "right" : 1
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
    "num-children" : 0,
    "name" : "a",
    "dataType" : "integer",
    "nullable" : true,
    "metadata" : { },
    "exprId" : {
      "id" : 0,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.Literal",
    "num-children" : 0,
    "value" : "1",
    "dataType" : "integer"
  } ], [ {
    "class" : "org.apache.spark.sql.catalyst.expressions.Alias",
    "num-children" : 1,
    "child" : 0,
    "name" : "j",
    "exprId" : {
      "id" : 11,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.Multiply",
    "num-children" : 2,
    "left" : 0,
    "right" : 1
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
    "num-children" : 0,
    "name" : "a",
    "dataType" : "integer",
    "nullable" : true,
    "metadata" : { },
    "exprId" : {
      "id" : 0,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.Literal",
    "num-children" : 0,
    "value" : "2",
    "dataType" : "integer"
  } ] ],
  "child" : 0
}, {
  "class" : "org.apache.spark.sql.catalyst.plans.logical.LocalRelation",
  "num-children" : 0,
  "output" : [ [ {
    "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
    "num-children" : 0,
    "name" : "a",
    "dataType" : "integer",
    "nullable" : true,
    "metadata" : { },
    "exprId" : {
      "id" : 0,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  } ] ],
  "data" : [ ]
} ]
```

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10311 from cloud-fan/toJson-reflection.
2015-12-21 12:47:07 -08:00
Dilip Biswal 474eb21a30 [SPARK-12398] Smart truncation of DataFrame / Dataset toString
When a DataFrame or Dataset has a long schema, we should intelligently truncate to avoid flooding the screen with unreadable information.
// Standard output
[a: int, b: int]

// Truncate many top level fields
[a: int, b, string ... 10 more fields]

// Truncate long inner structs
[a: struct<a: Int ... 10 more fields>]

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #10373 from dilipbiswal/spark-12398.
2015-12-21 12:46:06 -08:00
Reynold Xin f496031bd2 Bump master version to 2.0.0-SNAPSHOT.
Author: Reynold Xin <rxin@databricks.com>

Closes #10387 from rxin/version-bump.
2015-12-19 15:13:05 -08:00
Kousuke Saruta 6eba655259 [SPARK-12404][SQL] Ensure objects passed to StaticInvoke is Serializable
Now `StaticInvoke` receives `Any` as a object and `StaticInvoke` can be serialized but sometimes the object passed is not serializable.

For example, following code raises Exception because `RowEncoder#extractorsFor` invoked indirectly makes `StaticInvoke`.

```
case class TimestampContainer(timestamp: java.sql.Timestamp)
val rdd = sc.parallelize(1 to 2).map(_ => TimestampContainer(System.currentTimeMillis))
val df = rdd.toDF
val ds = df.as[TimestampContainer]
val rdd2 = ds.rdd                                 <----------------- invokes extractorsFor indirectory
```

I'll add test cases.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Author: Michael Armbrust <michael@databricks.com>

Closes #10357 from sarutak/SPARK-12404.
2015-12-18 14:05:06 -08:00
Davies Liu 4af647c77d [SPARK-12054] [SQL] Consider nullability of expression in codegen
This could simplify the generated code for expressions that is not nullable.

This PR fix lots of bugs about nullability.

Author: Davies Liu <davies@databricks.com>

Closes #10333 from davies/skip_nullable.
2015-12-18 10:09:17 -08:00
Dilip Biswal ee444fe4b8 [SPARK-11619][SQL] cannot use UDTF in DataFrame.selectExpr
Description of the problem from cloud-fan

Actually this line: https://github.com/apache/spark/blob/branch-1.5/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala#L689
When we use `selectExpr`, we pass in `UnresolvedFunction` to `DataFrame.select` and fall in the last case. A workaround is to do special handling for UDTF like we did for `explode`(and `json_tuple` in 1.6), wrap it with `MultiAlias`.
Another workaround is using `expr`, for example, `df.select(expr("explode(a)").as(Nil))`, I think `selectExpr` is no longer needed after we have the `expr` function....

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #9981 from dilipbiswal/spark-11619.
2015-12-18 09:54:30 -08:00
Herman van Hovell 658f66e620 [SPARK-8641][SQL] Native Spark Window functions
This PR removes Hive windows functions from Spark and replaces them with (native) Spark ones. The PR is on par with Hive in terms of features.

This has the following advantages:
* Better memory management.
* The ability to use spark UDAFs in Window functions.

cc rxin / yhuai

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

Closes #9819 from hvanhovell/SPARK-8641-2.
2015-12-17 15:16:35 -08:00
Wenchen Fan a783a8ed49 [SPARK-12320][SQL] throw exception if the number of fields does not line up for Tuple encoder
Author: Wenchen Fan <wenchen@databricks.com>

Closes #10293 from cloud-fan/err-msg.
2015-12-16 13:20:12 -08:00
Davies Liu 54c512ba90 [SPARK-8745] [SQL] remove GenerateProjection
cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #10316 from davies/remove_generate_projection.
2015-12-16 10:22:48 -08:00
Wenchen Fan a89e8b6122 [SPARK-10477][SQL] using DSL in ColumnPruningSuite to improve readability
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8645 from cloud-fan/test.
2015-12-15 18:29:19 -08:00
Nong Li 86ea64dd14 [SPARK-12271][SQL] Improve error message when Dataset.as[ ] has incompatible schemas.
Author: Nong Li <nong@databricks.com>

Closes #10260 from nongli/spark-11271.
2015-12-15 16:55:58 -08:00
Wenchen Fan 9ea1a8efca [SPARK-12274][SQL] WrapOption should not have type constraint for child
I think it was a mistake, and we have not catched it so far until https://github.com/apache/spark/pull/10260 which begin to check if the `fromRowExpression` is resolved.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10263 from cloud-fan/encoder.
2015-12-14 16:48:11 -08:00
Davies Liu 834e71489b [SPARK-12213][SQL] use multiple partitions for single distinct query
Currently, we could generate different plans for query with single distinct (depends on spark.sql.specializeSingleDistinctAggPlanning), one works better on low cardinality columns, the other
works better for high cardinality column (default one).

This PR change to generate a single plan (three aggregations and two exchanges), which work better in both cases, then we could safely remove the flag `spark.sql.specializeSingleDistinctAggPlanning` (introduced in 1.6).

For a query like `SELECT COUNT(DISTINCT a) FROM table` will be
```
AGG-4 (count distinct)
  Shuffle to a single reducer
    Partial-AGG-3 (count distinct, no grouping)
      Partial-AGG-2 (grouping on a)
        Shuffle by a
          Partial-AGG-1 (grouping on a)
```

This PR also includes large refactor for aggregation (reduce 500+ lines of code)

cc yhuai nongli marmbrus

Author: Davies Liu <davies@databricks.com>

Closes #10228 from davies/single_distinct.
2015-12-13 22:57:01 -08:00
Davies Liu c119a34d1e [SPARK-12258] [SQL] passing null into ScalaUDF (follow-up)
This is a follow-up PR for #10259

Author: Davies Liu <davies@databricks.com>

Closes #10266 from davies/null_udf2.
2015-12-11 11:15:53 -08:00
Davies Liu b1b4ee7f35 [SPARK-12258][SQL] passing null into ScalaUDF
Check nullability and passing them into ScalaUDF.

Closes #10249

Author: Davies Liu <davies@databricks.com>

Closes #10259 from davies/udf_null.
2015-12-10 17:22:18 -08:00
Wenchen Fan d8ec081c91 [SPARK-12252][SPARK-12131][SQL] refactor MapObjects to make it less hacky
in https://github.com/apache/spark/pull/10133 we found that, we shoud ensure the children of `TreeNode` are all accessible in the `productIterator`, or the behavior will be very confusing.

In this PR, I try to fix this problem by expsing the `loopVar`.

This also fixes SPARK-12131 which is caused by the hacky `MapObjects`.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10239 from cloud-fan/map-objects.
2015-12-10 15:11:13 +08:00
Michael Armbrust 3959489423 [SPARK-12069][SQL] Update documentation with Datasets
Author: Michael Armbrust <michael@databricks.com>

Closes #10060 from marmbrus/docs.
2015-12-08 15:58:35 -08:00
Andrew Ray 4bcb894948 [SPARK-12205][SQL] Pivot fails Analysis when aggregate is UnresolvedFunction
Delays application of ResolvePivot until all aggregates are resolved to prevent problems with UnresolvedFunction and adds unit test

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #10202 from aray/sql-pivot-unresolved-function.
2015-12-08 10:52:17 -08:00
gatorsmile c0b13d5565 [SPARK-12195][SQL] Adding BigDecimal, Date and Timestamp into Encoder
This PR is to add three more data types into Encoder, including `BigDecimal`, `Date` and `Timestamp`.

marmbrus cloud-fan rxin Could you take a quick look at these three types? Not sure if it can be merged to 1.6. Thank you very much!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10188 from gatorsmile/dataTypesinEncoder.
2015-12-08 10:15:58 -08:00
Wenchen Fan 381f17b540 [SPARK-12201][SQL] add type coercion rule for greatest/least
checked with hive, greatest/least should cast their children to a tightest common type,
i.e. `(int, long) => long`, `(int, string) => error`, `(decimal(10,5), decimal(5, 10)) => error`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10196 from cloud-fan/type-coercion.
2015-12-08 10:13:40 -08:00
Davies Liu 9cde7d5fa8 [SPARK-12032] [SQL] Re-order inner joins to do join with conditions first
Currently, the order of joins is exactly the same as SQL query, some conditions may not pushed down to the correct join, then those join will become cross product and is extremely slow.

This patch try to re-order the inner joins (which are common in SQL query), pick the joins that have self-contain conditions first, delay those that does not have conditions.

After this patch, the TPCDS query Q64/65 can run hundreds times faster.

cc marmbrus nongli

Author: Davies Liu <davies@databricks.com>

Closes #10073 from davies/reorder_joins.
2015-12-07 10:34:18 -08:00
gatorsmile 49efd03bac [SPARK-12138][SQL] Escape \u in the generated comments of codegen
When \u appears in a comment block (i.e. in /**/), code gen will break. So, in Expression and CodegenFallback, we escape \u to \\u.

yhuai Please review it. I did reproduce it and it works after the fix. Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10155 from gatorsmile/escapeU.
2015-12-06 11:15:02 -08:00
Josh Rosen b7204e1d41 [SPARK-12112][BUILD] Upgrade to SBT 0.13.9
We should upgrade to SBT 0.13.9, since this is a requirement in order to use SBT's new Maven-style resolution features (which will be done in a separate patch, because it's blocked by some binary compatibility issues in the POM reader plugin).

I also upgraded Scalastyle to version 0.8.0, which was necessary in order to fix a Scala 2.10.5 compatibility issue (see https://github.com/scalastyle/scalastyle/issues/156). The newer Scalastyle is slightly stricter about whitespace surrounding tokens, so I fixed the new style violations.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10112 from JoshRosen/upgrade-to-sbt-0.13.9.
2015-12-05 08:15:30 +08:00
Dmitry Erastov d0d8222778 [SPARK-6990][BUILD] Add Java linting script; fix minor warnings
This replaces https://github.com/apache/spark/pull/9696

Invoke Checkstyle and print any errors to the console, failing the step.
Use Google's style rules modified according to
https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide
Some important checks are disabled (see TODOs in `checkstyle.xml`) due to
multiple violations being present in the codebase.

Suggest fixing those TODOs in a separate PR(s).

More on Checkstyle can be found on the [official website](http://checkstyle.sourceforge.net/).

Sample output (from [build 46345](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/46345/consoleFull)) (duplicated because I run the build twice with different profiles):

> Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java:[217,7] (coding) MissingSwitchDefault: switch without "default" clause.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[198,10] (modifier) ModifierOrder: 'protected' modifier out of order with the JLS suggestions.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java:[217,7] (coding) MissingSwitchDefault: switch without "default" clause.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[198,10] (modifier) ModifierOrder: 'protected' modifier out of order with the JLS suggestions.
> [error] running /home/jenkins/workspace/SparkPullRequestBuilder2/dev/lint-java ; received return code 1

Also fix some of the minor violations that didn't require sweeping changes.

Apologies for the previous botched PRs - I finally figured out the issue.

cr: JoshRosen, pwendell

> I state that the contribution is my original work, and I license the work to the project under the project's open source license.

Author: Dmitry Erastov <derastov@gmail.com>

Closes #9867 from dskrvk/master.
2015-12-04 12:03:45 -08:00
Yin Huai ec2b6c26c9 [SPARK-12109][SQL] Expressions's simpleString should delegate to its toString.
https://issues.apache.org/jira/browse/SPARK-12109

The change of https://issues.apache.org/jira/browse/SPARK-11596 exposed the problem.
In the sql plan viz, the filter shows

![image](https://cloud.githubusercontent.com/assets/2072857/11547075/1a285230-9906-11e5-8481-2bb451e35ef1.png)

After changes in this PR, the viz is back to normal.
![image](https://cloud.githubusercontent.com/assets/2072857/11547080/2bc570f4-9906-11e5-8897-3b3bff173276.png)

Author: Yin Huai <yhuai@databricks.com>

Closes #10111 from yhuai/SPARK-12109.
2015-12-03 11:21:24 +08:00
Cheng Lian a1542ce2f3 [SPARK-12094][SQL] Prettier tree string for TreeNode
When examining plans of complex queries with multiple joins, a pain point of mine is that, it's hard to immediately see the sibling node of a specific query plan node. This PR adds tree lines for the tree string of a `TreeNode`, so that the result can be visually more intuitive.

Author: Cheng Lian <lian@databricks.com>

Closes #10099 from liancheng/prettier-tree-string.
2015-12-02 09:36:12 -08:00
Liang-Chi Hsieh 0f37d1d7ed [SPARK-11949][SQL] Check bitmasks to set nullable property
Following up #10038.

We can use bitmasks to determine which grouping expressions need to be set as nullable.

cc yhuai

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #10067 from viirya/fix-cube-following.
2015-12-01 21:51:33 -08:00
Yin Huai e96a70d5ab [SPARK-11596][SQL] In TreeNode's argString, if a TreeNode is not a child of the current TreeNode, we should only return the simpleString.
In TreeNode's argString, if a TreeNode is not a child of the current TreeNode, we will only return the simpleString.

I tested the [following case provided by Cristian](https://issues.apache.org/jira/browse/SPARK-11596?focusedCommentId=15019241&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15019241).
```
val c = (1 to 20).foldLeft[Option[DataFrame]] (None) { (curr, idx) =>
    println(s"PROCESSING >>>>>>>>>>> $idx")
    val df = sqlContext.sparkContext.parallelize((0 to 10).zipWithIndex).toDF("A", "B")
    val union = curr.map(_.unionAll(df)).getOrElse(df)
    union.cache()
    Some(union)
  }

c.get.explain(true)
```

Without the change, `c.get.explain(true)` took 100s. With the change, `c.get.explain(true)` took 26ms.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #10079 from yhuai/SPARK-11596.
2015-12-01 17:18:45 -08:00
Yin Huai 5872a9d89f [SPARK-11352][SQL] Escape */ in the generated comments.
https://issues.apache.org/jira/browse/SPARK-11352

Author: Yin Huai <yhuai@databricks.com>

Closes #10072 from yhuai/SPARK-11352.
2015-12-01 16:24:04 -08:00
Wenchen Fan fd95eeaf49 [SPARK-11954][SQL] Encoder for JavaBeans
create java version of `constructorFor` and `extractorFor` in `JavaTypeInference`

Author: Wenchen Fan <wenchen@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>

Closes #9937 from cloud-fan/pojo.
2015-12-01 10:35:12 -08:00
Wenchen Fan 9df24624af [SPARK-11856][SQL] add type cast if the real type is different but compatible with encoder schema
When we build the `fromRowExpression` for an encoder, we set up a lot of "unresolved" stuff and lost the required data type, which may lead to runtime error if the real type doesn't match the encoder's schema.
For example, we build an encoder for `case class Data(a: Int, b: String)` and the real type is `[a: int, b: long]`, then we will hit runtime error and say that we can't construct class `Data` with int and long, because we lost the information that `b` should be a string.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9840 from cloud-fan/err-msg.
2015-12-01 10:24:53 -08:00
Liang-Chi Hsieh c87531b765 [SPARK-11949][SQL] Set field nullable property for GroupingSets to get correct results for null values
JIRA: https://issues.apache.org/jira/browse/SPARK-11949

The result of cube plan uses incorrect schema. The schema of cube result should set nullable property to true because the grouping expressions will have null values.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #10038 from viirya/fix-cube.
2015-12-01 07:44:22 -08:00
Liang-Chi Hsieh 9693b0d5a5 [SPARK-12018][SQL] Refactor common subexpression elimination code
JIRA: https://issues.apache.org/jira/browse/SPARK-12018

The code of common subexpression elimination can be factored and simplified. Some unnecessary variables can be removed.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #10009 from viirya/refactor-subexpr-eliminate.
2015-11-30 20:56:42 -08:00
Herman van Hovell 3d28081e53 [SPARK-12024][SQL] More efficient multi-column counting.
In https://github.com/apache/spark/pull/9409 we enabled multi-column counting. The approach taken in that PR introduces a bit of overhead by first creating a row only to check if all of the columns are non-null.

This PR fixes that technical debt. Count now takes multiple columns as its input. In order to make this work I have also added support for multiple columns in the single distinct code path.

cc yhuai

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

Closes #10015 from hvanhovell/SPARK-12024.
2015-11-29 14:13:11 -08:00
gatorsmile 149cd692ee [SPARK-12028] [SQL] get_json_object returns an incorrect result when the value is null literals
When calling `get_json_object` for the following two cases, both results are `"null"`:

```scala
    val tuple: Seq[(String, String)] = ("5", """{"f1": null}""") :: Nil
    val df: DataFrame = tuple.toDF("key", "jstring")
    val res = df.select(functions.get_json_object($"jstring", "$.f1")).collect()
```
```scala
    val tuple2: Seq[(String, String)] = ("5", """{"f1": "null"}""") :: Nil
    val df2: DataFrame = tuple2.toDF("key", "jstring")
    val res3 = df2.select(functions.get_json_object($"jstring", "$.f1")).collect()
```

Fixed the problem and also added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10018 from gatorsmile/get_json_object.
2015-11-27 22:44:08 -08:00
Reynold Xin de28e4d4de [SPARK-11973][SQL] Improve optimizer code readability.
This is a followup for https://github.com/apache/spark/pull/9959.

I added more documentation and rewrote some monadic code into simpler ifs.

Author: Reynold Xin <rxin@databricks.com>

Closes #9995 from rxin/SPARK-11973.
2015-11-26 18:47:54 -08:00
Dilip Biswal bc16a67562 [SPARK-11863][SQL] Unable to resolve order by if it contains mixture of aliases and real columns
this is based on https://github.com/apache/spark/pull/9844, with some bug fix and clean up.

The problems is that, normal operator should be resolved based on its child, but `Sort` operator can also be resolved based on its grandchild. So we have 3 rules that can resolve `Sort`: `ResolveReferences`, `ResolveSortReferences`(if grandchild is `Project`) and `ResolveAggregateFunctions`(if grandchild is `Aggregate`).
For example, `select c1 as a , c2 as b from tab group by c1, c2 order by a, c2`, we need to resolve `a` and `c2` for `Sort`. Firstly `a` will be resolved in `ResolveReferences` based on its child, and when we reach `ResolveAggregateFunctions`, we will try to resolve both `a` and `c2` based on its grandchild, but failed because `a` is not a legal aggregate expression.

whoever merge this PR, please give the credit to dilipbiswal

Author: Dilip Biswal <dbiswal@us.ibm.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9961 from cloud-fan/sort.
2015-11-26 11:31:28 -08:00
Marcelo Vanzin 001f0528a8 [SPARK-12005][SQL] Work around VerifyError in HyperLogLogPlusPlus.
Just move the code around a bit; that seems to make the JVM happy.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9985 from vanzin/SPARK-12005.
2015-11-26 01:15:05 -08:00
Davies Liu 27d69a0573 [SPARK-11973] [SQL] push filter through aggregation with alias and literals
Currently, filter can't be pushed through aggregation with alias or literals, this patch fix that.

After this patch, the time of TPC-DS query 4 go down to 13 seconds from 141 seconds (10x improvements).

cc nongli  yhuai

Author: Davies Liu <davies@databricks.com>

Closes #9959 from davies/push_filter2.
2015-11-26 00:19:42 -08:00
Davies Liu d1930ec01a [SPARK-12003] [SQL] remove the prefix for name after expanded star
Right now, the expended start will include the name of expression as prefix for column, that's not better than without expending, we should not have the prefix.

Author: Davies Liu <davies@databricks.com>

Closes #9984 from davies/expand_star.
2015-11-25 21:25:20 -08:00
Daoyuan Wang 21e5606419 [SPARK-11983][SQL] remove all unused codegen fallback trait
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #9966 from adrian-wang/removeFallback.
2015-11-25 13:51:30 -08:00
Reynold Xin f315272279 [SPARK-11946][SQL] Audit pivot API for 1.6.
Currently pivot's signature looks like

```scala
scala.annotation.varargs
def pivot(pivotColumn: Column, values: Column*): GroupedData

scala.annotation.varargs
def pivot(pivotColumn: String, values: Any*): GroupedData
```

I think we can remove the one that takes "Column" types, since callers should always be passing in literals. It'd also be more clear if the values are not varargs, but rather Seq or java.util.List.

I also made similar changes for Python.

Author: Reynold Xin <rxin@databricks.com>

Closes #9929 from rxin/SPARK-11946.
2015-11-24 12:54:37 -08:00
Wenchen Fan 19530da690 [SPARK-11926][SQL] unify GetStructField and GetInternalRowField
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9909 from cloud-fan/get-struct.
2015-11-24 11:09:01 -08:00
Wenchen Fan e5aaae6e11 [SPARK-11942][SQL] fix encoder life cycle for CoGroup
we should pass in resolved encodera to logical `CoGroup` and bind them in physical `CoGroup`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9928 from cloud-fan/cogroup.
2015-11-24 09:28:39 -08:00
Mikhail Bautin 4021a28ac3 [SPARK-10707][SQL] Fix nullability computation in union output
Author: Mikhail Bautin <mbautin@gmail.com>

Closes #9308 from mbautin/SPARK-10707.
2015-11-23 22:26:08 -08:00
Wenchen Fan f2996e0d12 [SPARK-11921][SQL] fix nullable of encoder schema
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9906 from cloud-fan/nullable.
2015-11-23 10:15:40 -08:00
Wenchen Fan 1a5baaa651 [SPARK-11894][SQL] fix isNull for GetInternalRowField
We should use `InternalRow.isNullAt` to check if the field is null before calling `InternalRow.getXXX`

Thanks gatorsmile who discovered this bug.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9904 from cloud-fan/null.
2015-11-23 10:13:59 -08:00
Xiu Guo 94ce65dfcb [SPARK-11628][SQL] support column datatype of char(x) to recognize HiveChar
Can someone review my code to make sure I'm not missing anything? Thanks!

Author: Xiu Guo <xguo27@gmail.com>
Author: Xiu Guo <guoxi@us.ibm.com>

Closes #9612 from xguo27/SPARK-11628.
2015-11-23 08:53:40 -08:00
Liang-Chi Hsieh 426004a9c9 [SPARK-11908][SQL] Add NullType support to RowEncoder
JIRA: https://issues.apache.org/jira/browse/SPARK-11908

We should add NullType support to RowEncoder.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #9891 from viirya/rowencoder-nulltype.
2015-11-22 10:36:47 -08:00
Reynold Xin ff442bbcff [SPARK-11899][SQL] API audit for GroupedDataset.
1. Renamed map to mapGroup, flatMap to flatMapGroup.
2. Renamed asKey -> keyAs.
3. Added more documentation.
4. Changed type parameter T to V on GroupedDataset.
5. Added since versions for all functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #9880 from rxin/SPARK-11899.
2015-11-21 15:00:37 -08:00
Reynold Xin 54328b6d86 [SPARK-11900][SQL] Add since version for all encoders
Author: Reynold Xin <rxin@databricks.com>

Closes #9881 from rxin/SPARK-11900.
2015-11-21 00:10:13 -08:00
Wenchen Fan 7d3f922c4b [SPARK-11819][SQL][FOLLOW-UP] fix scala 2.11 build
seems scala 2.11 doesn't support: define private methods in `trait xxx` and use it in `object xxx extend xxx`.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9879 from cloud-fan/follow.
2015-11-20 23:31:19 -08:00
Michael Armbrust 68ed046836 [SPARK-11890][SQL] Fix compilation for Scala 2.11
Author: Michael Armbrust <michael@databricks.com>

Closes #9871 from marmbrus/scala211-break.
2015-11-20 15:38:04 -08:00
Nong Li 58b4e4f88a [SPARK-11787][SPARK-11883][SQL][FOLLOW-UP] Cleanup for this patch.
This mainly moves SqlNewHadoopRDD to the sql package. There is some state that is
shared between core and I've left that in core. This allows some other associated
minor cleanup.

Author: Nong Li <nong@databricks.com>

Closes #9845 from nongli/spark-11787.
2015-11-20 15:30:53 -08:00
Michael Armbrust 4b84c72dfb [SPARK-11636][SQL] Support classes defined in the REPL with Encoders
#theScaryParts (i.e. changes to the repl, executor classloaders and codegen)...

Author: Michael Armbrust <michael@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #9825 from marmbrus/dataset-replClasses2.
2015-11-20 15:17:17 -08:00
Nong Li 9ed4ad4265 [SPARK-11724][SQL] Change casting between int and timestamp to consistently treat int in seconds.
Hive has since changed this behavior as well. https://issues.apache.org/jira/browse/HIVE-3454

Author: Nong Li <nong@databricks.com>
Author: Nong Li <nongli@gmail.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #9685 from nongli/spark-11724.
2015-11-20 14:19:34 -08:00
Wenchen Fan 3b9d2a347f [SPARK-11819][SQL] nice error message for missing encoder
before this PR, when users try to get an encoder for an un-supported class, they will only get a very simple error message like `Encoder for type xxx is not supported`.

After this PR, the error message become more friendly, for example:
```
No Encoder found for abc.xyz.NonEncodable
- array element class: "abc.xyz.NonEncodable"
- field (class: "scala.Array", name: "arrayField")
- root class: "abc.xyz.AnotherClass"
```

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9810 from cloud-fan/error-message.
2015-11-20 12:04:42 -08:00
Liang-Chi Hsieh 60bfb11332 [SPARK-11817][SQL] Truncating the fractional seconds to prevent inserting a NULL
JIRA: https://issues.apache.org/jira/browse/SPARK-11817

Instead of return None, we should truncate the fractional seconds to prevent inserting NULL.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #9834 from viirya/truncate-fractional-sec.
2015-11-20 11:43:45 -08:00
Davies Liu ee21407747 [SPARK-11864][SQL] Improve performance of max/min
This PR has the following optimization:

1) The greatest/least already does the null-check, so the `If` and `IsNull` are not necessary.

2) In greatest/least, it should initialize the result using the first child (removing one block).

3) For primitive types, the generated greater expression is too complicated (`a > b ? 1 : (a < b) ? -1 : 0) > 0`), should be as simple as `a > b`

Combine these optimization, this could improve the performance of `ss_max` query by 30%.

Author: Davies Liu <davies@databricks.com>

Closes #9846 from davies/improve_max.
2015-11-19 17:14:10 -08:00
Andrew Ray 37cff1b1a7 [SPARK-11275][SQL] Incorrect results when using rollup/cube
Fixes bug with grouping sets (including cube/rollup) where aggregates that included grouping expressions would return the wrong (null) result.

Also simplifies the analyzer rule a bit and leaves column pruning to the optimizer.

Added multiple unit tests to DataFrameAggregateSuite and verified it passes hive compatibility suite:
```
build/sbt -Phive -Dspark.hive.whitelist='groupby.*_grouping.*' 'test-only org.apache.spark.sql.hive.execution.HiveCompatibilitySuite'
```

This is an alternative to pr https://github.com/apache/spark/pull/9419 but I think its better as it simplifies the analyzer rule instead of adding another special case to it.

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #9815 from aray/groupingset-agg-fix.
2015-11-19 15:11:30 -08:00
Wenchen Fan 47d1c2325c [SPARK-11750][SQL] revert SPARK-11727 and code clean up
After some experiment, I found it's not convenient to have separate encoder builders: `FlatEncoder` and `ProductEncoder`. For example, when create encoders for `ScalaUDF`, we have no idea if the type `T` is flat or not. So I revert the splitting change in https://github.com/apache/spark/pull/9693, while still keeping the bug fixes and tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9726 from cloud-fan/follow.
2015-11-19 12:54:25 -08:00
Yin Huai 962878843b [SPARK-11840][SQL] Restore the 1.5's behavior of planning a single distinct aggregation.
The impact of this change is for a query that has a single distinct column and does not have any grouping expression like
`SELECT COUNT(DISTINCT a) FROM table`
The plan will be changed from
```
AGG-2 (count distinct)
  Shuffle to a single reducer
    Partial-AGG-2 (count distinct)
      AGG-1 (grouping on a)
        Shuffle by a
          Partial-AGG-1 (grouping on 1)
```
to the following one (1.5 uses this)
```
AGG-2
  AGG-1 (grouping on a)
    Shuffle to a single reducer
      Partial-AGG-1(grouping on a)
```
The first plan is more robust. However, to better benchmark the impact of this change, we should use 1.5's plan and use the conf of `spark.sql.specializeSingleDistinctAggPlanning` to control the plan.

Author: Yin Huai <yhuai@databricks.com>

Closes #9828 from yhuai/distinctRewriter.
2015-11-19 11:02:17 -08:00
Reynold Xin f449992009 [SPARK-11849][SQL] Analyzer should replace current_date and current_timestamp with literals
We currently rely on the optimizer's constant folding to replace current_timestamp and current_date. However, this can still result in different values for different instances of current_timestamp/current_date if the optimizer is not running fast enough.

A better solution is to replace these functions in the analyzer in one shot.

Author: Reynold Xin <rxin@databricks.com>

Closes #9833 from rxin/SPARK-11849.
2015-11-19 10:48:04 -08:00
Nong Li 6d0848b53b [SPARK-11787][SQL] Improve Parquet scan performance when using flat schemas.
This patch adds an alternate to the Parquet RecordReader from the parquet-mr project
that is much faster for flat schemas. Instead of using the general converter mechanism
from parquet-mr, this directly uses the lower level APIs from parquet-columnar and a
customer RecordReader that directly assembles into UnsafeRows.

This is optionally disabled and only used for supported schemas.

Using the tpcds store sales table and doing a sum of increasingly more columns, the results
are:

For 1 Column:
  Before: 11.3M rows/second
  After: 18.2M rows/second

For 2 Columns:
  Before: 7.2M rows/second
  After: 11.2M rows/second

For 5 Columns:
  Before: 2.9M rows/second
  After: 4.5M rows/second

Author: Nong Li <nong@databricks.com>

Closes #9774 from nongli/parquet.
2015-11-18 18:38:45 -08:00
Reynold Xin e61367b9f9 [SPARK-11833][SQL] Add Java tests for Kryo/Java Dataset encoders
Also added some nicer error messages for incompatible types (private types and primitive types) for Kryo/Java encoder.

Author: Reynold Xin <rxin@databricks.com>

Closes #9823 from rxin/SPARK-11833.
2015-11-18 18:34:36 -08:00
Michael Armbrust 59a501359a [SPARK-11636][SQL] Support classes defined in the REPL with Encoders
Before this PR there were two things that would blow up if you called `df.as[MyClass]` if `MyClass` was defined in the REPL:
 - [x] Because `classForName` doesn't work on the munged names returned by `tpe.erasure.typeSymbol.asClass.fullName`
 - [x] Because we don't have anything to pass into the constructor for the `$outer` pointer.

Note that this PR is just adding the infrastructure for working with inner classes in encoder and is not yet sufficient to make them work in the REPL.  Currently, the implementation show in 95cec7d413 is causing a bug that breaks code gen due to some interaction between janino and the `ExecutorClassLoader`.  This will be addressed in a follow-up PR.

Author: Michael Armbrust <michael@databricks.com>

Closes #9602 from marmbrus/dataset-replClasses.
2015-11-18 16:48:09 -08:00
Reynold Xin 5df08949f5 [SPARK-11810][SQL] Java-based encoder for opaque types in Datasets.
This patch refactors the existing Kryo encoder expressions and adds support for Java serialization.

Author: Reynold Xin <rxin@databricks.com>

Closes #9802 from rxin/SPARK-11810.
2015-11-18 15:42:07 -08:00
JihongMa 09ad9533d5 [SPARK-11720][SQL][ML] Handle edge cases when count = 0 or 1 for Stats function
return Double.NaN for mean/average when count == 0 for all numeric types that is converted to Double, Decimal type continue to return null.

Author: JihongMa <linlin200605@gmail.com>

Closes #9705 from JihongMA/SPARK-11720.
2015-11-18 13:03:37 -08:00
Wenchen Fan 33b8373334 [SPARK-11725][SQL] correctly handle null inputs for UDF
If user use primitive parameters in UDF, there is no way for him to do the null-check for primitive inputs, so we are assuming the primitive input is null-propagatable for this case and return null if the input is null.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9770 from cloud-fan/udf.
2015-11-18 10:23:12 -08:00
Reynold Xin 5e2b44474c [SPARK-11802][SQL] Kryo-based encoder for opaque types in Datasets
I also found a bug with self-joins returning incorrect results in the Dataset API. Two test cases attached and filed SPARK-11803.

Author: Reynold Xin <rxin@databricks.com>

Closes #9789 from rxin/SPARK-11802.
2015-11-18 00:09:29 -08:00
Davies Liu 2f191c66b6 [SPARK-11643] [SQL] parse year with leading zero
Support the years between 0 <= year < 1000

Author: Davies Liu <davies@databricks.com>

Closes #9701 from davies/leading_zero.
2015-11-17 23:14:05 -08:00
gatorsmile 0158ff7737 [SPARK-8658][SQL][FOLLOW-UP] AttributeReference's equals method compares all the members
Based on the comment of cloud-fan in https://github.com/apache/spark/pull/9216, update the AttributeReference's hashCode function by including the hashCode of the other attributes including name, nullable and qualifiers.

Here, I am not 100% sure if we should include name in the hashCode calculation, since the original hashCode calculation does not include it.

marmbrus cloud-fan Please review if the changes are good.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9761 from gatorsmile/hashCodeNamedExpression.
2015-11-17 11:23:54 -08:00
mayuanwen e8833dd12c [SPARK-11679][SQL] Invoking method " apply(fields: java.util.List[StructField])" in "StructType" gets ClassCastException
In the previous method, fields.toArray will cast java.util.List[StructField] into Array[Object] which can not cast into Array[StructField], thus when invoking this method will throw "java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to [Lorg.apache.spark.sql.types.StructField;"
I directly cast java.util.List[StructField] into Array[StructField]  in this patch.

Author: mayuanwen <mayuanwen@qiyi.com>

Closes #9649 from jackieMaKing/Spark-11679.
2015-11-17 11:15:46 -08:00
Liang-Chi Hsieh d79d8b08ff [MINOR] [SQL] Fix randomly generated ArrayData in RowEncoderSuite
The randomly generated ArrayData used for the UDT `ExamplePoint` in `RowEncoderSuite` sometimes doesn't have enough elements. In this case, this test will fail. This patch is to fix it.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #9757 from viirya/fix-randomgenerated-udt.
2015-11-16 23:16:17 -08:00
Kevin Yu e01865af0d [SPARK-11447][SQL] change NullType to StringType during binaryComparison between NullType and StringType
During executing PromoteStrings rule, if one side of binaryComparison is StringType and the other side is not StringType, the current code will promote(cast) the StringType to DoubleType, and if the StringType doesn't contain the numbers, it will get null value. So if it is doing <=> (NULL-safe equal) with Null, it will not filter anything, caused the problem reported by this jira.

I proposal to the changes through this PR, can you review my code changes ?

This problem only happen for <=>, other operators works fine.

scala> val filteredDF = df.filter(df("column") > (new Column(Literal(null))))
filteredDF: org.apache.spark.sql.DataFrame = [column: string]

scala> filteredDF.show
+------+
|column|
+------+
+------+

scala> val filteredDF = df.filter(df("column") === (new Column(Literal(null))))
filteredDF: org.apache.spark.sql.DataFrame = [column: string]

scala> filteredDF.show
+------+
|column|
+------+
+------+

scala> df.registerTempTable("DF")

scala> sqlContext.sql("select * from DF where 'column' = NULL")
res27: org.apache.spark.sql.DataFrame = [column: string]

scala> res27.show
+------+
|column|
+------+
+------+

Author: Kevin Yu <qyu@us.ibm.com>

Closes #9720 from kevinyu98/working_on_spark-11447.
2015-11-16 22:54:29 -08:00
Reynold Xin fbad920dbf [SPARK-11768][SPARK-9196][SQL] Support now function in SQL (alias for current_timestamp).
This patch adds an alias for current_timestamp (now function).

Also fixes SPARK-9196 to re-enable the test case for current_timestamp.

Author: Reynold Xin <rxin@databricks.com>

Closes #9753 from rxin/SPARK-11768.
2015-11-16 20:47:46 -08:00
gatorsmile 75ee12f09c [SPARK-8658][SQL] AttributeReference's equals method compares all the members
This fix is to change the equals method to check all of the specified fields for equality of AttributeReference.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9216 from gatorsmile/namedExpressEqual.
2015-11-16 15:22:12 -08:00
Bartlomiej Alberski 31296628ac [SPARK-11553][SQL] Primitive Row accessors should not convert null to default value
Invocation of getters for type extending AnyVal returns default value (if field value is null) instead of throwing NPE. Please check comments for SPARK-11553 issue for more details.

Author: Bartlomiej Alberski <bartlomiej.alberski@allegrogroup.com>

Closes #9642 from alberskib/bugfix/SPARK-11553.
2015-11-16 15:14:38 -08:00
Wenchen Fan b1a9662623 [SPARK-11754][SQL] consolidate ExpressionEncoder.tuple and Encoders.tuple
These 2 are very similar, we can consolidate them into one.

Also add tests for it and fix a bug.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9729 from cloud-fan/tuple.
2015-11-16 12:45:34 -08:00
Liang-Chi Hsieh b0c3fd34e4 [SPARK-11743] [SQL] Add UserDefinedType support to RowEncoder
JIRA: https://issues.apache.org/jira/browse/SPARK-11743

RowEncoder doesn't support UserDefinedType now. We should add the support for it.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #9712 from viirya/rowencoder-udt.
2015-11-16 09:03:42 -08:00
Wenchen Fan 06f1fdba6d [SPARK-11752] [SQL] fix timezone problem for DateTimeUtils.getSeconds
code snippet to reproduce it:
```
TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
val t = Timestamp.valueOf("1900-06-11 12:14:50.789")
val us = fromJavaTimestamp(t)
assert(getSeconds(us) === t.getSeconds)
```

it will be good to add a regression test for it, but the reproducing code need to change the default timezone, and even we change it back, the `lazy val defaultTimeZone` in `DataTimeUtils` is fixed.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9728 from cloud-fan/seconds.
2015-11-16 08:58:40 -08:00
Yin Huai 3e2e1873b2 [SPARK-11738] [SQL] Making ArrayType orderable
https://issues.apache.org/jira/browse/SPARK-11738

Author: Yin Huai <yhuai@databricks.com>

Closes #9718 from yhuai/makingArrayOrderable.
2015-11-15 13:59:59 -08:00
Yin Huai d83c2f9f0b [SPARK-11736][SQL] Add monotonically_increasing_id to function registry.
https://issues.apache.org/jira/browse/SPARK-11736

Author: Yin Huai <yhuai@databricks.com>

Closes #9703 from yhuai/MonotonicallyIncreasingID.
2015-11-14 21:04:18 -08:00
Wenchen Fan d7b2b97ad6 [SPARK-11727][SQL] Split ExpressionEncoder into FlatEncoder and ProductEncoder
also add more tests for encoders, and fix bugs that I found:

* when convert array to catalyst array, we can only skip element conversion for native types(e.g. int, long, boolean), not `AtomicType`(String is AtomicType but we need to convert it)
* we should also handle scala `BigDecimal` when convert from catalyst `Decimal`.
* complex map type should be supported

other issues that still in investigation:

* encode java `BigDecimal` and decode it back, seems we will loss precision info.
* when encode case class that defined inside a object, `ClassNotFound` exception will be thrown.

I'll remove unused code in a follow-up PR.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9693 from cloud-fan/split.
2015-11-13 11:25:33 -08:00
Wenchen Fan 23b8188f75 [SPARK-11654][SQL][FOLLOW-UP] fix some mistakes and clean up
* rename `AppendColumn` to `AppendColumns` to be consistent with the physical plan name.
* clean up stale comments.
* always pass in resolved encoder to `TypedColumn.withInputType`(test added)
* enable a mistakenly disabled java test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9688 from cloud-fan/follow.
2015-11-13 11:13:09 -08:00
Michael Armbrust 41bbd23004 [SPARK-11654][SQL] add reduce to GroupedDataset
This PR adds a new method, `reduce`, to `GroupedDataset`, which allows similar operations to `reduceByKey` on a traditional `PairRDD`.

```scala
val ds = Seq("abc", "xyz", "hello").toDS()
ds.groupBy(_.length).reduce(_ + _).collect()  // not actually commutative :P

res0: Array(3 -> "abcxyz", 5 -> "hello")
```

While implementing this method and its test cases several more deficiencies were found in our encoder handling.  Specifically, in order to support positional resolution, named resolution and tuple composition, it is important to keep the unresolved encoder around and to use it when constructing new `Datasets` with the same object type but different output attributes.  We now divide the encoder lifecycle into three phases (that mirror the lifecycle of standard expressions) and have checks at various boundaries:

 - Unresoved Encoders: all users facing encoders (those constructed by implicits, static methods, or tuple composition) are unresolved, meaning they have only `UnresolvedAttributes` for named fields and `BoundReferences` for fields accessed by ordinal.
 - Resolved Encoders: internal to a `[Grouped]Dataset` the encoder is resolved, meaning all input has been resolved to a specific `AttributeReference`.  Any encoders that are placed into a logical plan for use in object construction should be resolved.
 - BoundEncoder: Are constructed by physical plans, right before actual conversion from row -> object is performed.

It is left to future work to add explicit checks for resolution and provide good error messages when it fails.  We might also consider enforcing the above constraints in the type system (i.e. `fromRow` only exists on a `ResolvedEncoder`), but we should probably wait before spending too much time on this.

Author: Michael Armbrust <michael@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9673 from marmbrus/pr/9628.
2015-11-12 17:20:30 -08:00
JihongMa d292f74831 [SPARK-11420] Updating Stddev support via Imperative Aggregate
switched stddev support from DeclarativeAggregate to ImperativeAggregate.

Author: JihongMa <linlin200605@gmail.com>

Closes #9380 from JihongMA/SPARK-11420.
2015-11-12 13:47:34 -08:00
Reynold Xin 30e7433643 [SPARK-11673][SQL] Remove the normal Project physical operator (and keep TungstenProject)
Also make full outer join being able to produce UnsafeRows.

Author: Reynold Xin <rxin@databricks.com>

Closes #9643 from rxin/SPARK-11673.
2015-11-12 08:14:08 -08:00
Daoyuan Wang 39b1e36fbc [SPARK-11396] [SQL] add native implementation of datetime function to_unix_timestamp
`to_unix_timestamp` is the deterministic version of `unix_timestamp`, as it accepts at least one parameters.

Since the behavior here is quite similar to `unix_timestamp`, I think the dataframe API is not necessary here.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #9347 from adrian-wang/to_unix_timestamp.
2015-11-11 20:36:21 -08:00
Andrew Ray b8ff6888e7 [SPARK-8992][SQL] Add pivot to dataframe api
This adds a pivot method to the dataframe api.

Following the lead of cube and rollup this adds a Pivot operator that is translated into an Aggregate by the analyzer.

Currently the syntax is like:
~~courseSales.pivot(Seq($"year"), $"course", Seq("dotNET", "Java"), sum($"earnings"))~~

~~Would we be interested in the following syntax also/alternatively? and~~

    courseSales.groupBy($"year").pivot($"course", "dotNET", "Java").agg(sum($"earnings"))
    //or
    courseSales.groupBy($"year").pivot($"course").agg(sum($"earnings"))

Later we can add it to `SQLParser`, but as Hive doesn't support it we cant add it there, right?

~~Also what would be the suggested Java friendly method signature for this?~~

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #7841 from aray/sql-pivot.
2015-11-11 16:23:24 -08:00