Commit graph

3002 commits

Author SHA1 Message Date
Sameer Agarwal c3a6269ca9 [SPARK-13789] Infer additional constraints from attribute equality
## What changes were proposed in this pull request?

This PR adds support for inferring an additional set of data constraints based on attribute equality. For e.g., if an operator has constraints of the form (`a = 5`, `a = b`), we can now automatically infer an additional constraint of the form `b = 5`

## How was this patch tested?

Tested that new constraints are properly inferred for filters (by adding a new test) and equi-joins (by modifying an existing test)

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11618 from sameeragarwal/infer-isequal-constraints.
2016-03-10 17:29:45 -08:00
Cheng Lian 1d542785b9 [SPARK-13244][SQL] Migrates DataFrame to Dataset
## What changes were proposed in this pull request?

This PR unifies DataFrame and Dataset by migrating existing DataFrame operations to Dataset and make `DataFrame` a type alias of `Dataset[Row]`.

Most Scala code changes are source compatible, but Java API is broken as Java knows nothing about Scala type alias (mostly replacing `DataFrame` with `Dataset<Row>`).

There are several noticeable API changes related to those returning arrays:

1.  `collect`/`take`

    -   Old APIs in class `DataFrame`:

        ```scala
        def collect(): Array[Row]
        def take(n: Int): Array[Row]
        ```

    -   New APIs in class `Dataset[T]`:

        ```scala
        def collect(): Array[T]
        def take(n: Int): Array[T]

        def collectRows(): Array[Row]
        def takeRows(n: Int): Array[Row]
        ```

    Two specialized methods `collectRows` and `takeRows` are added because Java doesn't support returning generic arrays. Thus, for example, `DataFrame.collect(): Array[T]` actually returns `Object` instead of `Array<T>` from Java side.

    Normally, Java users may fall back to `collectAsList` and `takeAsList`.  The two new specialized versions are added to avoid performance regression in ML related code (but maybe I'm wrong and they are not necessary here).

1.  `randomSplit`

    -   Old APIs in class `DataFrame`:

        ```scala
        def randomSplit(weights: Array[Double], seed: Long): Array[DataFrame]
        def randomSplit(weights: Array[Double]): Array[DataFrame]
        ```

    -   New APIs in class `Dataset[T]`:

        ```scala
        def randomSplit(weights: Array[Double], seed: Long): Array[Dataset[T]]
        def randomSplit(weights: Array[Double]): Array[Dataset[T]]
        ```

    Similar problem as above, but hasn't been addressed for Java API yet.  We can probably add `randomSplitAsList` to fix this one.

1.  `groupBy`

    Some original `DataFrame.groupBy` methods have conflicting signature with original `Dataset.groupBy` methods.  To distinguish these two, typed `Dataset.groupBy` methods are renamed to `groupByKey`.

Other noticeable changes:

1.  Dataset always do eager analysis now

    We used to support disabling DataFrame eager analysis to help reporting partially analyzed malformed logical plan on analysis failure.  However, Dataset encoders requires eager analysi during Dataset construction.  To preserve the error reporting feature, `AnalysisException` now takes an extra `Option[LogicalPlan]` argument to hold the partially analyzed plan, so that we can check the plan tree when reporting test failures.  This plan is passed by `QueryExecution.assertAnalyzed`.

## How was this patch tested?

Existing tests do the work.

## TODO

- [ ] Fix all tests
- [ ] Re-enable MiMA check
- [ ] Update ScalaDoc (`since`, `group`, and example code)

Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>
Author: Cheng Lian <liancheng@users.noreply.github.com>

Closes #11443 from liancheng/ds-to-df.
2016-03-10 17:00:17 -08:00
Davies Liu 020ff8cd34 [SPARK-13751] [SQL] generate better code for Filter
## What changes were proposed in this pull request?

This PR improve the codegen of Filter by:

1. filter out the rows early if it have null value in it that will cause the condition result in null or false. After this, we could simplify the condition, because the input are not nullable anymore.

2. Split the condition as conjunctive predicates, then check them one by one.

Here is a piece of generated code for Filter in TPCDS Q55:
```java
/* 109 */       /*** CONSUME: Filter ((((isnotnull(d_moy#149) && isnotnull(d_year#147)) && (d_moy#149 = 11)) && (d_year#147 = 1999)) && isnotnull(d_date_sk#141)) */
/* 110 */       /* input[0, int] */
/* 111 */       boolean project_isNull2 = rdd_row.isNullAt(0);
/* 112 */       int project_value2 = project_isNull2 ? -1 : (rdd_row.getInt(0));
/* 113 */       /* input[1, int] */
/* 114 */       boolean project_isNull3 = rdd_row.isNullAt(1);
/* 115 */       int project_value3 = project_isNull3 ? -1 : (rdd_row.getInt(1));
/* 116 */       /* input[2, int] */
/* 117 */       boolean project_isNull4 = rdd_row.isNullAt(2);
/* 118 */       int project_value4 = project_isNull4 ? -1 : (rdd_row.getInt(2));
/* 119 */
/* 120 */       if (project_isNull3) continue;
/* 121 */       if (project_isNull4) continue;
/* 122 */       if (project_isNull2) continue;
/* 123 */
/* 124 */       /* (input[1, int] = 11) */
/* 125 */       boolean filter_value6 = false;
/* 126 */       filter_value6 = project_value3 == 11;
/* 127 */       if (!filter_value6) continue;
/* 128 */
/* 129 */       /* (input[2, int] = 1999) */
/* 130 */       boolean filter_value9 = false;
/* 131 */       filter_value9 = project_value4 == 1999;
/* 132 */       if (!filter_value9) continue;
/* 133 */
/* 134 */       filter_metricValue1.add(1);
/* 135 */
/* 136 */       /*** CONSUME: Project [d_date_sk#141] */
/* 137 */
/* 138 */       project_rowWriter1.write(0, project_value2);
/* 139 */       append(project_result1.copy());
```

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #11585 from davies/gen_filter.
2016-03-10 16:40:16 -08:00
Dongjoon Hyun 91fed8e9c5 [SPARK-3854][BUILD] Scala style: require spaces before {.
## What changes were proposed in this pull request?

Since the opening curly brace, '{', has many usages as discussed in [SPARK-3854](https://issues.apache.org/jira/browse/SPARK-3854), this PR adds a ScalaStyle rule to prevent '){' pattern  for the following majority pattern and fixes the code accordingly. If we enforce this in ScalaStyle from now, it will improve the Scala code quality and reduce review time.
```
// Correct:
if (true) {
  println("Wow!")
}

// Incorrect:
if (true){
   println("Wow!")
}
```
IntelliJ also shows new warnings based on this.

## How was this patch tested?

Pass the Jenkins ScalaStyle test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11637 from dongjoon-hyun/SPARK-3854.
2016-03-10 15:57:22 -08:00
Tathagata Das 3d2b6f56e3 [SQL][TEST] Increased timeouts to reduce flakiness in ContinuousQueryManagerSuite
## What changes were proposed in this pull request?

ContinuousQueryManager is sometimes flaky on Jenkins. I could not reproduce it on my machine, so I guess it about the waiting times which causes problems if Jenkins is loaded. I have increased the wait time in the hope that it will be less flaky.

## How was this patch tested?

I reran the unit test many times on a loop in my machine. I am going to run it a few time in Jenkins, that's the real test.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #11638 from tdas/cqm-flaky-test.
2016-03-10 14:38:19 -08:00
Nong Li 747d2f5381 [SPARK-13790] Speed up ColumnVector's getDecimal
## What changes were proposed in this pull request?

We should reuse an object similar to the other non-primitive type getters. For
a query that computes averages over decimal columns, this shows a 10% speedup
on overall query times.

## How was this patch tested?

Existing tests and this benchmark

```
TPCDS Snappy:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
--------------------------------------------------------------------------------
q27-agg (master)                       10627 / 11057         10.8          92.3
q27-agg (this patch)                     9722 / 9832         11.8          84.4
```

Author: Nong Li <nong@databricks.com>

Closes #11624 from nongli/spark-13790.
2016-03-10 13:31:19 -08:00
Sameer Agarwal 19f4ac6dc7 [SPARK-13759][SQL] Add IsNotNull constraints for expressions with an inequality
## What changes were proposed in this pull request?

This PR adds support for inferring `IsNotNull` constraints from expressions with an `!==`. More specifically, if an operator has a condition on `a !== b`, we know that both `a` and `b` in the operator output can no longer be null.

## How was this patch tested?

1. Modified a test in `ConstraintPropagationSuite` to test for expressions with an inequality.
2. Added a test in `NullFilteringSuite` for making sure an Inner join with a "non-equal" condition appropriately filters out null from their input.

cc nongli

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11594 from sameeragarwal/isnotequal-constraints.
2016-03-10 12:16:46 -08:00
Liang-Chi Hsieh d24801ad28 [SPARK-13636] [SQL] Directly consume UnsafeRow in wholestage codegen plans
JIRA: https://issues.apache.org/jira/browse/SPARK-13636

## What changes were proposed in this pull request?

As shown in the wholestage codegen verion of Sort operator, when Sort is top of Exchange (or other operator that produce UnsafeRow), we will create variables from UnsafeRow, than create another UnsafeRow using these variables. We should avoid the unnecessary unpack and pack variables from UnsafeRows.

## How was this patch tested?

All existing wholestage codegen tests should be passed.

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

Closes #11484 from viirya/direct-consume-unsaferow.
2016-03-10 10:04:56 -08:00
Dongjoon Hyun 9525c563de [MINOR][SQL] Replace DataFrameWriter.stream() with startStream() in comments.
## What changes were proposed in this pull request?

According to #11627 , this PR replace `DataFrameWriter.stream()` with `startStream()` in comments of `ContinuousQueryListener.java`.

## How was this patch tested?

Manual. (It changes on comments.)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11629 from dongjoon-hyun/minor_rename.
2016-03-09 23:54:00 -08:00
Reynold Xin 8a3acb792d [SPARK-13794][SQL] Rename DataFrameWriter.stream() DataFrameWriter.startStream()
## What changes were proposed in this pull request?
The new name makes it more obvious with the verb "start" that we are actually starting some execution.

## How was this patch tested?
This is just a rename. Existing unit tests should cover it.

Author: Reynold Xin <rxin@databricks.com>

Closes #11627 from rxin/SPARK-13794.
2016-03-09 21:04:56 -08:00
hyukjinkwon aa0eba2c35 [SPARK-13766][SQL] Consistent file extensions for files written by internal data sources
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-13766
This PR makes the file extensions (written by internal datasource) consistent.

**Before**

- TEXT, CSV and JSON
```
[.COMPRESSION_CODEC_NAME]
```

- Parquet
```
[.COMPRESSION_CODEC_NAME].parquet
```

- ORC
```
.orc
```

**After**

- TEXT, CSV and JSON
```
.txt[.COMPRESSION_CODEC_NAME]
.csv[.COMPRESSION_CODEC_NAME]
.json[.COMPRESSION_CODEC_NAME]
```

- Parquet
```
[.COMPRESSION_CODEC_NAME].parquet
```

- ORC
```
[.COMPRESSION_CODEC_NAME].orc
```

When the compression codec is set,
- For Parquet and ORC, each still stays in Parquet and ORC format but just have compressed data internally. So, I think it is okay to name `.parquet` and `.orc` at the end.

- For Text, CSV and JSON, each does not stays in each format but it has different data format according to compression codec. So, each has the names `.json`, `.csv` and `.txt` before the compression extension.

## How was this patch tested?

Unit tests are used and `./dev/run_tests` for coding style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11604 from HyukjinKwon/SPARK-13766.
2016-03-09 19:12:46 -08:00
Yin Huai 790646125e Revert "[SPARK-13760][SQL] Fix BigDecimal constructor for FloatType"
This reverts commit 926e9c45a2.
2016-03-09 18:41:38 -08:00
Sameer Agarwal 926e9c45a2 [SPARK-13760][SQL] Fix BigDecimal constructor for FloatType
## What changes were proposed in this pull request?

A very minor change for using `BigDecimal.decimal(f: Float)` instead of `BigDecimal(f: float)`. The latter is deprecated and can result in inconsistencies due to an implicit conversion to `Double`.

## How was this patch tested?

N/A

cc yhuai

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11597 from sameeragarwal/bigdecimal.
2016-03-09 18:16:29 -08:00
Andrew Or 37fcda3e6c [SPARK-13747][SQL] Fix concurrent query with fork-join pool
## What changes were proposed in this pull request?

Fix this use case, which was already fixed in SPARK-10548 in 1.6 but was broken in master due to #9264:

```
(1 to 100).par.foreach { _ => sc.parallelize(1 to 5).map { i => (i, i) }.toDF("a", "b").count() }
```

This threw `IllegalArgumentException` consistently before this patch. For more detail, see the JIRA.

## How was this patch tested?

New test in `SQLExecutionSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #11586 from andrewor14/fix-concurrent-sql.
2016-03-09 17:34:28 -08:00
Sameer Agarwal dbf2a7cfad [SPARK-13781][SQL] Use ExpressionSets in ConstraintPropagationSuite
## What changes were proposed in this pull request?

This PR is a small follow up on https://github.com/apache/spark/pull/11338 (https://issues.apache.org/jira/browse/SPARK-13092) to use `ExpressionSet` as part of the verification logic in `ConstraintPropagationSuite`.
## How was this patch tested?

No new tests added. Just changes the verification logic in `ConstraintPropagationSuite`.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11611 from sameeragarwal/expression-set.
2016-03-09 15:27:18 -08:00
gatorsmile c6aa356cd8 [SPARK-13527][SQL] Prune Filters based on Constraints
#### What changes were proposed in this pull request?

Remove all the deterministic conditions in a [[Filter]] that are contained in the Child's Constraints.

For example, the first query can be simplified to the second one.

```scala
    val queryWithUselessFilter = tr1
      .where("tr1.a".attr > 10 || "tr1.c".attr < 10)
      .join(tr2.where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr))
      .where(
        ("tr1.a".attr > 10 || "tr1.c".attr < 10) &&
        'd.attr < 100 &&
        "tr2.a".attr === "tr1.a".attr)
```
```scala
    val query = tr1
      .where("tr1.a".attr > 10 || "tr1.c".attr < 10)
      .join(tr2.where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr))
```
#### How was this patch tested?

Six test cases are added.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11406 from gatorsmile/FilterRemoval.
2016-03-09 12:50:55 -08:00
Davies Liu 3dc9ae2e15 [SPARK-13523] [SQL] Reuse exchanges in a query
## What changes were proposed in this pull request?

It’s possible to have common parts in a query, for example, self join, it will be good to avoid the duplicated part to same CPUs and memory (Broadcast or cache).

Exchange will materialize the underlying RDD by shuffle or collect, it’s a great point to check duplicates and reuse them. Duplicated exchanges means they generate exactly the same result inside a query.

In order to find out the duplicated exchanges, we should be able to compare SparkPlan to check that they have same results or not. We already have that for LogicalPlan, so we should move that into QueryPlan to make it available for SparkPlan.

Once we can find the duplicated exchanges, we should replace all of them with same SparkPlan object (could be wrapped by ReusedExchage for explain), then the plan tree become a DAG. Since all the planner only work with tree, so this rule should be the last one for the entire planning.

After the rule, the plan will looks like:

```
WholeStageCodegen
:  +- Project [id#0L]
:     +- BroadcastHashJoin [id#0L], [id#2L], Inner, BuildRight, None
:        :- Project [id#0L]
:        :  +- BroadcastHashJoin [id#0L], [id#1L], Inner, BuildRight, None
:        :     :- Range 0, 1, 4, 1024, [id#0L]
:        :     +- INPUT
:        +- INPUT
:- BroadcastExchange HashedRelationBroadcastMode(true,List(id#1L),List(id#1L))
:  +- WholeStageCodegen
:     :  +- Range 0, 1, 4, 1024, [id#1L]
+- ReusedExchange [id#2L], BroadcastExchange HashedRelationBroadcastMode(true,List(id#1L),List(id#1L))
```

![bjoin](https://cloud.githubusercontent.com/assets/40902/13414787/209e8c5c-df0a-11e5-8a0f-edff69d89e83.png)

For three ways SortMergeJoin,
```
== Physical Plan ==
WholeStageCodegen
:  +- Project [id#0L]
:     +- SortMergeJoin [id#0L], [id#4L], None
:        :- INPUT
:        +- INPUT
:- WholeStageCodegen
:  :  +- Project [id#0L]
:  :     +- SortMergeJoin [id#0L], [id#3L], None
:  :        :- INPUT
:  :        +- INPUT
:  :- WholeStageCodegen
:  :  :  +- Sort [id#0L ASC], false, 0
:  :  :     +- INPUT
:  :  +- Exchange hashpartitioning(id#0L, 200), None
:  :     +- WholeStageCodegen
:  :        :  +- Range 0, 1, 4, 33554432, [id#0L]
:  +- WholeStageCodegen
:     :  +- Sort [id#3L ASC], false, 0
:     :     +- INPUT
:     +- ReusedExchange [id#3L], Exchange hashpartitioning(id#0L, 200), None
+- WholeStageCodegen
   :  +- Sort [id#4L ASC], false, 0
   :     +- INPUT
   +- ReusedExchange [id#4L], Exchange hashpartitioning(id#0L, 200), None
```
![sjoin](https://cloud.githubusercontent.com/assets/40902/13414790/27aea61c-df0a-11e5-8cbf-fbc985c31d95.png)

If the same ShuffleExchange or BroadcastExchange, execute()/executeBroadcast() will be called by different parents, they should cached the RDD/Broadcast, return the same one for all the parents.

## How was this patch tested?

Added some unit tests for this.  Had done some manual tests on TPCDS query Q59 and Q64, we can see some exchanges are re-used (this requires a change in PhysicalRDD to for sameResult, is be done in #11514 ).

Author: Davies Liu <davies@databricks.com>

Closes #11403 from davies/dedup.
2016-03-09 12:04:29 -08:00
hyukjinkwon cad29a40b2 [SPARK-13728][SQL] Fix ORC PPD test so that pushed filters can be checked.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13728

https://github.com/apache/spark/pull/11509 makes the output only single ORC file.
It was 10 files but this PR writes only single file. So, this could not skip stripes in ORC by the pushed down filters.
So, this PR simply repartitions data into 10 so that the test could pass.
## How was this patch tested?

unittest and `./dev/run_tests` for code style test.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11593 from HyukjinKwon/SPARK-13728.
2016-03-09 10:48:53 -08:00
gatorsmile 23369c3bd2 [SPARK-13763][SQL] Remove Project when its Child's Output is Nil
#### What changes were proposed in this pull request?

As shown in another PR: https://github.com/apache/spark/pull/11596, we are using `SELECT 1` as a dummy table, when the table is used for SQL statements in which a table reference is required, but the contents of the table are not important. For example,

```SQL
SELECT value FROM (select 1) dummyTable Lateral View explode(array(1,2,3)) adTable as value
```
Before the PR, the optimized plan contains a useless `Project` after Optimizer executing the `ColumnPruning` rule, as shown below:

```
== Analyzed Logical Plan ==
value: int
Project [value#22]
+- Generate explode(array(1, 2, 3)), true, false, Some(adtable), [value#22]
   +- SubqueryAlias dummyTable
      +- Project [1 AS 1#21]
         +- OneRowRelation$

== Optimized Logical Plan ==
Generate explode([1,2,3]), false, false, Some(adtable), [value#22]
+- Project
   +- OneRowRelation$
```

After the fix, the optimized plan removed the useless `Project`, as shown below:
```
== Optimized Logical Plan ==
Generate explode([1,2,3]), false, false, Some(adtable), [value#22]
+- OneRowRelation$
```

This PR is to remove `Project` when its Child's output is Nil

#### How was this patch tested?

Added a new unit test case into the suite `ColumnPruningSuite.scala`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11599 from gatorsmile/projectOneRowRelation.
2016-03-09 10:29:27 -08:00
Davies Liu 7791d0c3a9 Revert "[SPARK-13668][SQL] Reorder filter/join predicates to short-circuit isNotNull checks"
This reverts commit e430614eae.
2016-03-09 10:05:57 -08:00
Davies Liu 9634e17d01 [SPARK-13242] [SQL] codegen fallback in case-when if there many branches
## What changes were proposed in this pull request?

If there are many branches in a CaseWhen expression, the generated code could go above the 64K limit for single java method, will fail to compile. This PR change it to fallback to interpret mode if there are more than 20 branches.

This PR is based on #11243 and #11221, thanks to joehalliwell

Closes #11243
Closes #11221

## How was this patch tested?

Add a test with 50 branches.

Author: Davies Liu <davies@databricks.com>

Closes #11592 from davies/fix_when.
2016-03-09 09:27:28 -08:00
Dilip Biswal 53ba6d6e59 [SPARK-13698][SQL] Fix Analysis Exceptions when Using Backticks in Generate
## What changes were proposed in this pull request?
Analysis exception occurs while running the following query.
```
SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) `a` AS `ints`
```
```
Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve '`ints`' given input columns: [a, `ints`]; line 1 pos 7
'Project ['ints]
+- Generate explode(a#0.b), true, false, Some(a), [`ints`#8]
   +- SubqueryAlias nestedarray
      +- LocalRelation [a#0], [[[[1,2,3]]]]
```

## How was this patch tested?

Added new unit tests in SQLQuerySuite and HiveQlSuite

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

Closes #11538 from dilipbiswal/SPARK-13698.
2016-03-09 21:49:37 +08:00
Dongjoon Hyun c3689bc24e [SPARK-13702][CORE][SQL][MLLIB] Use diamond operator for generic instance creation in Java code.
## What changes were proposed in this pull request?

In order to make `docs/examples` (and other related code) more simple/readable/user-friendly, this PR replaces existing codes like the followings by using `diamond` operator.

```
-    final ArrayList<Product2<Object, Object>> dataToWrite =
-      new ArrayList<Product2<Object, Object>>();
+    final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>();
```

Java 7 or higher supports **diamond** operator which replaces the type arguments required to invoke the constructor of a generic class with an empty set of type parameters (<>). Currently, Spark Java code use mixed usage of this.

## How was this patch tested?

Manual.
Pass the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11541 from dongjoon-hyun/SPARK-13702.
2016-03-09 10:31:26 +00:00
Takuya UESHIN 2c5af7d4d9 [SPARK-13640][SQL] Synchronize ScalaReflection.mirror method.
## What changes were proposed in this pull request?

`ScalaReflection.mirror` method should be synchronized when scala version is `2.10` because `universe.runtimeMirror` is not thread safe.

## How was this patch tested?

I added a test to check thread safety of `ScalaRefection.mirror` method in `ScalaReflectionSuite`, which will throw the following Exception in Scala `2.10` without this patch:

```
[info] - thread safety of mirror *** FAILED *** (49 milliseconds)
[info]   java.lang.UnsupportedOperationException: tail of empty list
[info]   at scala.collection.immutable.Nil$.tail(List.scala:339)
[info]   at scala.collection.immutable.Nil$.tail(List.scala:334)
[info]   at scala.reflect.internal.SymbolTable.popPhase(SymbolTable.scala:172)
[info]   at scala.reflect.internal.Symbols$Symbol.unsafeTypeParams(Symbols.scala:1477)
[info]   at scala.reflect.internal.Symbols$TypeSymbol.tpe(Symbols.scala:2777)
[info]   at scala.reflect.internal.Mirrors$RootsBase.init(Mirrors.scala:235)
[info]   at scala.reflect.runtime.JavaMirrors$class.createMirror(JavaMirrors.scala:34)
[info]   at scala.reflect.runtime.JavaMirrors$class.runtimeMirror(JavaMirrors.scala:61)
[info]   at scala.reflect.runtime.JavaUniverse.runtimeMirror(JavaUniverse.scala:12)
[info]   at scala.reflect.runtime.JavaUniverse.runtimeMirror(JavaUniverse.scala:12)
[info]   at org.apache.spark.sql.catalyst.ScalaReflection$.mirror(ScalaReflection.scala:36)
[info]   at org.apache.spark.sql.catalyst.ScalaReflectionSuite$$anonfun$12$$anonfun$apply$mcV$sp$1$$anonfun$apply$1$$anonfun$apply$2.apply(ScalaReflectionSuite.scala:256)
[info]   at org.apache.spark.sql.catalyst.ScalaReflectionSuite$$anonfun$12$$anonfun$apply$mcV$sp$1$$anonfun$apply$1$$anonfun$apply$2.apply(ScalaReflectionSuite.scala:252)
[info]   at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
[info]   at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
[info]   at scala.concurrent.impl.ExecutionContextImpl$$anon$3.exec(ExecutionContextImpl.scala:107)
[info]   at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
[info]   at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
[info]   at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
[info]   at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
```

Notice that the test will pass when Scala version is `2.11`.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #11487 from ueshin/issues/SPARK-13640.
2016-03-09 10:23:27 +00:00
Dongjoon Hyun f3201aeeb0 [SPARK-13692][CORE][SQL] Fix trivial Coverity/Checkstyle defects
## What changes were proposed in this pull request?

This issue fixes the following potential bugs and Java coding style detected by Coverity and Checkstyle.

- Implement both null and type checking in equals functions.
- Fix wrong type casting logic in SimpleJavaBean2.equals.
- Add `implement Cloneable` to `UTF8String` and `SortedIterator`.
- Remove dereferencing before null check in `AbstractBytesToBytesMapSuite`.
- Fix coding style: Add '{}' to single `for` statement in mllib examples.
- Remove unused imports in `ColumnarBatch` and `JavaKinesisStreamSuite`.
- Remove unused fields in `ChunkFetchIntegrationSuite`.
- Add `stop()` to prevent resource leak.

Please note that the last two checkstyle errors exist on newly added commits after [SPARK-13583](https://issues.apache.org/jira/browse/SPARK-13583).

## How was this patch tested?

manual via `./dev/lint-java` and Coverity site.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11530 from dongjoon-hyun/SPARK-13692.
2016-03-09 10:12:23 +00:00
Jakob Odersky 035d3acdf3 [SPARK-7286][SQL] Deprecate !== in favour of =!=
This PR replaces #9925 which had issues with CI. **Please see the original PR for any previous discussions.**

## What changes were proposed in this pull request?
Deprecate the SparkSQL column operator !== and use =!= as an alternative.
Fixes subtle issues related to operator precedence (basically, !== does not have the same priority as its logical negation, ===).

## How was this patch tested?
All currently existing tests.

Author: Jakob Odersky <jodersky@gmail.com>

Closes #11588 from jodersky/SPARK-7286.
2016-03-08 18:11:09 -08:00
Hossein cc4ab37ee7 [SPARK-13754] Keep old data source name for backwards compatibility
## Motivation
CSV data source was contributed by Databricks. It is the inlined version of https://github.com/databricks/spark-csv. The data source name was `com.databricks.spark.csv`. As a result there are many tables created on older versions of spark with that name as the source. For backwards compatibility we should keep the old name.

## Proposed changes
`com.databricks.spark.csv` was added to list of `backwardCompatibilityMap` in `ResolvedDataSource.scala`

## Tests
A unit test was added to `CSVSuite` to parse a csv file using the old name.

Author: Hossein <hossein@databricks.com>

Closes #11589 from falaki/SPARK-13754.
2016-03-08 17:45:15 -08:00
Davies Liu 982ef2b87e [SPARK-13750][SQL] fix sizeInBytes of HadoopFsRelation
## What changes were proposed in this pull request?

This PR fix the sizeInBytes of HadoopFsRelation.

## How was this patch tested?

Added regression test for that.

Author: Davies Liu <davies@databricks.com>

Closes #11590 from davies/fix_sizeInBytes.
2016-03-08 17:42:52 -08:00
Josh Rosen 81f54acc9c [SPARK-13755] Escape quotes in SQL plan visualization node labels
When generating Graphviz DOT files in the SQL query visualization we need to escape double-quotes inside node labels. This is a followup to #11309, which fixed a similar graph in Spark Core's DAG visualization.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11587 from JoshRosen/graphviz-escaping.
2016-03-08 16:28:22 -08:00
Sameer Agarwal e430614eae [SPARK-13668][SQL] Reorder filter/join predicates to short-circuit isNotNull checks
## What changes were proposed in this pull request?

If a filter predicate or a join condition consists of `IsNotNull` checks, we should reorder these checks such that these non-nullability checks are evaluated before the rest of the predicates.

For e.g., if a filter predicate is of the form `a > 5 && isNotNull(b)`, we should rewrite this as `isNotNull(b) && a > 5` during physical plan generation.

## How was this patch tested?

new unit tests that verify the physical plan for both filters and joins in `ReorderedPredicateSuite`

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11511 from sameeragarwal/reorder-isnotnull.
2016-03-08 15:40:45 -08:00
Michael Armbrust 1e28840594 [SPARK-13738][SQL] Cleanup Data Source resolution
Follow-up to #11509, that simply refactors the interface that we use when resolving a pluggable `DataSource`.
 - Multiple functions share the same set of arguments so we make this a case class, called `DataSource`.  Actual resolution is now done by calling a function on this class.
 - Instead of having multiple methods named `apply` (some of which do writing some of which do reading) we now explicitly have `resolveRelation()` and `write(mode, df)`.
 - Get rid of `Array[String]` since this is an internal API and was forcing us to awkwardly call `toArray` in a bunch of places.

Author: Michael Armbrust <michael@databricks.com>

Closes #11572 from marmbrus/dataSourceResolution.
2016-03-08 15:19:26 -08:00
Dongjoon Hyun 076009b949 [SPARK-13400] Stop using deprecated Octal escape literals
## What changes were proposed in this pull request?

This removes the remaining deprecated Octal escape literals. The followings are the warnings on those two lines.
```
LiteralExpressionSuite.scala:99: Octal escape literals are deprecated, use \u0000 instead.
HiveQlSuite.scala:74: Octal escape literals are deprecated, use \u002c instead.
```

## How was this patch tested?

Manual.
During building, there should be no warning on `Octal escape literals`.
```
mvn -DskipTests clean install
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11584 from dongjoon-hyun/SPARK-13400.
2016-03-08 15:00:26 -08:00
Wenchen Fan 46881b4ea2 [SPARK-12727][SQL] support SQL generation for aggregate with multi-distinct
## What changes were proposed in this pull request?

This PR add SQL generation support for aggregate with multi-distinct, by simply moving the `DistinctAggregationRewriter` rule to optimizer.

More discussions are needed as this breaks an import contract: analyzed plan should be able to run without optimization.  However, the `ComputeCurrentTime` rule has kind of broken it already, and I think maybe we should add a new phase for this kind of rules, because strictly speaking they don't belong to analysis and is coupled with the physical plan implementation.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11579 from cloud-fan/distinct.
2016-03-08 11:45:08 -08:00
Davies Liu 78d3b6051e [SPARK-13657] [SQL] Support parsing very long AND/OR expressions
## What changes were proposed in this pull request?

In order to avoid StackOverflow when parse a expression with hundreds of ORs, we should use loop instead of recursive functions to flatten the tree as list. This PR also build a balanced tree to reduce the depth of generated And/Or expression, to avoid StackOverflow in analyzer/optimizer.

## How was this patch tested?

Add new unit tests. Manually tested with TPCDS Q3 with hundreds predicates in it [1]. These predicates help to reduce the number of partitions, then the query time went from 60 seconds to 8 seconds.

[1] https://github.com/cloudera/impala-tpcds-kit/blob/master/queries/q3.sql

Author: Davies Liu <davies@databricks.com>

Closes #11501 from davies/long_or.
2016-03-08 10:23:19 -08:00
Wenchen Fan 7d05d02bff [SPARK-13637][SQL] use more information to simplify the code in Expand builder
## What changes were proposed in this pull request?

The code in `Expand.apply` can be simplified by existing information:

* the `groupByExprs` parameter are all `Attribute`s
* the `child` parameter is a `Project` that append aliased group by expressions to its child's output

## How was this patch tested?

by existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11485 from cloud-fan/expand.
2016-03-08 23:34:42 +08:00
Davies Liu 25bba58d16 [SPARK-13404] [SQL] Create variables for input row when it's actually used
## What changes were proposed in this pull request?

This PR change the way how we generate the code for the output variables passing from a plan to it's parent.

Right now, they are generated before call consume() of it's parent. It's not efficient, if the parent is a Filter or Join, which could filter out most the rows, the time to access some of the columns that are not used by the Filter or Join are wasted.

This PR try to improve this by defering the access of columns until they are actually used by a plan. After this PR, a plan does not need to generate code to evaluate the variables for output, just passing the ExprCode to its parent by `consume()`. In `parent.consumeChild()`, it will check the output from child and `usedInputs`, generate the code for those columns that is part of `usedInputs` before calling `doConsume()`.

This PR also change the `if` from
```
if (cond) {
  xxx
}
```
to
```
if (!cond) continue;
xxx
```
The new one could help to reduce the nested indents for multiple levels of Filter and BroadcastHashJoin.

It also added some comments for operators.

## How was the this patch tested?

Unit tests. Manually ran TPCDS Q55, this PR improve the performance about 30% (scale=10, from 2.56s to 1.96s)

Author: Davies Liu <davies@databricks.com>

Closes #11274 from davies/gen_defer.
2016-03-07 20:09:08 -08:00
Andrew Or da7bfac488 [SPARK-13689][SQL] Move helper things in CatalystQl to new utils object
## What changes were proposed in this pull request?

When we add more DDL parsing logic in the future, SparkQl will become very big. To keep it smaller, we'll introduce helper "parser objects", e.g. one to parse alter table commands. However, these parser objects will need to access some helper methods that exist in CatalystQl. The proposal is to move those methods to an isolated ParserUtils object.

This is based on viirya's changes in #11048. It prefaces the bigger fix for SPARK-13139 to make the diff of that patch smaller.

## How was this patch tested?

No change in functionality, so just Jenkins.

Author: Andrew Or <andrew@databricks.com>

Closes #11529 from andrewor14/parser-utils.
2016-03-07 18:01:27 -08:00
Tim Preece 46f25c2413 [SPARK-13648] Add Hive Cli to classes for isolated classloader
## What changes were proposed in this pull request?

Adding the hive-cli classes to the classloader

## How was this patch tested?

The hive Versionssuite tests were run

This is my original work and I license the work to the project under the project's open source license.

Author: Tim Preece <tim.preece.in.oz@gmail.com>

Closes #11495 from preecet/master.
2016-03-07 15:23:07 -08:00
Michael Armbrust e720dda42e [SPARK-13665][SQL] Separate the concerns of HadoopFsRelation
`HadoopFsRelation` is used for reading most files into Spark SQL.  However today this class mixes the concerns of file management, schema reconciliation, scan building, bucketing, partitioning, and writing data.  As a result, many data sources are forced to reimplement the same functionality and the various layers have accumulated a fair bit of inefficiency.  This PR is a first cut at separating this into several components / interfaces that are each described below.  Additionally, all implementations inside of Spark (parquet, csv, json, text, orc, svmlib) have been ported to the new API `FileFormat`.  External libraries, such as spark-avro will also need to be ported to work with Spark 2.0.

### HadoopFsRelation
A simple `case class` that acts as a container for all of the metadata required to read from a datasource.  All discovery, resolution and merging logic for schemas and partitions has been removed.  This an internal representation that no longer needs to be exposed to developers.

```scala
case class HadoopFsRelation(
    sqlContext: SQLContext,
    location: FileCatalog,
    partitionSchema: StructType,
    dataSchema: StructType,
    bucketSpec: Option[BucketSpec],
    fileFormat: FileFormat,
    options: Map[String, String]) extends BaseRelation
```

### FileFormat
The primary interface that will be implemented by each different format including external libraries.  Implementors are responsible for reading a given format and converting it into `InternalRow` as well as writing out an `InternalRow`.  A format can optionally return a schema that is inferred from a set of files.

```scala
trait FileFormat {
  def inferSchema(
      sqlContext: SQLContext,
      options: Map[String, String],
      files: Seq[FileStatus]): Option[StructType]

  def prepareWrite(
      sqlContext: SQLContext,
      job: Job,
      options: Map[String, String],
      dataSchema: StructType): OutputWriterFactory

  def buildInternalScan(
      sqlContext: SQLContext,
      dataSchema: StructType,
      requiredColumns: Array[String],
      filters: Array[Filter],
      bucketSet: Option[BitSet],
      inputFiles: Array[FileStatus],
      broadcastedConf: Broadcast[SerializableConfiguration],
      options: Map[String, String]): RDD[InternalRow]
}
```

The current interface is based on what was required to get all the tests passing again, but still mixes a couple of concerns (i.e. `bucketSet` is passed down to the scan instead of being resolved by the planner).  Additionally, scans are still returning `RDD`s instead of iterators for single files.  In a future PR, bucketing should be removed from this interface and the scan should be isolated to a single file.

### FileCatalog
This interface is used to list the files that make up a given relation, as well as handle directory based partitioning.

```scala
trait FileCatalog {
  def paths: Seq[Path]
  def partitionSpec(schema: Option[StructType]): PartitionSpec
  def allFiles(): Seq[FileStatus]
  def getStatus(path: Path): Array[FileStatus]
  def refresh(): Unit
}
```

Currently there are two implementations:
 - `HDFSFileCatalog` - based on code from the old `HadoopFsRelation`.  Infers partitioning by recursive listing and caches this data for performance
 - `HiveFileCatalog` - based on the above, but it uses the partition spec from the Hive Metastore.

### ResolvedDataSource
Produces a logical plan given the following description of a Data Source (which can come from DataFrameReader or a metastore):
 - `paths: Seq[String] = Nil`
 - `userSpecifiedSchema: Option[StructType] = None`
 - `partitionColumns: Array[String] = Array.empty`
 - `bucketSpec: Option[BucketSpec] = None`
 - `provider: String`
 - `options: Map[String, String]`

This class is responsible for deciding which of the Data Source APIs a given provider is using (including the non-file based ones).  All reconciliation of partitions, buckets, schema from metastores or inference is done here.

### DataSourceAnalysis / DataSourceStrategy
Responsible for analyzing and planning reading/writing of data using any of the Data Source APIs, including:
 - pruning the files from partitions that will be read based on filters.
 - appending partition columns*
 - applying additional filters when a data source can not evaluate them internally.
 - constructing an RDD that is bucketed correctly when required*
 - sanity checking schema match-up and other analysis when writing.

*In the future we should do that following:
 - Break out file handling into its own Strategy as its sufficiently complex / isolated.
 - Push the appending of partition columns down in to `FileFormat` to avoid an extra copy / unvectorization.
 - Use a custom RDD for scans instead of `SQLNewNewHadoopRDD2`

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

Closes #11509 from marmbrus/fileDataSource.
2016-03-07 15:15:10 -08:00
hyukjinkwon 8577260abd [SPARK-13442][SQL] Make type inference recognize boolean types
## What changes were proposed in this pull request?

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

This PR adds the support for inferring `BooleanType` for schema.
It supports to infer case-insensitive `true` / `false` as `BooleanType`.

Unittests were added for `CSVInferSchemaSuite` and `CSVSuite` for end-to-end test.

## How was the this patch tested?

This was tested with unittests and with `dev/run_tests` for coding style

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11315 from HyukjinKwon/SPARK-13442.
2016-03-07 14:32:01 -08:00
gatorsmile b6071a7001 [SPARK-13722][SQL] No Push Down for Non-deterministics Predicates through Generate
#### What changes were proposed in this pull request?

Non-deterministic predicates should not be pushed through Generate.

#### How was this patch tested?

Added a test case in `FilterPushdownSuite.scala`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11562 from gatorsmile/pushPredicateDownWindow.
2016-03-07 12:09:27 -08:00
Sameer Agarwal ef77003178 [SPARK-13495][SQL] Add Null Filters in the query plan for Filters/Joins based on their data constraints
## What changes were proposed in this pull request?

This PR adds an optimizer rule to eliminate reading (unnecessary) NULL values if they are not required for correctness by inserting `isNotNull` filters is the query plan. These filters are currently inserted beneath existing `Filter` and `Join` operators and are inferred based on their data constraints.

Note: While this optimization is applicable to all types of join, it primarily benefits `Inner` and `LeftSemi` joins.

## How was this patch tested?

1. Added a new `NullFilteringSuite` that tests for `IsNotNull` filters in the query plan for joins and filters. Also, tests interaction with the `CombineFilters` optimizer rules.
2. Test generated ExpressionTrees via `OrcFilterSuite`
3. Test filter source pushdown logic via `SimpleTextHadoopFsRelationSuite`

cc yhuai nongli

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11372 from sameeragarwal/gen-isnotnull.
2016-03-07 12:04:59 -08:00
Wenchen Fan 4896411176 [SPARK-13694][SQL] QueryPlan.expressions should always include all expressions
## What changes were proposed in this pull request?

It's weird that expressions don't always have all the expressions in it. This PR marks `QueryPlan.expressions` final to forbid sub classes overriding it to exclude some expressions. Currently only `Generate` override it, we can use `producedAttributes` to fix the unresolved attribute problem for it.

Note that this PR doesn't fix the problem in #11497

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11532 from cloud-fan/generate.
2016-03-07 10:32:34 -08:00
Dilip Biswal d7eac9d795 [SPARK-13651] Generator outputs are not resolved correctly resulting in run time error
## What changes were proposed in this pull request?

```
Seq(("id1", "value1")).toDF("key", "value").registerTempTable("src")
sqlContext.sql("SELECT t1.* FROM src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) t1 AS key, value")
```
Results in following logical plan

```
Project [key#2,value#3]
+- Generate explode(HiveGenericUDF#org.apache.hadoop.hive.ql.udf.generic.GenericUDFMap(key1,100,key2,200)), true, false, Some(genoutput), [key#2,value#3]
   +- SubqueryAlias src
      +- Project [_1#0 AS key#2,_2#1 AS value#3]
         +- LocalRelation [_1#0,_2#1], [[id1,value1]]
```

The above query fails with following runtime error.
```
java.lang.ClassCastException: java.lang.Integer cannot be cast to org.apache.spark.unsafe.types.UTF8String
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:46)
	at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getUTF8String(rows.scala:221)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(generated.java:42)
	at org.apache.spark.sql.execution.Generate$$anonfun$doExecute$1$$anonfun$apply$9.apply(Generate.scala:98)
	at org.apache.spark.sql.execution.Generate$$anonfun$doExecute$1$$anonfun$apply$9.apply(Generate.scala:96)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:370)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:370)
	at scala.collection.Iterator$class.foreach(Iterator.scala:742)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1194)
        <stack-trace omitted.....>
```
In this case the generated outputs are wrongly resolved from its child (LocalRelation) due to
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L537-L548
## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

Added unit tests in hive/SQLQuerySuite and AnalysisSuite

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

Closes #11497 from dilipbiswal/spark-13651.
2016-03-07 09:46:28 -08:00
Andrew Or bc7a3ec290 [SPARK-13685][SQL] Rename catalog.Catalog to ExternalCatalog
## What changes were proposed in this pull request?

Today we have `analysis.Catalog` and `catalog.Catalog`. In the future the former will call the latter. When that happens, if both of them are still called `Catalog` it will be very confusing. This patch renames the latter `ExternalCatalog` because it is expected to talk to external systems.

## How was this patch tested?

Jenkins.

Author: Andrew Or <andrew@databricks.com>

Closes #11526 from andrewor14/rename-catalog.
2016-03-07 00:14:40 -08:00
Cheng Lian 8ff88094da Revert "[SPARK-13616][SQL] Let SQLBuilder convert logical plan without a project on top of it"
This reverts commit f87ce0504e.

According to discussion in #11466, let's revert PR #11466 for safe.

Author: Cheng Lian <lian@databricks.com>

Closes #11539 from liancheng/revert-pr-11466.
2016-03-06 12:54:04 +08:00
gatorsmile adce5ee721 [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 <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #11283 from gatorsmile/groupingSetsToSQL.
2016-03-05 19:25:03 +08:00
Nong Li a6e2bd31f5 [SPARK-13255] [SQL] Update vectorized reader to directly return ColumnarBatch instead of InternalRows.
## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

Currently, the parquet reader returns rows one by one which is bad for performance. This patch
updates the reader to directly return ColumnarBatches. This is only enabled with whole stage
codegen, which is the only operator currently that is able to consume ColumnarBatches (instead
of rows). The current implementation is a bit of a hack to get this to work and we should do
more refactoring of these low level interfaces to make this work better.

## How was this patch tested?

```
Results:
TPCDS:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
---------------------------------------------------------------------------------
q55 (before)                             8897 / 9265         12.9          77.2
q55                                      5486 / 5753         21.0          47.6
```

Author: Nong Li <nong@databricks.com>

Closes #11435 from nongli/spark-13255.
2016-03-04 15:15:48 -08:00
Andrew Or b7d4147421 [SPARK-13633][SQL] Move things into catalyst.parser package
## What changes were proposed in this pull request?

This patch simply moves things to existing package `o.a.s.sql.catalyst.parser` in an effort to reduce the size of the diff in #11048. This is conceptually the same as a recently merged patch #11482.

## How was this patch tested?

Jenkins.

Author: Andrew Or <andrew@databricks.com>

Closes #11506 from andrewor14/parser-package.
2016-03-04 10:32:00 -08:00
Rajesh Balamohan 204b02b56a [SPARK-12925] Improve HiveInspectors.unwrap for StringObjectInspector.…
Earlier fix did not copy the bytes and it is possible for higher level to reuse Text object. This was causing issues. Proposed fix now copies the bytes from Text. This still avoids the expensive encoding/decoding

Author: Rajesh Balamohan <rbalamohan@apache.org>

Closes #11477 from rajeshbalamohan/SPARK-12925.2.
2016-03-04 10:59:40 +00:00
Davies Liu dd83c209f1 [SPARK-13603][SQL] support SQL generation for subquery
## What changes were proposed in this pull request?

This is support SQL generation for subquery expressions, which will be replaced to a SubqueryHolder inside SQLBuilder recursively.

## How was this patch tested?

Added unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11453 from davies/sql_subquery.
2016-03-04 16:18:15 +08:00
thomastechs f6ac7c30d4 [SPARK-12941][SQL][MASTER] Spark-SQL JDBC Oracle dialect fails to map string datatypes to Oracle VARCHAR datatype mapping
## What changes were proposed in this pull request?
A test suite added for the bug fix -SPARK 12941; for the mapping of the StringType to corresponding in Oracle

## How was this patch tested?
manual tests done
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: thomastechs <thomas.sebastian@tcs.com>
Author: THOMAS SEBASTIAN <thomas.sebastian@tcs.com>

Closes #11489 from thomastechs/thomastechs-12941-master-new.
2016-03-03 20:35:40 -08:00
Davies Liu d062587dd2 [SPARK-13601] [TESTS] use 1 partition in tests to avoid race conditions
## What changes were proposed in this pull request?

Fix race conditions when cleanup files.

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #11507 from davies/flaky.
2016-03-03 17:46:28 -08:00
Davies Liu b373a88862 [SPARK-13415][SQL] Visualize subquery in SQL web UI
## What changes were proposed in this pull request?

This PR support visualization for subquery in SQL web UI, also improve the explain of subquery, especially when it's used together with whole stage codegen.

For example:
```python
>>> sqlContext.range(100).registerTempTable("range")
>>> sqlContext.sql("select id / (select sum(id) from range) from range where id > (select id from range limit 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias(('id / subquery#9), None)]
:  +- 'SubqueryAlias subquery#9
:     +- 'Project [unresolvedalias('sum('id), None)]
:        +- 'UnresolvedRelation `range`, None
+- 'Filter ('id > subquery#8)
   :  +- 'SubqueryAlias subquery#8
   :     +- 'GlobalLimit 1
   :        +- 'LocalLimit 1
   :           +- 'Project [unresolvedalias('id, None)]
   :              +- 'UnresolvedRelation `range`, None
   +- 'UnresolvedRelation `range`, None

== Analyzed Logical Plan ==
(id / scalarsubquery()): double
Project [(cast(id#0L as double) / cast(subquery#9 as double)) AS (id / scalarsubquery())#11]
:  +- SubqueryAlias subquery#9
:     +- Aggregate [(sum(id#0L),mode=Complete,isDistinct=false) AS sum(id)#10L]
:        +- SubqueryAlias range
:           +- Range 0, 100, 1, 4, [id#0L]
+- Filter (id#0L > subquery#8)
   :  +- SubqueryAlias subquery#8
   :     +- GlobalLimit 1
   :        +- LocalLimit 1
   :           +- Project [id#0L]
   :              +- SubqueryAlias range
   :                 +- Range 0, 100, 1, 4, [id#0L]
   +- SubqueryAlias range
      +- Range 0, 100, 1, 4, [id#0L]

== Optimized Logical Plan ==
Project [(cast(id#0L as double) / cast(subquery#9 as double)) AS (id / scalarsubquery())#11]
:  +- SubqueryAlias subquery#9
:     +- Aggregate [(sum(id#0L),mode=Complete,isDistinct=false) AS sum(id)#10L]
:        +- Range 0, 100, 1, 4, [id#0L]
+- Filter (id#0L > subquery#8)
   :  +- SubqueryAlias subquery#8
   :     +- GlobalLimit 1
   :        +- LocalLimit 1
   :           +- Project [id#0L]
   :              +- Range 0, 100, 1, 4, [id#0L]
   +- Range 0, 100, 1, 4, [id#0L]

== Physical Plan ==
WholeStageCodegen
:  +- Project [(cast(id#0L as double) / cast(subquery#9 as double)) AS (id / scalarsubquery())#11]
:     :  +- Subquery subquery#9
:     :     +- WholeStageCodegen
:     :        :  +- TungstenAggregate(key=[], functions=[(sum(id#0L),mode=Final,isDistinct=false)], output=[sum(id)#10L])
:     :        :     +- INPUT
:     :        +- Exchange SinglePartition, None
:     :           +- WholeStageCodegen
:     :              :  +- TungstenAggregate(key=[], functions=[(sum(id#0L),mode=Partial,isDistinct=false)], output=[sum#14L])
:     :              :     +- Range 0, 1, 4, 100, [id#0L]
:     +- Filter (id#0L > subquery#8)
:        :  +- Subquery subquery#8
:        :     +- CollectLimit 1
:        :        +- WholeStageCodegen
:        :           :  +- Project [id#0L]
:        :           :     +- Range 0, 1, 4, 100, [id#0L]
:        +- Range 0, 1, 4, 100, [id#0L]
```

The web UI looks like:

![subquery](https://cloud.githubusercontent.com/assets/40902/13377963/932bcbae-dda7-11e5-82f7-03c9be85d77c.png)

This PR also change the tree structure of WholeStageCodegen to make it consistent than others. Before this change, Both WholeStageCodegen and InputAdapter hold a references to the same plans, those could be updated without notify another, causing problems, this is discovered by #11403 .

## How was this patch tested?

Existing tests, also manual tests with the example query, check the explain and web UI.

Author: Davies Liu <davies@databricks.com>

Closes #11417 from davies/viz_subquery.
2016-03-03 17:36:48 -08:00
Shixiong Zhu ad0de99f3d [SPARK-13584][SQL][TESTS] Make ContinuousQueryManagerSuite not output logs to the console
## What changes were proposed in this pull request?

Make ContinuousQueryManagerSuite not output logs to the console. The logs will still output to `unit-tests.log`.

I also updated `SQLListenerMemoryLeakSuite` to use `quietly` to avoid changing the log level which won't output logs to `unit-tests.log`.

## How was this patch tested?

Just check Jenkins output.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11439 from zsxwing/quietly-ContinuousQueryManagerSuite.
2016-03-03 15:41:56 -08:00
Andrew Or 3edcc40223 [SPARK-13632][SQL] Move commands.scala to command package
## What changes were proposed in this pull request?

This patch simply moves things to a new package in an effort to reduce the size of the diff in #11048. Currently the new package only has one file, but in the future we'll add many new commands in SPARK-13139.

## How was this patch tested?

Jenkins.

Author: Andrew Or <andrew@databricks.com>

Closes #11482 from andrewor14/commands-package.
2016-03-03 15:24:38 -08:00
Dongjoon Hyun 941b270b70 [MINOR] Fix typos in comments and testcase name of code
## What changes were proposed in this pull request?

This PR fixes typos in comments and testcase name of code.

## How was this patch tested?

manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11481 from dongjoon-hyun/minor_fix_typos_in_code.
2016-03-03 22:42:12 +00:00
hyukjinkwon cf95d728c6 [SPARK-13543][SQL] Support for specifying compression codec for Parquet/ORC via option()
## What changes were proposed in this pull request?

This PR adds the support to specify compression codecs for both ORC and Parquet.

## How was this patch tested?

unittests within IDE and code style tests with `dev/run_tests`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11464 from HyukjinKwon/SPARK-13543.
2016-03-03 10:30:55 -08:00
Sean Owen 645c3a85e2 [SPARK-13423][HOTFIX] Static analysis fixes for 2.x / fixed for Scala 2.10
## What changes were proposed in this pull request?

Fixes compile problem due to inadvertent use of `Option.contains`, only in Scala 2.11. The change should have been to replace `Option.exists(_ == x)` with `== Some(x)`. Replacing exists with contains only makes sense for collections. Replacing use of `Option.exists` still makes sense though as it's misleading.

## How was this patch tested?

Jenkins tests / compilation

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: Sean Owen <sowen@cloudera.com>

Closes #11493 from srowen/SPARK-13423.2.
2016-03-03 15:11:02 +00:00
Dongjoon Hyun b5f02d6743 [SPARK-13583][CORE][STREAMING] Remove unused imports and add checkstyle rule
## What changes were proposed in this pull request?

After SPARK-6990, `dev/lint-java` keeps Java code healthy and helps PR review by saving much time.
This issue aims remove unused imports from Java/Scala code and add `UnusedImports` checkstyle rule to help developers.

## How was this patch tested?
```
./dev/lint-java
./build/sbt compile
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11438 from dongjoon-hyun/SPARK-13583.
2016-03-03 10:12:32 +00:00
Sean Owen e97fc7f176 [SPARK-13423][WIP][CORE][SQL][STREAMING] Static analysis fixes for 2.x
## What changes were proposed in this pull request?

Make some cross-cutting code improvements according to static analysis. These are individually up for discussion since they exist in separate commits that can be reverted. The changes are broadly:

- Inner class should be static
- Mismatched hashCode/equals
- Overflow in compareTo
- Unchecked warnings
- Misuse of assert, vs junit.assert
- get(a) + getOrElse(b) -> getOrElse(a,b)
- Array/String .size -> .length (occasionally, -> .isEmpty / .nonEmpty) to avoid implicit conversions
- Dead code
- tailrec
- exists(_ == ) -> contains find + nonEmpty -> exists filter + size -> count
- reduce(_+_) -> sum map + flatten -> map

The most controversial may be .size -> .length simply because of its size. It is intended to avoid implicits that might be expensive in some places.

## How was the this patch tested?

Existing Jenkins unit tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #11292 from srowen/SPARK-13423.
2016-03-03 09:54:09 +00:00
Dongjoon Hyun 02b7677e95 [HOT-FIX] Recover some deprecations for 2.10 compatibility.
## What changes were proposed in this pull request?

#11479 [SPARK-13627] broke 2.10 compatibility: [2.10-Build](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-scala-2.10/292/console)
At this moment, we need to support both 2.10 and 2.11.
This PR recovers some deprecated methods which were replace by [SPARK-13627].

## How was this patch tested?

Jenkins build: Both 2.10, 2.11.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11488 from dongjoon-hyun/hotfix_compatibility_with_2.10.
2016-03-03 09:53:02 +00:00
Liang-Chi Hsieh 7b25dc7b7e [SPARK-13466] [SQL] Remove projects that become redundant after column pruning rule
JIRA: https://issues.apache.org/jira/browse/SPARK-13466

## What changes were proposed in this pull request?

With column pruning rule in optimizer, some Project operators will become redundant. We should remove these redundant Projects.

For an example query:

    val input = LocalRelation('key.int, 'value.string)

    val query =
      Project(Seq($"x.key", $"y.key"),
        Join(
          SubqueryAlias("x", input),
          BroadcastHint(SubqueryAlias("y", input)), Inner, None))

After the first run of column pruning, it would like:

    Project(Seq($"x.key", $"y.key"),
      Join(
        Project(Seq($"x.key"), SubqueryAlias("x", input)),
        Project(Seq($"y.key"),      <-- inserted by the rule
        BroadcastHint(SubqueryAlias("y", input))),
        Inner, None))

Actually we don't need the outside Project now. This patch will remove it:

    Join(
      Project(Seq($"x.key"), SubqueryAlias("x", input)),
      Project(Seq($"y.key"),
      BroadcastHint(SubqueryAlias("y", input))),
      Inner, None)

## How was the this patch tested?

Unit test is added into ColumnPruningSuite.

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

Closes #11341 from viirya/remove-redundant-project.
2016-03-03 00:06:46 -08:00
Liang-Chi Hsieh 1085bd862a [SPARK-13635] [SQL] Enable LimitPushdown optimizer rule because we have whole-stage codegen for Limit
JIRA: https://issues.apache.org/jira/browse/SPARK-13635

## What changes were proposed in this pull request?

LimitPushdown optimizer rule has been disabled due to no whole-stage codegen for Limit. As we have whole-stage codegen for Limit now, we should enable it.

## How was this patch tested?

As we only re-enable LimitPushdown optimizer rule, no need to add new tests for it.

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

Closes #11483 from viirya/enable-limitpushdown.
2016-03-02 23:46:23 -08:00
Liang-Chi Hsieh f87ce0504e [SPARK-13616][SQL] Let SQLBuilder convert logical plan without a project on top of it
JIRA: https://issues.apache.org/jira/browse/SPARK-13616

## What changes were proposed in this pull request?

It is possibly that a logical plan has been removed `Project` from the top of it. Or the plan doesn't has a top `Project` from the beginning because it is not necessary. Currently the `SQLBuilder` can't convert such plans back to SQL. This change is to add this feature.

## How was this patch tested?

A test is added to `LogicalPlanToSQLSuite`.

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

Closes #11466 from viirya/sqlbuilder-notopselect.
2016-03-02 22:21:49 -08:00
Dongjoon Hyun 9c274ac4a6 [SPARK-13627][SQL][YARN] Fix simple deprecation warnings.
## What changes were proposed in this pull request?

This PR aims to fix the following deprecation warnings.
  * MethodSymbolApi.paramss--> paramLists
  * AnnotationApi.tpe -> tree.tpe
  * BufferLike.readOnly -> toList.
  * StandardNames.nme -> termNames
  * scala.tools.nsc.interpreter.AbstractFileClassLoader -> scala.reflect.internal.util.AbstractFileClassLoader
  * TypeApi.declarations-> decls

## How was this patch tested?

Check the compile build log and pass the tests.
```
./build/sbt
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11479 from dongjoon-hyun/SPARK-13627.
2016-03-02 20:34:22 -08:00
Wenchen Fan b60b813799 [SPARK-13617][SQL] remove unnecessary GroupingAnalytics trait
## What changes were proposed in this pull request?

The `trait GroupingAnalytics` only has one implementation, it's an unnecessary abstraction. This PR removes it, and does some code simplification when resolving `GroupingSet`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11469 from cloud-fan/groupingset.
2016-03-02 20:18:57 -08:00
Takeshi YAMAMURO 6250cf1e00 [SPARK-13528][SQL] Make the short names of compression codecs consistent in ParquetRelation
## What changes were proposed in this pull request?
This pr to make the short names of compression codecs in `ParquetRelation` consistent against other ones. This pr comes from #11324.

## How was this patch tested?
Add more tests in `TextSuite`.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #11408 from maropu/SPARK-13528.
2016-03-02 15:30:41 -08:00
Nong Li e2780ce825 [SPARK-13574] [SQL] Add benchmark to measure string dictionary decode.
## What changes were proposed in this pull request?

Also updated the other benchmarks when the default to use vectorized decode was flipped.

Author: Nong Li <nong@databricks.com>

Closes #11454 from nongli/benchmark.
2016-03-02 15:03:19 -08:00
Davies Liu b5a59a0fe2 [SPARK-13601] call failure callbacks before writer.close()
## What changes were proposed in this pull request?

In order to tell OutputStream that the task has failed or not, we should call the failure callbacks BEFORE calling writer.close().

## How was this patch tested?

Added new unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11450 from davies/callback.
2016-03-02 14:35:44 -08:00
gatorsmile 9e01fe2ed1 [SPARK-13535][SQL] Fix Analysis Exceptions when Using Backticks in Transform Clause
#### What changes were proposed in this pull request?
```SQL
FROM
(FROM test SELECT TRANSFORM(key, value) USING 'cat' AS (`thing1` int, thing2 string)) t
SELECT thing1 + 1
```
This query returns an analysis error, like:
```
Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve '`thing1`' given input columns: [`thing1`, thing2]; line 3 pos 7
'Project [unresolvedalias(('thing1 + 1), None)]
+- SubqueryAlias t
   +- ScriptTransformation [key#2,value#3], cat, [`thing1`#6,thing2#7], HiveScriptIOSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim,	)),List((field.delim,	)),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
      +- SubqueryAlias test
         +- Project [_1#0 AS key#2,_2#1 AS value#3]
            +- LocalRelation [_1#0,_2#1], [[1,1],[2,2],[3,3],[4,4],[5,5]]
```

The backpacks of \`thing1\` should be cleaned before entering Parser/Analyzer. This PR fixes this issue.

#### How was this patch tested?

Added a test case and modified an existing test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11415 from gatorsmile/scriptTransform.
2016-03-02 23:07:48 +01:00
gatorsmile 8f8d8a2315 [SPARK-13609] [SQL] Support Column Pruning for MapPartitions
#### What changes were proposed in this pull request?

This PR is to prune unnecessary columns when the operator is  `MapPartitions`. The solution is to add an extra `Project` in the child node.

For the other two operators `AppendColumns` and `MapGroups`, it sounds doable. More discussions are required. The major reason is the current implementation of the `inputPlan` of `groupBy` is based on the child of `AppendColumns`. It might be a bug? Thus, will submit a separate PR.

#### How was this patch tested?

Added a test case in ColumnPruningSuite to verify the rule. Added another test case in DatasetSuite.scala to verify the data.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11460 from gatorsmile/datasetPruningNew.
2016-03-02 09:59:22 -08:00
lgieron d8afd45f89 [SPARK-13515] Make FormatNumber work irrespective of locale.
## What changes were proposed in this pull request?

Change in class FormatNumber to make it work irrespective of locale.

## How was this patch tested?

Unit tests.

Author: lgieron <lgieron@gmail.com>

Closes #11396 from lgieron/SPARK-13515_Fix_Format_Number.
2016-03-02 15:57:27 +00:00
sureshthalamati e42724b12b [SPARK-13167][SQL] Include rows with null values for partition column when reading from JDBC datasources.
Rows with null values in partition column are not included in the results because none of the partition
where clause specify is null predicate on the partition column. This fix adds is null predicate on the partition column  to the first JDBC partition where clause.

Example:
JDBCPartition(THEID < 1 or THEID is null, 0),JDBCPartition(THEID >= 1 AND THEID < 2,1),
JDBCPartition(THEID >= 2, 2)

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #11063 from sureshthalamati/nullable_jdbc_part_col_spark-13167.
2016-03-01 17:34:21 -08:00
Davies Liu a640c5b4fb [SPARK-13598] [SQL] remove LeftSemiJoinBNL
## What changes were proposed in this pull request?

Broadcast left semi join without joining keys is already supported in BroadcastNestedLoopJoin, it has the same implementation as LeftSemiJoinBNL, we should remove that.

## How was this patch tested?

Updated unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11448 from davies/remove_bnl.
2016-03-01 17:27:57 -08:00
Davies Liu c27ba0d547 [SPARK-13582] [SQL] defer dictionary decoding in parquet reader
## What changes were proposed in this pull request?

This PR defer the resolution from a id of dictionary to value until the column is actually accessed (inside getInt/getLong), this is very useful for those columns and rows that are filtered out. It's also useful for binary type, we will not need to copy all the byte arrays.

This PR also change the underlying type for small decimal that could be fit within a Int, in order to use getInt() to lookup the value from IntDictionary.

## How was this patch tested?

Manually test TPCDS Q7 with scale factor 10, saw about 30% improvements (after PR #11274).

Author: Davies Liu <davies@databricks.com>

Closes #11437 from davies/decode_dict.
2016-03-01 13:07:04 -08:00
Liang-Chi Hsieh c43899a04e [SPARK-13511] [SQL] Add wholestage codegen for limit
JIRA: https://issues.apache.org/jira/browse/SPARK-13511

## What changes were proposed in this pull request?

Current limit operator doesn't support wholestage codegen. This is open to add support for it.

In the `doConsume` of `GlobalLimit` and `LocalLimit`, we use a count term to count the processed rows. Once the row numbers catches the limit number, we set the variable `stopEarly` of `BufferedRowIterator` newly added in this pr to `true` that indicates we want to stop processing remaining rows. Then when the wholestage codegen framework checks `shouldStop()`, it will stop the processing of the row iterator.

Before this, the executed plan for a query `sqlContext.range(N).limit(100).groupBy().sum()` is:

    TungstenAggregate(key=[], functions=[(sum(id#5L),mode=Final,isDistinct=false)], output=[sum(id)#6L])
    +- TungstenAggregate(key=[], functions=[(sum(id#5L),mode=Partial,isDistinct=false)], output=[sum#9L])
       +- GlobalLimit 100
          +- Exchange SinglePartition, None
             +- LocalLimit 100
                +- Range 0, 1, 1, 524288000, [id#5L]

After add wholestage codegen support:

    WholeStageCodegen
    :  +- TungstenAggregate(key=[], functions=[(sum(id#40L),mode=Final,isDistinct=false)], output=[sum(id)#41L])
    :     +- TungstenAggregate(key=[], functions=[(sum(id#40L),mode=Partial,isDistinct=false)], output=[sum#44L])
    :        +- GlobalLimit 100
    :           +- INPUT
    +- Exchange SinglePartition, None
       +- WholeStageCodegen
          :  +- LocalLimit 100
          :     +- Range 0, 1, 1, 524288000, [id#40L]

## How was this patch tested?

A test is added into BenchmarkWholeStageCodegen.

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

Closes #11391 from viirya/wholestage-limit.
2016-03-01 08:43:02 -08:00
Sameer Agarwal 4bd697da03 [SPARK-13123][SQL] Implement whole state codegen for sort
## What changes were proposed in this pull request?
This PR adds support for implementing whole state codegen for sort. Builds heaving on nongli 's PR: https://github.com/apache/spark/pull/11008 (which actually implements the feature), and adds the following changes on top:

- [x]  Generated code updates peak execution memory metrics
- [x]  Unit tests in `WholeStageCodegenSuite` and `SQLMetricsSuite`

## How was this patch tested?

New unit tests in `WholeStageCodegenSuite` and `SQLMetricsSuite`. Further, all existing sort tests should pass.

Author: Sameer Agarwal <sameer@databricks.com>
Author: Nong Li <nong@databricks.com>

Closes #11359 from sameeragarwal/sort-codegen.
2016-02-29 12:59:46 -08:00
gatorsmile bc65f60ef7 [SPARK-13544][SQL] Rewrite/Propagate Constraints for Aliases in Aggregate
#### What changes were proposed in this pull request?

After analysis by Analyzer, two operators could have alias. They are `Project` and `Aggregate`. So far, we only rewrite and propagate constraints if `Alias` is defined in `Project`. This PR is to resolve this issue in `Aggregate`.

#### How was this patch tested?

Added a test case for `Aggregate` in `ConstraintPropagationSuite`.

marmbrus sameeragarwal

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11422 from gatorsmile/validConstraintsInUnaryNodes.
2016-02-29 10:10:04 -08:00
hyukjinkwon 02aa499dfb [SPARK-13509][SPARK-13507][SQL] Support for writing CSV with a single function call
https://issues.apache.org/jira/browse/SPARK-13507
https://issues.apache.org/jira/browse/SPARK-13509

## What changes were proposed in this pull request?
This PR adds the support to write CSV data directly by a single call to the given path.

Several unitests were added for each functionality.
## How was this patch tested?

This was tested with unittests and with `dev/run_tests` for coding style

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

Closes #11389 from HyukjinKwon/SPARK-13507-13509.
2016-02-29 09:44:29 -08:00
Cheng Lian 916fc34f98 [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 <lian@databricks.com>

Closes #11421 from liancheng/spark-13540-object-as-outer-scope.
2016-03-01 01:07:45 +08:00
Rahul Tanwani dd3b5455c6 [SPARK-13309][SQL] Fix type inference issue with CSV data
Fix type inference issue for sparse CSV data - https://issues.apache.org/jira/browse/SPARK-13309

Author: Rahul Tanwani <rahul@Rahuls-MacBook-Pro.local>

Closes #11194 from tanwanirahul/master.
2016-02-28 23:16:34 -08:00
Liang-Chi Hsieh 6dfc4a764c [SPARK-13537][SQL] Fix readBytes in VectorizedPlainValuesReader
JIRA: https://issues.apache.org/jira/browse/SPARK-13537

## What changes were proposed in this pull request?

In readBytes of VectorizedPlainValuesReader, we use buffer[offset] to access bytes in buffer. It is incorrect because offset is added with Platform.BYTE_ARRAY_OFFSET when initialization. We should fix it.

## How was this patch tested?

`ParquetHadoopFsRelationSuite` sometimes (depending on the randomly generated data) will be [failed](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52136/consoleFull) by this bug. After applying this, the test can be passed.

I added a test to `ParquetHadoopFsRelationSuite` with the data which will fail without this patch.

The error exception:

    [info] ParquetHadoopFsRelationSuite:
    [info] - test all data types - StringType (440 milliseconds)
    [info] - test all data types - BinaryType (434 milliseconds)
    [info] - test all data types - BooleanType (406 milliseconds)
    20:59:38.618 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 2597.0 (TID 67966)
    java.lang.ArrayIndexOutOfBoundsException: 46
	at org.apache.spark.sql.execution.datasources.parquet.VectorizedPlainValuesReader.readBytes(VectorizedPlainValuesReader.java:88)

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

Closes #11418 from viirya/fix-readbytes.
2016-02-28 21:16:06 -08:00
Andrew Or cca79fad66 [SPARK-13526][SQL] Move SQLContext per-session states to new class
## What changes were proposed in this pull request?

This creates a `SessionState`, which groups a few fields that existed in `SQLContext`. Because `HiveContext` extends `SQLContext` we also need to make changes there. This is mainly a cleanup task that will soon pave the way for merging the two contexts.

## How was this patch tested?

Existing unit tests; this patch introduces no change in behavior.

Author: Andrew Or <andrew@databricks.com>

Closes #11405 from andrewor14/refactor-session.
2016-02-27 19:51:28 -08:00
Nong Li d780ed8b5c [SPARK-13533][SQL] Fix readBytes in VectorizedPlainValuesReader
## What changes were proposed in this pull request?

Fix readBytes in VectorizedPlainValuesReader. This fixes a copy and paste issue.

## How was this patch tested?

Ran ParquetHadoopFsRelationSuite which failed before this.

Author: Nong Li <nong@databricks.com>

Closes #11414 from nongli/spark-13533.
2016-02-27 19:45:57 -08:00
Liang-Chi Hsieh 3814d0bcf6 [SPARK-13530][SQL] Add ShortType support to UnsafeRowParquetRecordReader
JIRA: https://issues.apache.org/jira/browse/SPARK-13530

## What changes were proposed in this pull request?

By enabling vectorized parquet scanner by default, the unit test `ParquetHadoopFsRelationSuite` based on `HadoopFsRelationTest` will be failed due to the lack of short type support in `UnsafeRowParquetRecordReader`. We should fix it.

The error exception:

    [info] ParquetHadoopFsRelationSuite:
    [info] - test all data types - StringType (499 milliseconds)
    [info] - test all data types - BinaryType (447 milliseconds)
    [info] - test all data types - BooleanType (520 milliseconds)
    [info] - test all data types - ByteType (418 milliseconds)
    00:22:58.920 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 124.0 (TID 1949)
    org.apache.commons.lang.NotImplementedException: Unimplemented type: ShortType
	at org.apache.spark.sql.execution.datasources.parquet.UnsafeRowParquetRecordReader$ColumnReader.readIntBatch(UnsafeRowParquetRecordReader.java:769)
	at org.apache.spark.sql.execution.datasources.parquet.UnsafeRowParquetRecordReader$ColumnReader.readBatch(UnsafeRowParquetRecordReader.java:640)
	at org.apache.spark.sql.execution.datasources.parquet.UnsafeRowParquetRecordReader$ColumnReader.access$000(UnsafeRowParquetRecordReader.java:461)
	at org.apache.spark.sql.execution.datasources.parquet.UnsafeRowParquetRecordReader.nextBatch(UnsafeRowParquetRecordReader.java:224)
## How was this patch tested?

The unit test `ParquetHadoopFsRelationSuite` based on `HadoopFsRelationTest` will be [failed](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52110/consoleFull) due to the lack of short type support in UnsafeRowParquetRecordReader. By adding this support, the test can be passed.

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

Closes #11412 from viirya/add-shorttype-support.
2016-02-27 11:41:35 -08:00
Nong Li 7a0cb4e587 [SPARK-13518][SQL] Enable vectorized parquet scanner by default
## What changes were proposed in this pull request?

Change the default of the flag to enable this feature now that the implementation is complete.

## How was this patch tested?

The new parquet reader should be a drop in, so will be exercised by the existing tests.

Author: Nong Li <nong@databricks.com>

Closes #11397 from nongli/spark-13518.
2016-02-26 22:36:32 -08:00
Nong Li 0598a2b81d [SPARK-13499] [SQL] Performance improvements for parquet reader.
## What changes were proposed in this pull request?

This patch includes these performance fixes:
  - Remove unnecessary setNotNull() calls. The NULL bits are cleared already.
  - Speed up RLE group decoding
  - Speed up dictionary decoding by decoding NULLs directly into the result.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

In addition to the updated benchmarks, on TPCDS, the result of these changes
running Q55 (sf40) is:

```
TPCDS:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
---------------------------------------------------------------------------------
q55 (Before)                             6398 / 6616         18.0          55.5
q55 (After)                              4983 / 5189         23.1          43.3
```

Author: Nong Li <nong@databricks.com>

Closes #11375 from nongli/spark-13499.
2016-02-26 12:43:50 -08:00
Davies Liu 6df1e55a65 [SPARK-12313] [SQL] improve performance of BroadcastNestedLoopJoin
## What changes were proposed in this pull request?

Currently, BroadcastNestedLoopJoin is implemented for worst case, it's too slow, very easy to hang forever. This PR will create fast path for some joinType and buildSide, also improve the worst case (will use much less memory than before).

Before this PR, one task requires O(N*K) + O(K) in worst cases, N is number of rows from one partition of streamed table, it could hang the job (because of GC).

In order to workaround this for InnerJoin, we have to disable auto-broadcast, switch to CartesianProduct: This could be workaround for InnerJoin, see https://forums.databricks.com/questions/6747/how-do-i-get-a-cartesian-product-of-a-huge-dataset.html

In this PR, we will have fast path for these joins :

 InnerJoin with BuildLeft or BuildRight
 LeftOuterJoin with BuildRight
 RightOuterJoin with BuildLeft
 LeftSemi with BuildRight

These fast paths are all stream based (take one pass on streamed table), required O(1) memory.

All other join types and build types will take two pass on streamed table, one pass to find the matched rows that includes streamed part, which require O(1) memory, another pass to find the rows from build table that does not have a matched row from streamed table, which required O(K) memory, K is the number rows from build side, one bit per row, should be much smaller than the memory for broadcast. The following join types work in this way:

LeftOuterJoin with BuildLeft
RightOuterJoin with BuildRight
FullOuterJoin with BuildLeft or BuildRight
LeftSemi with BuildLeft

This PR also added tests for all the join types for BroadcastNestedLoopJoin.

After this PR, for InnerJoin with one small table, BroadcastNestedLoopJoin should be faster than CartesianProduct, we don't need that workaround anymore.

## How was the this patch tested?

Added unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11328 from davies/nested_loop.
2016-02-26 09:58:05 -08:00
Cheng Lian 99dfcedbfd [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 <lian@databricks.com>

Closes #11388 from liancheng/remove-df-rdd-ops.
2016-02-27 00:28:30 +08:00
hyukjinkwon 9812a24aa8 [SPARK-13503][SQL] Support to specify the (writing) option for compression codec for TEXT
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-13503
This PR makes the TEXT datasource can compress output by option instead of manually setting Hadoop configurations.
For reflecting codec by names, it is similar with https://github.com/apache/spark/pull/10805 and https://github.com/apache/spark/pull/10858.

## How was this patch tested?

This was tested with unittests and with `dev/run_tests` for coding style

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11384 from HyukjinKwon/SPARK-13503.
2016-02-25 23:57:29 -08:00
Reynold Xin 26ac60806c [SPARK-13487][SQL] User-facing RuntimeConfig interface
## What changes were proposed in this pull request?
This patch creates the public API for runtime configuration and an implementation for it. The public runtime configuration includes configs for existing SQL, as well as Hadoop Configuration.

This new interface is currently dead code. It will be added to SQLContext and a session entry point to Spark when we add that.

## How was this patch tested?
a new unit test suite

Author: Reynold Xin <rxin@databricks.com>

Closes #11378 from rxin/SPARK-13487.
2016-02-25 23:10:40 -08:00
thomastechs 8afe49141d [SPARK-12941][SQL][MASTER] Spark-SQL JDBC Oracle dialect fails to map string datatypes to Oracle VARCHAR datatype
## What changes were proposed in this pull request?

This Pull request is used for the fix SPARK-12941, creating a data type mapping to Oracle for the corresponding data type"Stringtype" from dataframe. This PR is for the master branch fix, where as another PR is already tested with the branch 1.4

## How was the this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
This patch was tested using the Oracle docker .Created a new integration suite for the same.The oracle.jdbc jar was to be downloaded from the maven repository.Since there was no jdbc jar available in the maven repository, the jar was downloaded from oracle site manually and installed in the local; thus tested. So, for SparkQA test case run, the ojdbc jar might be manually placed in the local maven repository(com/oracle/ojdbc6/11.2.0.2.0) while Spark QA test run.

Author: thomastechs <thomas.sebastian@tcs.com>

Closes #11306 from thomastechs/master.
2016-02-25 22:52:25 -08:00
Takeshi YAMAMURO 1b39fafa75 [SPARK-13361][SQL] Add benchmark codes for Encoder#compress() in CompressionSchemeBenchmark
This pr added benchmark codes for Encoder#compress().
Also, it replaced the benchmark results with new ones because the output format of `Benchmark` changed.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #11236 from maropu/CompressionSpike.
2016-02-25 20:17:48 -08:00
Josh Rosen 633d63a48a [SPARK-12757] Add block-level read/write locks to BlockManager
## Motivation

As a pre-requisite to off-heap caching of blocks, we need a mechanism to prevent pages / blocks from being evicted while they are being read. With on-heap objects, evicting a block while it is being read merely leads to memory-accounting problems (because we assume that an evicted block is a candidate for garbage-collection, which will not be true during a read), but with off-heap memory this will lead to either data corruption or segmentation faults.

## Changes

### BlockInfoManager and reader/writer locks

This patch adds block-level read/write locks to the BlockManager. It introduces a new `BlockInfoManager` component, which is contained within the `BlockManager`, holds the `BlockInfo` objects that the `BlockManager` uses for tracking block metadata, and exposes APIs for locking blocks in either shared read or exclusive write modes.

`BlockManager`'s `get*()` and `put*()` methods now implicitly acquire the necessary locks. After a `get()` call successfully retrieves a block, that block is locked in a shared read mode. A `put()` call will block until it acquires an exclusive write lock. If the write succeeds, the write lock will be downgraded to a shared read lock before returning to the caller. This `put()` locking behavior allows us store a block and then immediately turn around and read it without having to worry about it having been evicted between the write and the read, which will allow us to significantly simplify `CacheManager` in the future (see #10748).

See `BlockInfoManagerSuite`'s test cases for a more detailed specification of the locking semantics.

### Auto-release of locks at the end of tasks

Our locking APIs support explicit release of locks (by calling `unlock()`), but it's not always possible to guarantee that locks will be released prior to the end of the task. One reason for this is our iterator interface: since our iterators don't support an explicit `close()` operator to signal that no more records will be consumed, operations like `take()` or `limit()` don't have a good means to release locks on their input iterators' blocks. Another example is broadcast variables, whose block locks can only be released at the end of the task.

To address this, `BlockInfoManager` uses a pair of maps to track the set of locks acquired by each task. Lock acquisitions automatically record the current task attempt id by obtaining it from `TaskContext`. When a task finishes, code in `Executor` calls `BlockInfoManager.unlockAllLocksForTask(taskAttemptId)` to free locks.

### Locking and the MemoryStore

In order to prevent in-memory blocks from being evicted while they are being read, the `MemoryStore`'s `evictBlocksToFreeSpace()` method acquires write locks on blocks which it is considering as candidates for eviction. These lock acquisitions are non-blocking, so a block which is being read will not be evicted. By holding write locks until the eviction is performed or skipped (in case evicting the blocks would not free enough memory), we avoid a race where a new reader starts to read a block after the block has been marked as an eviction candidate but before it has been removed.

### Locking and remote block transfer

This patch makes small changes to to block transfer and network layer code so that locks acquired by the BlockTransferService are released as soon as block transfer messages are consumed and released by Netty. This builds on top of #11193, a bug fix related to freeing of network layer ManagedBuffers.

## FAQ

- **Why not use Java's built-in [`ReadWriteLock`](https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/locks/ReadWriteLock.html)?**

  Our locks operate on a per-task rather than per-thread level. Under certain circumstances a task may consist of multiple threads, so using `ReadWriteLock` would mean that we might call `unlock()` from a thread which didn't hold the lock in question, an operation which has undefined semantics. If we could rely on Java 8 classes, we might be able to use [`StampedLock`](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/locks/StampedLock.html) to work around this issue.

- **Why not detect "leaked" locks in tests?**:

  See above notes about `take()` and `limit`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10705 from JoshRosen/pin-pages.
2016-02-25 17:17:56 -08:00
Davies Liu 751724b132 Revert "[SPARK-13457][SQL] Removes DataFrame RDD operations"
This reverts commit 157fe64f3e.
2016-02-25 11:53:48 -08:00
Cheng Lian 157fe64f3e [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 <lian@databricks.com>

Closes #11323 from liancheng/remove-df-rdd-ops.
2016-02-25 23:07:59 +08:00
Cheng Lian 3fa6491be6 [SPARK-13473][SQL] Don't push predicate through project with nondeterministic field(s)
## What changes were proposed in this pull request?

Predicates shouldn't be pushed through project with nondeterministic field(s).

See https://github.com/graphframes/graphframes/pull/23 and SPARK-13473 for more details.

This PR targets master, branch-1.6, and branch-1.5.

## How was this patch tested?

A test case is added in `FilterPushdownSuite`. It constructs a query plan where a filter is over a project with a nondeterministic field. Optimized query plan shouldn't change in this case.

Author: Cheng Lian <lian@databricks.com>

Closes #11348 from liancheng/spark-13473-no-ppd-through-nondeterministic-project-field.
2016-02-25 20:43:03 +08:00
Reynold Xin 2b2c8c3323 [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 <rxin@databricks.com>

Closes #11363 from rxin/SPARK-13486.
2016-02-25 17:49:50 +08:00
Davies Liu 07f92ef1fa [SPARK-13376] [SPARK-13476] [SQL] improve column pruning
## What changes were proposed in this pull request?

This PR mostly rewrite the ColumnPruning rule to support most of the SQL logical plans (except those for Dataset).

This PR also fix a bug in Generate, it should always output UnsafeRow, added an regression test for that.

## How was this patch tested?

This is test by unit tests, also manually test with TPCDS Q78, which could prune all unused columns successfully, improved the performance by 78% (from 22s to 12s).

Author: Davies Liu <davies@databricks.com>

Closes #11354 from davies/fix_column_pruning.
2016-02-25 00:13:07 -08:00