Commit graph

2039 commits

Author SHA1 Message Date
bomeng 74fe235ab5 [SPARK-14398][SQL] Audit non-reserved keyword list in ANTLR4 parser
## What changes were proposed in this pull request?

I have compared non-reserved list in Antlr3 and Antlr4 one by one as well as all the existing keywords defined in Antlr4, added the missing keywords to the non-reserved keywords list.  If we need to support more syntax, we can add more keywords by then.

Any recommendation for the above is welcome.

## How was this patch tested?

I manually checked the keywords one by one. Please let me know if there is a better way to test.

Another thought: I suggest to put all the keywords definition and non-reserved list in order, that will be much easier to check in the future.

Author: bomeng <bmeng@us.ibm.com>

Closes #12191 from bomeng/SPARK-14398.
2016-04-19 09:09:58 +02:00
Sameer Agarwal 4eae1dbd7c [SPARK-14718][SQL] Avoid mutating ExprCode in doGenCode
## What changes were proposed in this pull request?

The `doGenCode` method currently takes in an `ExprCode`, mutates it and returns the java code to evaluate the given expression. It should instead just return a new `ExprCode` to avoid passing around mutable objects during code generation.

## How was this patch tested?

Existing Tests

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12483 from sameeragarwal/new-exprcode-2.
2016-04-18 20:28:22 -07:00
Sameer Agarwal 8bd8121329 [SPARK-14710][SQL] Rename gen/genCode to genCode/doGenCode to better reflect the semantics
## What changes were proposed in this pull request?

Per rxin's suggestions, this patch renames `s/gen/genCode` and `s/genCode/doGenCode` to better reflect the semantics of these 2 function calls.

## How was this patch tested?

N/A (refactoring only)

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12475 from sameeragarwal/gencode.
2016-04-18 14:03:40 -07:00
Reynold Xin e4ae974294 [HOTFIX] Fix Scala 2.10 compilation break. 2016-04-18 12:57:23 -07:00
Dongjoon Hyun d280d1da1a [SPARK-14580][SPARK-14655][SQL] Hive IfCoercion should preserve predicate.
## What changes were proposed in this pull request?

Currently, `HiveTypeCoercion.IfCoercion` removes all predicates whose return-type are null. However, some UDFs need evaluations because they are designed to throw exceptions. This PR fixes that to preserve the predicates. Also, `assert_true` is implemented as Spark SQL function.

**Before**
```
scala> sql("select if(assert_true(false),2,3)").head
res2: org.apache.spark.sql.Row = [3]
```

**After**
```
scala> sql("select if(assert_true(false),2,3)").head
... ASSERT_TRUE ...
```

**Hive**
```
hive> select if(assert_true(false),2,3);
OK
Failed with exception java.io.IOException:org.apache.hadoop.hive.ql.metadata.HiveException: ASSERT_TRUE(): assertion failed.
```

## How was this patch tested?

Pass the Jenkins tests (including a new testcase in `HivePlanTest`)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12340 from dongjoon-hyun/SPARK-14580.
2016-04-18 12:26:56 -07:00
Tathagata Das 775cf17eaa [SPARK-14473][SQL] Define analysis rules to catch operations not supported in streaming
## What changes were proposed in this pull request?

There are many operations that are currently not supported in the streaming execution. For example:
 - joining two streams
 - unioning a stream and a batch source
 - sorting
 - window functions (not time windows)
 - distinct aggregates

Furthermore, executing a query with a stream source as a batch query should also fail.

This patch add an additional step after analysis in the QueryExecution which will check that all the operations in the analyzed logical plan is supported or not.

## How was this patch tested?
unit tests.

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

Closes #12246 from tdas/SPARK-14473.
2016-04-18 11:09:33 -07:00
Dongjoon Hyun 432d1399cb [SPARK-14614] [SQL] Add bround function
## What changes were proposed in this pull request?

This PR aims to add `bound` function (aka Banker's round) by extending current `round` implementation. [Hive supports `bround` since 1.3.0.](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF)

**Hive (1.3 ~ 2.0)**
```
hive> select round(2.5), bround(2.5);
OK
3.0	2.0
```

**After this PR**
```scala
scala> sql("select round(2.5), bround(2.5)").head
res0: org.apache.spark.sql.Row = [3,2]
```

## How was this patch tested?

Pass the Jenkins tests (with extended tests).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12376 from dongjoon-hyun/SPARK-14614.
2016-04-18 10:44:51 -07:00
hyukjinkwon 9f678e9754 [MINOR] Remove inappropriate type notation and extra anonymous closure within functional transformations
## What changes were proposed in this pull request?

This PR removes

- Inappropriate type notations
    For example, from
    ```scala
    words.foreachRDD { (rdd: RDD[String], time: Time) =>
    ...
    ```
    to
    ```scala
    words.foreachRDD { (rdd, time) =>
    ...
    ```

- Extra anonymous closure within functional transformations.
    For example,
    ```scala
    .map(item => {
      ...
    })
    ```

    which can be just simply as below:

    ```scala
    .map { item =>
      ...
    }
    ```

and corrects some obvious style nits.

## How was this patch tested?

This was tested after adding rules in `scalastyle-config.xml`, which ended up with not finding all perfectly.

The rules applied were below:

- For the first correction,

```xml
<check customId="NoExtraClosure" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
    <parameters><parameter name="regex">(?m)\.[a-zA-Z_][a-zA-Z0-9]*\(\s*[^,]+s*=>\s*\{[^\}]+\}\s*\)</parameter></parameters>
</check>
```

```xml
<check customId="NoExtraClosure" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
    <parameters><parameter name="regex">\.[a-zA-Z_][a-zA-Z0-9]*\s*[\{|\(]([^\n>,]+=>)?\s*\{([^()]|(?R))*\}^[,]</parameter></parameters>
</check>
```

- For the second correction
```xml
<check customId="TypeNotation" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
    <parameters><parameter name="regex">\.[a-zA-Z_][a-zA-Z0-9]*\s*[\{|\(]\s*\([^):]*:R))*\}^[,]</parameter></parameters>
</check>
```

**Those rules were not added**

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12413 from HyukjinKwon/SPARK-style.
2016-04-16 14:56:23 +01:00
Reynold Xin f4be0946af [SPARK-14677][SQL] Make the max number of iterations configurable for Catalyst
## What changes were proposed in this pull request?
We currently hard code the max number of optimizer/analyzer iterations to 100. This patch makes it configurable. While I'm at it, I also added the SessionCatalog to the optimizer, so we can use information there in optimization.

## How was this patch tested?
Updated unit tests to reflect the change.

Author: Reynold Xin <rxin@databricks.com>

Closes #12434 from rxin/SPARK-14677.
2016-04-15 20:28:09 -07:00
Yin Huai b2dfa84959 [SPARK-14668][SQL] Move CurrentDatabase to Catalyst
## What changes were proposed in this pull request?

This PR moves `CurrentDatabase` from sql/hive package to sql/catalyst. It also adds the function description, which looks like the following.

```
scala> sqlContext.sql("describe function extended current_database").collect.foreach(println)
[Function: current_database]
[Class: org.apache.spark.sql.execution.command.CurrentDatabase]
[Usage: current_database() - Returns the current database.]
[Extended Usage:
> SELECT current_database()]
```

## How was this patch tested?
Existing tests

Author: Yin Huai <yhuai@databricks.com>

Closes #12424 from yhuai/SPARK-14668.
2016-04-15 17:48:41 -07:00
Wenchen Fan 297ba3f1b4 [SPARK-14275][SQL] Reimplement TypedAggregateExpression to DeclarativeAggregate
## What changes were proposed in this pull request?

`ExpressionEncoder` is just a container for serialization and deserialization expressions, we can use these expressions to build `TypedAggregateExpression` directly, so that it can fit in `DeclarativeAggregate`, which is more efficient.

One trick is, for each buffer serializer expression, it will reference to the result object of serialization and function call. To avoid re-calculating this result object, we can serialize the buffer object to a single struct field, so that we can use a special `Expression` to only evaluate result object once.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12067 from cloud-fan/typed_udaf.
2016-04-15 12:10:00 +08:00
Dongjoon Hyun d7e124edfe [SPARK-14545][SQL] Improve LikeSimplification by adding a%b rule
## What changes were proposed in this pull request?

Current `LikeSimplification` handles the following four rules.
- 'a%' => expr.StartsWith("a")
- '%b' => expr.EndsWith("b")
- '%a%' => expr.Contains("a")
- 'a' => EqualTo("a")

This PR adds the following rule.
- 'a%b' => expr.Length() >= 2 && expr.StartsWith("a") && expr.EndsWith("b")

Here, 2 is statically calculated from "a".size + "b".size.

**Before**
```
scala> sql("select a from (select explode(array('abc','adc')) a) T where a like 'a%c'").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Filter a#5 LIKE a%c
:     +- INPUT
+- Generate explode([abc,adc]), false, false, [a#5]
   +- Scan OneRowRelation[]
```

**After**
```
scala> sql("select a from (select explode(array('abc','adc')) a) T where a like 'a%c'").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Filter ((length(a#5) >= 2) && (StartsWith(a#5, a) && EndsWith(a#5, c)))
:     +- INPUT
+- Generate explode([abc,adc]), false, false, [a#5]
   +- Scan OneRowRelation[]
```

## How was this patch tested?

Pass the Jenkins tests (including new testcase).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12312 from dongjoon-hyun/SPARK-14545.
2016-04-14 13:34:29 -07:00
Liang-Chi Hsieh 28efdd3fd7 [SPARK-14592][SQL] Native support for CREATE TABLE LIKE DDL command
## What changes were proposed in this pull request?
JIRA: https://issues.apache.org/jira/browse/SPARK-14592

This patch adds native support for DDL command `CREATE TABLE LIKE`.

The SQL syntax is like:

    CREATE TABLE table_name LIKE existing_table
    CREATE TABLE IF NOT EXISTS table_name LIKE existing_table

## How was this patch tested?
`HiveDDLCommandSuite`. `HiveQuerySuite` already tests `CREATE TABLE LIKE`.

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

This patch had conflicts when merged, resolved by
Committer: Andrew Or <andrew@databricks.com>

Closes #12362 from viirya/create-table-like.
2016-04-14 11:08:08 -07:00
Liwei Lin 3e27940a19 [SPARK-14630][BUILD][CORE][SQL][STREAMING] Code style: public abstract methods should have explicit return types
## What changes were proposed in this pull request?

Currently many public abstract methods (in abstract classes as well as traits) don't declare return types explicitly, such as in [o.a.s.streaming.dstream.InputDStream](https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala#L110):
```scala
def start() // should be: def start(): Unit
def stop()  // should be: def stop(): Unit
```

These methods exist in core, sql, streaming; this PR fixes them.

## How was this patch tested?

N/A

## Which piece of scala style rule led to the changes?

the rule was added separately in https://github.com/apache/spark/pull/12396

Author: Liwei Lin <lwlin7@gmail.com>

Closes #12389 from lw-lin/public-abstract-methods.
2016-04-14 10:14:38 -07:00
hyukjinkwon 6fc3dc8839 [MINOR][SQL] Remove extra anonymous closure within functional transformations
## What changes were proposed in this pull request?

This PR removes extra anonymous closure within functional transformations.

For example,

```scala
.map(item => {
  ...
})
```

which can be just simply as below:

```scala
.map { item =>
  ...
}
```

## How was this patch tested?

Related unit tests and `sbt scalastyle`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12382 from HyukjinKwon/minor-extra-closers.
2016-04-14 09:43:41 +01:00
hyukjinkwon b4819404a6 [SPARK-14596][SQL] Remove not used SqlNewHadoopRDD and some more unused imports
## What changes were proposed in this pull request?

Old `HadoopFsRelation` API includes `buildInternalScan()` which uses `SqlNewHadoopRDD` in `ParquetRelation`.
Because now the old API is removed, `SqlNewHadoopRDD` is not used anymore.

So, this PR removes `SqlNewHadoopRDD` and several unused imports.

This was discussed in https://github.com/apache/spark/pull/12326.

## How was this patch tested?

Several related existing unit tests and `sbt scalastyle`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12354 from HyukjinKwon/SPARK-14596.
2016-04-14 15:43:44 +08:00
Davies Liu dbbe149070 [SPARK-14581] [SQL] push predicatese through more logical plans
## What changes were proposed in this pull request?

Right now, filter push down only works with Project, Aggregate, Generate and Join, they can't be pushed through many other plans.

This PR added support for Union, Intersect, Except and all unary plans.

## How was this patch tested?

Added tests.

Author: Davies Liu <davies@databricks.com>

Closes #12342 from davies/filter_hint.
2016-04-13 13:01:13 -07:00
Andrew Or 7d2ed8cc03 [SPARK-14388][SQL] Implement CREATE TABLE
## What changes were proposed in this pull request?

This patch implements the `CREATE TABLE` command using the `SessionCatalog`. Previously we handled only `CTAS` and `CREATE TABLE ... USING`. This requires us to refactor `CatalogTable` to accept various fields (e.g. bucket and skew columns) and pass them to Hive.

WIP: Note that I haven't verified whether this actually works yet! But I believe it does.

## How was this patch tested?

Tests will come in a future commit.

Author: Andrew Or <andrew@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #12271 from andrewor14/create-table-ddl.
2016-04-13 11:08:34 -07:00
Davies Liu 372baf0479 [SPARK-14578] [SQL] Fix codegen for CreateExternalRow with nested wide schema
## What changes were proposed in this pull request?

The wide schema, the expression of fields will be splitted into multiple functions, but the variable for loopVar can't be accessed in splitted functions, this PR change them as class member.

## How was this patch tested?

Added regression test.

Author: Davies Liu <davies@databricks.com>

Closes #12338 from davies/nested_row.
2016-04-12 17:26:37 -07:00
bomeng bcd2076274 [SPARK-14414][SQL] improve the error message class hierarchy
## What changes were proposed in this pull request?

Before we are using `AnalysisException`, `ParseException`, `NoSuchFunctionException` etc when a parsing error encounters. I am trying to make it consistent and also **minimum** code impact to the current implementation by changing the class hierarchy.
1. `NoSuchItemException` is removed, since it is an abstract class and it just simply takes a message string.
2. `NoSuchDatabaseException`, `NoSuchTableException`, `NoSuchPartitionException` and `NoSuchFunctionException` now extends `AnalysisException`, as well as `ParseException`, they are all under `AnalysisException` umbrella, but you can also determine how to use them in a granular way.

## How was this patch tested?
The existing test cases should cover this patch.

Author: bomeng <bmeng@us.ibm.com>

Closes #12314 from bomeng/SPARK-14414.
2016-04-12 13:43:39 -07:00
Davies Liu 85e68b4bea [SPARK-14562] [SQL] improve constraints propagation in Union
## What changes were proposed in this pull request?

Currently, Union only takes intersect of the constraints from it's children, all others are dropped, we should try to merge them together.

This PR try to merge the constraints that have the same reference but came from different children, for example: `a > 10` and `a < 100` could be merged as `a > 10 || a < 100`.

## How was this patch tested?

Added more cases in existing test.

Author: Davies Liu <davies@databricks.com>

Closes #12328 from davies/union_const.
2016-04-12 12:29:54 -07:00
Dongjoon Hyun b0f5497e95 [SPARK-14508][BUILD] Add a new ScalaStyle Rule OmitBracesInCase
## What changes were proposed in this pull request?

According to the [Spark Code Style Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide) and [Scala Style Guide](http://docs.scala-lang.org/style/control-structures.html#curlybraces), we had better enforce the following rule.
  ```
  case: Always omit braces in case clauses.
  ```
This PR makes a new ScalaStyle rule, 'OmitBracesInCase', and enforces it to the code.

## How was this patch tested?

Pass the Jenkins tests (including Scala style checking)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12280 from dongjoon-hyun/SPARK-14508.
2016-04-12 00:43:28 -07:00
Andrew Or 83fb96403b [SPARK-14132][SPARK-14133][SQL] Alter table partition DDLs
## What changes were proposed in this pull request?

This implements a few alter table partition commands using the `SessionCatalog`. In particular:
```
ALTER TABLE ... ADD PARTITION ...
ALTER TABLE ... DROP PARTITION ...
ALTER TABLE ... RENAME PARTITION ... TO ...
```
The following operations are not supported, and an `AnalysisException` with a helpful error message will be thrown if the user tries to use them:
```
ALTER TABLE ... EXCHANGE PARTITION ...
ALTER TABLE ... ARCHIVE PARTITION ...
ALTER TABLE ... UNARCHIVE PARTITION ...
ALTER TABLE ... TOUCH ...
ALTER TABLE ... COMPACT ...
ALTER TABLE ... CONCATENATE
MSCK REPAIR TABLE ...
```

## How was this patch tested?

`DDLSuite`, `DDLCommandSuite` and `HiveDDLCommandSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #12220 from andrewor14/alter-partition-ddl.
2016-04-11 20:59:45 -07:00
Dongjoon Hyun 5de26194a3 [SPARK-14502] [SQL] Add optimization for Binary Comparison Simplification
## What changes were proposed in this pull request?

We can simplifies binary comparisons with semantically-equal operands:

1. Replace '<=>' with 'true' literal.
2. Replace '=', '<=', and '>=' with 'true' literal if both operands are non-nullable.
3. Replace '<' and '>' with 'false' literal if both operands are non-nullable.

For example, the following example plan
```
scala> sql("SELECT * FROM (SELECT explode(array(1,2,3)) a) T WHERE a BETWEEN a AND a+7").explain()
...
:  +- Filter ((a#59 >= a#59) && (a#59 <= (a#59 + 7)))
...
```
will be optimized into the following.
```
:  +- Filter (a#47 <= (a#47 + 7))
```

## How was this patch tested?

Pass the Jenkins tests including new `BinaryComparisonSimplificationSuite`.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12267 from dongjoon-hyun/SPARK-14502.
2016-04-11 09:52:50 -07:00
Davies Liu 652c470309 [SPARK-14528] [SQL] Fix same result of Union
## What changes were proposed in this pull request?

This PR fix resultResult() for Union.

## How was this patch tested?

Added regression test.

Author: Davies Liu <davies@databricks.com>

Closes #12295 from davies/fix_sameResult.
2016-04-11 09:43:16 -07:00
gatorsmile 9f838bd242 [SPARK-14362][SPARK-14406][SQL][FOLLOW-UP] DDL Native Support: Drop View and Drop Table
#### What changes were proposed in this pull request?
This PR is to address the comment: https://github.com/apache/spark/pull/12146#discussion-diff-59092238. It removes the function `isViewSupported` from `SessionCatalog`. After the removal, we still can capture the user errors if users try to drop a table using `DROP VIEW`.

#### How was this patch tested?
Modified the existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12284 from gatorsmile/followupDropTable.
2016-04-10 20:46:15 -07:00
Dongjoon Hyun a7ce473bd0 [SPARK-14415][SQL] All functions should show usages by command DESC FUNCTION
## What changes were proposed in this pull request?

Currently, many functions do now show usages like the followings.
```
scala> sql("desc function extended `sin`").collect().foreach(println)
[Function: sin]
[Class: org.apache.spark.sql.catalyst.expressions.Sin]
[Usage: To be added.]
[Extended Usage:
To be added.]
```

This PR adds descriptions for functions and adds a testcase prevent adding function without usage.
```
scala>  sql("desc function extended `sin`").collect().foreach(println);
[Function: sin]
[Class: org.apache.spark.sql.catalyst.expressions.Sin]
[Usage: sin(x) - Returns the sine of x.]
[Extended Usage:
> SELECT sin(0);
 0.0]
```

The only exceptions are `cube`, `grouping`, `grouping_id`, `rollup`, `window`.

## How was this patch tested?

Pass the Jenkins tests (including new testcases.)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12185 from dongjoon-hyun/SPARK-14415.
2016-04-10 11:46:45 -07:00
Yin Huai 3fb09afd5e [SPARK-14506][SQL] HiveClientImpl's toHiveTable misses a table property for external tables
## What changes were proposed in this pull request?

For an external table's metadata (in Hive's representation), its table type needs to be EXTERNAL_TABLE. Also, there needs to be a field called EXTERNAL set in the table property with a value of TRUE (for a MANAGED_TABLE it will be FALSE) based on https://github.com/apache/hive/blob/release-1.2.1/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1095-L1105. HiveClientImpl's toHiveTable misses to set this table property.

## How was this patch tested?

Added a new test.

Author: Yin Huai <yhuai@databricks.com>

Closes #12275 from yhuai/SPARK-14506.
2016-04-09 23:32:17 -07:00
gatorsmile dfce9665c4 [SPARK-14362][SPARK-14406][SQL] DDL Native Support: Drop View and Drop Table
#### What changes were proposed in this pull request?

This PR is to provide a native support for DDL `DROP VIEW` and `DROP TABLE`. The PR includes native parsing and native analysis.

Based on the HIVE DDL document for [DROP_VIEW_WEB_LINK](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-
DropView
), `DROP VIEW` is defined as,
**Syntax:**
```SQL
DROP VIEW [IF EXISTS] [db_name.]view_name;
```
 - to remove metadata for the specified view.
 - illegal to use DROP TABLE on a view.
 - illegal to use DROP VIEW on a table.
 - this command only works in `HiveContext`. In `SQLContext`, we will get an exception.

This PR also handles `DROP TABLE`.
**Syntax:**
```SQL
DROP TABLE [IF EXISTS] table_name [PURGE];
```
- Previously, the `DROP TABLE` command only can drop Hive tables in `HiveContext`. Now, after this PR, this command also can drop temporary table, external table, external data source table in `SQLContext`.
- In `HiveContext`, we will not issue an exception if the to-be-dropped table does not exist and users did not specify `IF EXISTS`. Instead, we just log an error message. If `IF EXISTS` is specified, we will not issue any error message/exception.
- In `SQLContext`, we will issue an exception if the to-be-dropped table does not exist, unless `IF EXISTS` is specified.
- Data will not be deleted if the tables are `external`, unless table type is `managed_table`.

#### How was this patch tested?
For verifying command parsing, added test cases in `spark/sql/hive/HiveDDLCommandSuite.scala`
For verifying command analysis, added test cases in `spark/sql/hive/execution/HiveDDLSuite.scala`

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

Closes #12146 from gatorsmile/dropView.
2016-04-09 17:40:36 -07:00
Yong Tang cd2fed7012 [SPARK-14335][SQL] Describe function command returns wrong output
## What changes were proposed in this pull request?

…because some of built-in functions are not in function registry.

This fix tries to fix issues in `describe function` command where some of the outputs
still shows Hive's function because some built-in functions are not in FunctionRegistry.

The following built-in functions have been added to FunctionRegistry:
```
-
!
*
/
&
%
^
+
<
<=
<=>
=
==
>
>=
|
~
and
in
like
not
or
rlike
when
```

The following listed functions are not added, but hard coded in `commands.scala` (hvanhovell):
```
!=
<>
between
case
```
Below are the existing result of the above functions that have not been added:
```
spark-sql> describe function `!=`;
Function: <>
Class: org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual
Usage: a <> b - Returns TRUE if a is not equal to b
```
```
spark-sql> describe function `<>`;
Function: <>
Class: org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual
Usage: a <> b - Returns TRUE if a is not equal to b
```
```
spark-sql> describe function `between`;
Function: between
Class: org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween
Usage: between a [NOT] BETWEEN b AND c - evaluate if a is [not] in between b and c
```
```
spark-sql> describe function `case`;
Function: case
Class: org.apache.hadoop.hive.ql.udf.generic.GenericUDFCase
Usage: CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END - When a = b, returns c; when a = d, return e; else return f
```

## How was this patch tested?

Existing tests passed. Additional test cases added.

Author: Yong Tang <yong.tang.github@outlook.com>

Closes #12128 from yongtang/SPARK-14335.
2016-04-09 13:54:30 -07:00
bomeng 10a95781ee [SPARK-14496][SQL] fix some javadoc typos
## What changes were proposed in this pull request?

Minor issues. Found 2 typos while browsing the code.

## How was this patch tested?
None.

Author: bomeng <bmeng@us.ibm.com>

Closes #12264 from bomeng/SPARK-14496.
2016-04-09 22:30:54 +09:00
Jacek Laskowski 6447098013 [SPARK-14402][HOTFIX] Fix ExpressionDescription annotation
## What changes were proposed in this pull request?

Fix for the error introduced in c59abad052:

```
/Users/jacek/dev/oss/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:626: error: annotation argument needs to be a constant; found: "_FUNC_(str) - ".+("Returns str, with the first letter of each word in uppercase, all other letters in ").+("lowercase. Words are delimited by white space.")
    "Returns str, with the first letter of each word in uppercase, all other letters in " +
                                                                                          ^
```

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #12192 from jaceklaskowski/SPARK-14402-HOTFIX.
2016-04-08 11:36:41 +01:00
Wenchen Fan 49fb237081 [SPARK-14270][SQL] whole stage codegen support for typed filter
## What changes were proposed in this pull request?

We implement typed filter by `MapPartitions`, which doesn't work well with whole stage codegen. This PR use `Filter` to implement typed filter and we can get the whole stage codegen support for free.

This PR also introduced `DeserializeToObject` and `SerializeFromObject`, to seperate serialization logic from object operator, so that it's eaiser to write optimization rules for adjacent object operators.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12061 from cloud-fan/whole-stage-codegen.
2016-04-07 17:23:34 -07:00
Andrew Or ae1db91d15 [SPARK-14410][SQL] Push functions existence check into catalog
## What changes were proposed in this pull request?

This is a followup to #12117 and addresses some of the TODOs introduced there. In particular, the resolution of database is now pushed into session catalog, which knows about the current database. Further, the logic for checking whether a function exists is pushed into the external catalog.

No change in functionality is expected.

## How was this patch tested?

`SessionCatalogSuite`, `DDLSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #12198 from andrewor14/function-exists.
2016-04-07 16:23:17 -07:00
Davies Liu aa852215f8 [SPARK-12740] [SPARK-13932] support grouping()/grouping_id() in having/order clause
## What changes were proposed in this pull request?

This PR brings the support of using grouping()/grouping_id() in HAVING/ORDER BY clause.

The resolved grouping()/grouping_id() will be replaced by unresolved "spark_gropuing_id" virtual attribute, then resolved by ResolveMissingAttribute.

This PR also fix the HAVING clause that access a grouping column that is not presented in SELECT clause, for example:
```sql
select count(1) from (select 1 as a) t group by a having a > 0
```
## How was this patch tested?

Add new tests.

Author: Davies Liu <davies@databricks.com>

Closes #12235 from davies/grouping_having.
2016-04-07 11:51:34 -07:00
Reynold Xin e11aa9ec5c [SPARK-14452][SQL] Explicit APIs in Scala for specifying encoders
## What changes were proposed in this pull request?
The Scala Dataset public API currently only allows users to specify encoders through SQLContext.implicits. This is OK but sometimes people want to explicitly get encoders without a SQLContext (e.g. Aggregator implementations). This patch adds public APIs to Encoders class for getting Scala encoders.

## How was this patch tested?
None - I will update test cases once https://github.com/apache/spark/pull/12231 is merged.

Author: Reynold Xin <rxin@databricks.com>

Closes #12232 from rxin/SPARK-14452.
2016-04-07 00:46:57 -07:00
Marcelo Vanzin 21d5ca128b [SPARK-14134][CORE] Change the package name used for shading classes.
The current package name uses a dash, which is a little weird but seemed
to work. That is, until a new test tried to mock a class that references
one of those shaded types, and then things started failing.

Most changes are just noise to fix the logging configs.

For reference, SPARK-8815 also raised this issue, although at the time it
did not cause any issues in Spark, so it was not addressed.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #11941 from vanzin/SPARK-14134.
2016-04-06 19:33:51 -07:00
Herman van Hovell d76592276f [SPARK-12610][SQL] Left Anti Join
### What changes were proposed in this pull request?

This PR adds support for `LEFT ANTI JOIN` to Spark SQL. A `LEFT ANTI JOIN` is the exact opposite of a `LEFT SEMI JOIN` and can be used to identify rows in one dataset that are not in another dataset. Note that `nulls` on the left side of the join cannot match a row on the right hand side of the join; the result is that left anti join will always select a row with a `null` in one or more of its keys.

We currently add support for the following SQL join syntax:

    SELECT   *
    FROM      tbl1 A
              LEFT ANTI JOIN tbl2 B
               ON A.Id = B.Id

Or using a dataframe:

    tbl1.as("a").join(tbl2.as("b"), $"a.id" === $"b.id", "left_anti)

This PR provides serves as the basis for implementing `NOT EXISTS` and `NOT IN (...)` correlated sub-queries. It would also serve as good basis for implementing an more efficient `EXCEPT` operator.

The PR has been (losely) based on PR's by both davies (https://github.com/apache/spark/pull/10706) and chenghao-intel (https://github.com/apache/spark/pull/10563); credit should be given where credit is due.

This PR adds supports for `LEFT ANTI JOIN` to `BroadcastHashJoin` (including codegeneration), `ShuffledHashJoin` and `BroadcastNestedLoopJoin`.

### How was this patch tested?

Added tests to `JoinSuite` and ported `ExistenceJoinSuite` from https://github.com/apache/spark/pull/10563.

cc davies chenghao-intel rxin

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

Closes #12214 from hvanhovell/SPARK-12610.
2016-04-06 19:25:10 -07:00
Dongjoon Hyun d717ae1fd7 [SPARK-14444][BUILD] Add a new scalastyle NoScalaDoc to prevent ScalaDoc-style multiline comments
## What changes were proposed in this pull request?

According to the [Spark Code Style Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide#SparkCodeStyleGuide-Indentation), this PR adds a new scalastyle rule to prevent the followings.
```
/** In Spark, we don't use the ScalaDoc style so this
  * is not correct.
  */
```

## How was this patch tested?

Pass the Jenkins tests (including `lint-scala`).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12221 from dongjoon-hyun/SPARK-14444.
2016-04-06 16:02:55 -07:00
Davies Liu 5a4b11a901 [SPARK-14224] [SPARK-14223] [SPARK-14310] [SQL] fix RowEncoder and parquet reader for wide table
## What changes were proposed in this pull request?

1) fix the RowEncoder for wide table (many columns) by splitting the generate code into multiple functions.
2) Separate DataSourceScan as RowDataSourceScan and BatchedDataSourceScan
3) Disable the returning columnar batch in parquet reader if there are many columns.
4) Added a internal config for maximum number of fields (nested) columns supported by whole stage codegen.

Closes #12098

## How was this patch tested?

Add a tests for table with 1000 columns.

Author: Davies Liu <davies@databricks.com>

Closes #12047 from davies/many_columns.
2016-04-06 15:33:39 -07:00
bomeng 3c8d882165 [SPARK-14383][SQL] missing "|" in the g4 file
## What changes were proposed in this pull request?

A very trivial one. It missed "|" between DISTRIBUTE and UNSET.

## How was this patch tested?

I do not think it is really needed.

Author: bomeng <bmeng@us.ibm.com>

Closes #12156 from bomeng/SPARK-14383.
2016-04-06 11:12:48 -07:00
bomeng 5abd02c02b [SPARK-14429][SQL] Improve LIKE pattern in "SHOW TABLES / FUNCTIONS LIKE <pattern>" DDL
LIKE <pattern> is commonly used in SHOW TABLES / FUNCTIONS etc DDL. In the pattern, user can use `|` or `*` as wildcards.

1. Currently, we used `replaceAll()` to replace `*` with `.*`, but the replacement was scattered in several places; I have created an utility method and use it in all the places;

2. Consistency with Hive: the pattern is case insensitive in Hive and white spaces will be trimmed, but current pattern matching does not do that. For example, suppose we have tables (t1, t2, t3), `SHOW TABLES LIKE ' T* ' ` will list all the t-tables. Please use Hive to verify it.

3. Combined with `|`, the result will be sorted. For pattern like `'  B*|a*  '`, it will list the result in a-b order.

I've made some changes to the utility method to make sure we will get the same result as Hive does.

A new method was created in StringUtil and test cases were added.

andrewor14

Author: bomeng <bmeng@us.ibm.com>

Closes #12206 from bomeng/SPARK-14429.
2016-04-06 11:06:14 -07:00
Kousuke Saruta 10494feae0 [SPARK-14426][SQL] Merge PerserUtils and ParseUtils
## What changes were proposed in this pull request?

We have ParserUtils and ParseUtils which are both utility collections for use during the parsing process.
Those names and what they are used for is very similar so I think we can merge them.

Also, the original unescapeSQLString method may have a fault. When "\u0061" style character literals are passed to the method, it's not unescaped successfully.
This patch fix the bug.

## How was this patch tested?

Added a new test case.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #12199 from sarutak/merge-ParseUtils-and-ParserUtils.
2016-04-06 10:57:46 -07:00
Wenchen Fan f6456fa80b [SPARK-14296][SQL] whole stage codegen support for Dataset.map
## What changes were proposed in this pull request?

This PR adds a new operator `MapElements` for `Dataset.map`, it's a 1-1 mapping and is easier to adapt to whole stage codegen framework.

## How was this patch tested?

new test in `WholeStageCodegenSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12087 from cloud-fan/map.
2016-04-06 12:09:10 +08:00
Andrew Or 45d8cdee39 [SPARK-14129][SPARK-14128][SQL] Alter table DDL commands
## What changes were proposed in this pull request?

In Spark 2.0, we want to handle the most common `ALTER TABLE` commands ourselves instead of passing the entire query text to Hive. This is done using the new `SessionCatalog` API introduced recently.

The commands supported in this patch include:
```
ALTER TABLE ... RENAME TO ...
ALTER TABLE ... SET TBLPROPERTIES ...
ALTER TABLE ... UNSET TBLPROPERTIES ...
ALTER TABLE ... SET LOCATION ...
ALTER TABLE ... SET SERDE ...
```
The commands we explicitly do not support are:
```
ALTER TABLE ... CLUSTERED BY ...
ALTER TABLE ... SKEWED BY ...
ALTER TABLE ... NOT CLUSTERED
ALTER TABLE ... NOT SORTED
ALTER TABLE ... NOT SKEWED
ALTER TABLE ... NOT STORED AS DIRECTORIES
```
For these we throw exceptions complaining that they are not supported.

## How was this patch tested?

`DDLSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #12121 from andrewor14/alter-table-ddl.
2016-04-05 14:54:07 -07:00
Dongjoon Hyun c59abad052 [SPARK-14402][SQL] initcap UDF doesn't match Hive/Oracle behavior in lowercasing rest of string
## What changes were proposed in this pull request?

Current, SparkSQL `initCap` is using `toTitleCase` function. However, `UTF8String.toTitleCase` implementation changes only the first letter and just copy the other letters: e.g. sParK --> SParK. This is the correct implementation `toTitleCase`.
```
hive> select initcap('sParK');
Spark
```
```
scala> sql("select initcap('sParK')").head
res0: org.apache.spark.sql.Row = [SParK]
```

This PR updates the implementation of `initcap` using `toLowerCase` and `toTitleCase`.

## How was this patch tested?

Pass the Jenkins tests (including new testcase).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12175 from dongjoon-hyun/SPARK-14402.
2016-04-05 13:31:00 -07:00
Burak Yavuz 9ee5c25717 [SPARK-14353] Dataset Time Window window API for Python, and SQL
## What changes were proposed in this pull request?

The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.

With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
 - `window(timeColumn, windowDuration)`
 - `window(timeColumn, windowDuration, slideDuration)`
 - `window(timeColumn, windowDuration, slideDuration, startTime)`

In Python, users can access all APIs above, but in addition they can do
 - In Python:
   `window(timeColumn, windowDuration, startTime=...)`

that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.

## How was this patch tested?

Unit tests + manual tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #12136 from brkyvz/python-windows.
2016-04-05 13:18:39 -07:00
Yin Huai 72544d6f2a [SPARK-14123][SPARK-14384][SQL] Handle CreateFunction/DropFunction
## What changes were proposed in this pull request?
This PR implements CreateFunction and DropFunction commands. Besides implementing these two commands, we also change how to manage functions. Here are the main changes.
* `FunctionRegistry` will be a container to store all functions builders and it will not actively load any functions. Because of this change, we do not need to maintain a separate registry for HiveContext. So, `HiveFunctionRegistry` is deleted.
* SessionCatalog takes care the job of loading a function if this function is not in the `FunctionRegistry` but its metadata is stored in the external catalog. For this case, SessionCatalog will (1) load the metadata from the external catalog, (2) load all needed resources (i.e. jars and files), (3) create a function builder based on the function definition, (4) register the function builder in the `FunctionRegistry`.
* A `UnresolvedGenerator` is created. So, the parser will not need to call `FunctionRegistry` directly during parsing, which is not a good time to create a Hive UDTF. In the analysis phase, we will resolve `UnresolvedGenerator`.

This PR is based on viirya's https://github.com/apache/spark/pull/12036/

## How was this patch tested?
Existing tests and new tests.

## TODOs
[x] Self-review
[x] Cleanup
[x] More tests for create/drop functions (we need to more tests for permanent functions).
[ ] File JIRAs for all TODOs
[x] Standardize the error message when a function does not exist.

Author: Yin Huai <yhuai@databricks.com>
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #12117 from yhuai/function.
2016-04-05 12:27:06 -07:00
Wenchen Fan f77f11c671 [SPARK-14345][SQL] Decouple deserializer expression resolution from ObjectOperator
## What changes were proposed in this pull request?

This PR decouples deserializer expression resolution from `ObjectOperator`, so that we can use deserializer expression in normal operators. This is needed by #12061 and #12067 , I abstracted the logic out and put them in this PR to reduce code change in the future.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12131 from cloud-fan/separate.
2016-04-05 10:53:54 -07:00
gatorsmile 7807173679 [SPARK-14349][SQL] Issue Error Messages for Unsupported Operators/DML/DDL in SQL Context.
#### What changes were proposed in this pull request?

Currently, the weird error messages are issued if we use Hive Context-only operations in SQL Context.

For example,
- When calling `Drop Table` in SQL Context, we got the following message:
```
Expected exception org.apache.spark.sql.catalyst.parser.ParseException to be thrown, but java.lang.ClassCastException was thrown.
```

- When calling `Script Transform` in SQL Context, we got the message:
```
assertion failed: No plan for ScriptTransformation [key#9,value#10], cat, [tKey#155,tValue#156], null
+- LogicalRDD [key#9,value#10], MapPartitionsRDD[3] at beforeAll at BeforeAndAfterAll.scala:187
```

Updates:
Based on the investigation from hvanhovell , the root cause is `visitChildren`, which is the default implementation. It always returns the result of the last defined context child. After merging the code changes from hvanhovell , it works! Thank you hvanhovell !

#### How was this patch tested?
A few test cases are added.

Not sure if the same issue exist for the other operators/DDL/DML. hvanhovell

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Herman van Hovell <hvanhovell@questtec.nl>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12134 from gatorsmile/hiveParserCommand.
2016-04-05 11:19:46 +02:00
Dilip Biswal 2715bc68bd [SPARK-14348][SQL] Support native execution of SHOW TBLPROPERTIES command
## What changes were proposed in this pull request?

This PR adds Native execution of SHOW TBLPROPERTIES command.

Command Syntax:
``` SQL
SHOW TBLPROPERTIES table_name[(property_key_literal)]
```
## How was this patch tested?

Tests added in HiveComandSuiie and DDLCommandSuite

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

Closes #12133 from dilipbiswal/dkb_show_tblproperties.
2016-04-05 08:41:59 +02:00
Dongjoon Hyun 3f749f7ed4 [SPARK-14355][BUILD] Fix typos in Exception/Testcase/Comments and static analysis results
## What changes were proposed in this pull request?

This PR contains the following 5 types of maintenance fix over 59 files (+94 lines, -93 lines).
- Fix typos(exception/log strings, testcase name, comments) in 44 lines.
- Fix lint-java errors (MaxLineLength) in 6 lines. (New codes after SPARK-14011)
- Use diamond operators in 40 lines. (New codes after SPARK-13702)
- Fix redundant semicolon in 5 lines.
- Rename class `InferSchemaSuite` to `CSVInferSchemaSuite` in CSVInferSchemaSuite.scala.

## How was this patch tested?

Manual and pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12139 from dongjoon-hyun/SPARK-14355.
2016-04-03 18:14:16 -07:00
bomeng c238cd0744 [SPARK-14341][SQL] Throw exception on unsupported create / drop macro ddl
## What changes were proposed in this pull request?

We throw an AnalysisException that looks like this:

```
scala> sqlContext.sql("CREATE TEMPORARY MACRO SIGMOID (x DOUBLE) 1.0 / (1.0 + EXP(-x))")
org.apache.spark.sql.catalyst.parser.ParseException:
Unsupported SQL statement
== SQL ==
CREATE TEMPORARY MACRO SIGMOID (x DOUBLE) 1.0 / (1.0 + EXP(-x))
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.nativeCommand(ParseDriver.scala:66)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser$$anonfun$parsePlan$1.apply(ParseDriver.scala:56)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser$$anonfun$parsePlan$1.apply(ParseDriver.scala:53)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:86)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:53)
  at org.apache.spark.sql.SQLContext.parseSql(SQLContext.scala:198)
  at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:749)
  ... 48 elided

```

## How was this patch tested?

Add test cases in HiveQuerySuite.scala

Author: bomeng <bmeng@us.ibm.com>

Closes #12125 from bomeng/SPARK-14341.
2016-04-03 17:15:02 +02:00
Reynold Xin 7be4620508 [HOTFIX] Fix Scala 2.10 compilation 2016-04-02 23:05:23 -07:00
Dongjoon Hyun 4a6e78abd9 [MINOR][DOCS] Use multi-line JavaDoc comments in Scala code.
## What changes were proposed in this pull request?

This PR aims to fix all Scala-Style multiline comments into Java-Style multiline comments in Scala codes.
(All comment-only changes over 77 files: +786 lines, −747 lines)

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12130 from dongjoon-hyun/use_multiine_javadoc_comments.
2016-04-02 17:50:40 -07:00
Dongjoon Hyun f705037617 [SPARK-14338][SQL] Improve SimplifyConditionals rule to handle null in IF/CASEWHEN
## What changes were proposed in this pull request?

Currently, `SimplifyConditionals` handles `true` and `false` to optimize branches. This PR improves `SimplifyConditionals` to take advantage of `null` conditions for `if` and `CaseWhen` expressions, too.

**Before**
```
scala> sql("SELECT IF(null, 1, 0)").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [if (null) 1 else 0 AS (IF(CAST(NULL AS BOOLEAN), 1, 0))#4]
:     +- INPUT
+- Scan OneRowRelation[]
scala> sql("select case when cast(null as boolean) then 1 else 2 end").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [CASE WHEN null THEN 1 ELSE 2 END AS CASE WHEN CAST(NULL AS BOOLEAN) THEN 1 ELSE 2 END#14]
:     +- INPUT
+- Scan OneRowRelation[]
```

**After**
```
scala> sql("SELECT IF(null, 1, 0)").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [0 AS (IF(CAST(NULL AS BOOLEAN), 1, 0))#4]
:     +- INPUT
+- Scan OneRowRelation[]
scala> sql("select case when cast(null as boolean) then 1 else 2 end").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [2 AS CASE WHEN CAST(NULL AS BOOLEAN) THEN 1 ELSE 2 END#4]
:     +- INPUT
+- Scan OneRowRelation[]
```

**Hive**
```
hive> select if(null,1,2);
OK
2
hive> select case when cast(null as boolean) then 1 else 2 end;
OK
2
```

## How was this patch tested?

Pass the Jenkins tests (including new extended test cases).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12122 from dongjoon-hyun/SPARK-14338.
2016-04-02 17:48:53 -07:00
Jacek Laskowski 06694f1c68 [MINOR] Typo fixes
## What changes were proposed in this pull request?

Typo fixes. No functional changes.

## How was this patch tested?

Built the sources and ran with samples.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #11802 from jaceklaskowski/typo-fixes.
2016-04-02 08:12:04 -07:00
Dongjoon Hyun fa1af0aff7 [SPARK-14251][SQL] Add SQL command for printing out generated code for debugging
## What changes were proposed in this pull request?

This PR implements `EXPLAIN CODEGEN` SQL command which returns generated codes like `debugCodegen`. In `spark-shell`, we don't need to `import debug` module. In `spark-sql`, we can use this SQL command now.

**Before**
```
scala> import org.apache.spark.sql.execution.debug._
scala> sql("select 'a' as a group by 1").debugCodegen()
Found 2 WholeStageCodegen subtrees.
== Subtree 1 / 2 ==
...

Generated code:
...

== Subtree 2 / 2 ==
...

Generated code:
...
```

**After**
```
scala> sql("explain extended codegen select 'a' as a group by 1").collect().foreach(println)
[Found 2 WholeStageCodegen subtrees.]
[== Subtree 1 / 2 ==]
...
[]
[Generated code:]
...
[]
[== Subtree 2 / 2 ==]
...
[]
[Generated code:]
...
```

## How was this patch tested?

Pass the Jenkins tests (including new testcases)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12099 from dongjoon-hyun/SPARK-14251.
2016-04-01 22:45:52 -07:00
Cheng Lian 27e71a2cd9 [SPARK-14244][SQL] Don't use SizeBasedWindowFunction.n created on executor side when evaluating window functions
## What changes were proposed in this pull request?

`SizeBasedWindowFunction.n` is a global singleton attribute created for evaluating size based aggregate window functions like `CUME_DIST`. However, this attribute gets different expression IDs when created on both driver side and executor side. This PR adds `withPartitionSize` method to `SizeBasedWindowFunction` so that we can easily rewrite `SizeBasedWindowFunction.n` on executor side.

## How was this patch tested?

A test case is added in `HiveSparkSubmitSuite`, which supports launching multi-process clusters.

Author: Cheng Lian <lian@databricks.com>

Closes #12040 from liancheng/spark-14244-fix-sized-window-function.
2016-04-01 22:00:24 -07:00
Michael Armbrust 0fc4aaa71c [SPARK-14255][SQL] Streaming Aggregation
This PR adds the ability to perform aggregations inside of a `ContinuousQuery`.  In order to implement this feature, the planning of aggregation has augmented with a new `StatefulAggregationStrategy`.  Unlike batch aggregation, stateful-aggregation uses the `StateStore` (introduced in #11645) to persist the results of partial aggregation across different invocations.  The resulting physical plan performs the aggregation using the following progression:
   - Partial Aggregation
   - Shuffle
   - Partial Merge (now there is at most 1 tuple per group)
   - StateStoreRestore (now there is 1 tuple from this batch + optionally one from the previous)
   - Partial Merge (now there is at most 1 tuple per group)
   - StateStoreSave (saves the tuple for the next batch)
   - Complete (output the current result of the aggregation)

The following refactoring was also performed to allow us to plug into existing code:
 - The get/put implementation is taken from #12013
 - The logic for breaking down and de-duping the physical execution of aggregation has been move into a new pattern `PhysicalAggregation`
 - The `AttributeReference` used to identify the result of an `AggregateFunction` as been moved into the `AggregateExpression` container.  This change moves the reference into the same object as the other intermediate references used in aggregation and eliminates the need to pass around a `Map[(AggregateFunction, Boolean), Attribute]`.  Further clean up (using a different aggregation container for logical/physical plans) is deferred to a followup.
 - Some planning logic is moved from the `SessionState` into the `QueryExecution` to make it easier to override in the streaming case.
 - The ability to write a `StreamTest` that checks only the output of the last batch has been added to simulate the future addition of output modes.

Author: Michael Armbrust <michael@databricks.com>

Closes #12048 from marmbrus/statefulAgg.
2016-04-01 15:15:16 -07:00
Burak Yavuz 1b829ce139 [SPARK-14160] Time Windowing functions for Datasets
## What changes were proposed in this pull request?

This PR adds the function `window` as a column expression.

`window` can be used to bucket rows into time windows given a time column. With this expression, performing time series analysis on batch data, as well as streaming data should become much more simpler.

### Usage

Assume the following schema:

`sensor_id, measurement, timestamp`

To average 5 minute data every 1 minute (window length of 5 minutes, slide duration of 1 minute), we will use:
```scala
df.groupBy(window("timestamp", “5 minutes”, “1 minute”), "sensor_id")
  .agg(mean("measurement").as("avg_meas"))
```

This will generate windows such as:
```
09:00:00-09:05:00
09:01:00-09:06:00
09:02:00-09:07:00 ...
```

Intervals will start at every `slideDuration` starting at the unix epoch (1970-01-01 00:00:00 UTC).
To start intervals at a different point of time, e.g. 30 seconds after a minute, the `startTime` parameter can be used.

```scala
df.groupBy(window("timestamp", “5 minutes”, “1 minute”, "30 second"), "sensor_id")
  .agg(mean("measurement").as("avg_meas"))
```

This will generate windows such as:
```
09:00:30-09:05:30
09:01:30-09:06:30
09:02:30-09:07:30 ...
```

Support for Python will be made in a follow up PR after this.

## How was this patch tested?

This patch has some basic unit tests for the `TimeWindow` expression testing that the parameters pass validation, and it also has some unit/integration tests testing the correctness of the windowing and usability in complex operations (multi-column grouping, multi-column projections, joins).

Author: Burak Yavuz <brkyvz@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #12008 from brkyvz/df-time-window.
2016-04-01 13:19:24 -07:00
Liang-Chi Hsieh a884daad80 [SPARK-14191][SQL] Remove invalid Expand operator constraints
`Expand` operator now uses its child plan's constraints as its valid constraints (i.e., the base of constraints). This is not correct because `Expand` will set its group by attributes to null values. So the nullability of these attributes should be true.

E.g., for an `Expand` operator like:

    val input = LocalRelation('a.int, 'b.int, 'c.int).where('c.attr > 10 && 'a.attr < 5 && 'b.attr > 2)
    Expand(
      Seq(
        Seq('c, Literal.create(null, StringType), 1),
        Seq('c, 'a, 2)),
      Seq('c, 'a, 'gid.int),
      Project(Seq('a, 'c), input))

The `Project` operator has the constraints `IsNotNull('a)`, `IsNotNull('b)` and `IsNotNull('c)`. But the `Expand` should not have `IsNotNull('a)` in its constraints.

This PR is the first step for this issue and remove invalid constraints of `Expand` operator.

A test is added to `ConstraintPropagationSuite`.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #11995 from viirya/fix-expand-constraints.
2016-04-01 13:08:09 -07:00
Liang-Chi Hsieh df68beb85d [SPARK-13995][SQL] Extract correct IsNotNull constraints for Expression
## What changes were proposed in this pull request?

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

We infer relative `IsNotNull` constraints from logical plan's expressions in `constructIsNotNullConstraints` now. However, we don't consider the case of (nested) `Cast`.

For example:

    val tr = LocalRelation('a.int, 'b.long)
    val plan = tr.where('a.attr === 'b.attr).analyze

Then, the plan's constraints will have `IsNotNull(Cast(resolveColumn(tr, "a"), LongType))`, instead of `IsNotNull(resolveColumn(tr, "a"))`. This PR fixes it.

Besides, as `IsNotNull` constraints are most useful for `Attribute`, we should do recursing through any `Expression` that is null intolerant and construct `IsNotNull` constraints for all `Attribute`s under these Expressions.

For example, consider the following constraints:

    val df = Seq((1,2,3)).toDF("a", "b", "c")
    df.where("a + b = c").queryExecution.analyzed.constraints

The inferred isnotnull constraints should be isnotnull(a), isnotnull(b), isnotnull(c), instead of isnotnull(a + c) and isnotnull(c).

## How was this patch tested?

Test is added into `ConstraintPropagationSuite`.

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

Closes #11809 from viirya/constraint-cast.
2016-04-01 13:00:55 -07:00
sureshthalamati a471c7f9ea [SPARK-14133][SQL] Throws exception for unsupported create/drop/alter index , and lock/unlock operations.
## What changes were proposed in this pull request?

This  PR  throws Unsupported Operation exception for create index, drop index, alter index , lock table , lock database, unlock table, and unlock database operations that are not supported in Spark SQL. Currently these operations are executed executed by Hive.

Error:
spark-sql> drop index my_index on my_table;
Error in query:
Unsupported operation: drop index(line 1, pos 0)

## How was this patch tested?
Added test cases to HiveQuerySuite

yhuai hvanhovell andrewor14

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

Closes #12069 from sureshthalamati/unsupported_ddl_spark-14133.
2016-04-01 18:33:31 +02:00
Dilip Biswal 0b04f8fdf1 [SPARK-14184][SQL] Support native execution of SHOW DATABASE command and fix SHOW TABLE to use table identifier pattern
## What changes were proposed in this pull request?

This PR addresses the following

1. Supports native execution of SHOW DATABASES command
2. Fixes SHOW TABLES to apply the identifier_with_wildcards pattern if supplied.

SHOW TABLE syntax
```
SHOW TABLES [IN database_name] ['identifier_with_wildcards'];
```
SHOW DATABASES syntax
```
SHOW (DATABASES|SCHEMAS) [LIKE 'identifier_with_wildcards'];
```

## How was this patch tested?
Tests added in SQLQuerySuite (both hive and sql contexts) and DDLCommandSuite

Note: Since the table name pattern was not working , tests are added in both SQLQuerySuite to
verify the application of the table pattern.

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

Closes #11991 from dilipbiswal/dkb_show_database.
2016-04-01 18:27:11 +02:00
gatorsmile 446c45bd87 [SPARK-14182][SQL] Parse DDL Command: Alter View
This PR is to provide native parsing support for DDL commands: `Alter View`. Since its AST trees are highly similar to `Alter Table`. Thus, both implementation are integrated into the same one.

Based on the Hive DDL document:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL and https://cwiki.apache.org/confluence/display/Hive/PartitionedViews

**Syntax:**
```SQL
ALTER VIEW view_name RENAME TO new_view_name
```
 - to change the name of a view to a different name

**Syntax:**
```SQL
ALTER VIEW view_name SET TBLPROPERTIES ('comment' = new_comment);
```
 - to add metadata to a view

**Syntax:**
```SQL
ALTER VIEW view_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key')
```
 - to remove metadata from a view

**Syntax:**
```SQL
ALTER VIEW view_name ADD [IF NOT EXISTS] PARTITION spec1[, PARTITION spec2, ...]
```
 - to add the partitioning metadata for a view.
 - the syntax of partition spec in `ALTER VIEW` is identical to `ALTER TABLE`, **EXCEPT** that it is **ILLEGAL** to specify a `LOCATION` clause.

**Syntax:**
```SQL
ALTER VIEW view_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...]
```
 - to drop the related partition metadata for a view.

Added the related test cases to `DDLCommandSuite`

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

Closes #11987 from gatorsmile/parseAlterView.
2016-03-31 12:04:03 -07:00
Herman van Hovell a9b93e0739 [SPARK-14211][SQL] Remove ANTLR3 based parser
### What changes were proposed in this pull request?

This PR removes the ANTLR3 based parser, and moves the new ANTLR4 based parser into the `org.apache.spark.sql.catalyst.parser package`.

### How was this patch tested?

Existing unit tests.

cc rxin andrewor14 yhuai

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

Closes #12071 from hvanhovell/SPARK-14211.
2016-03-31 09:25:09 -07:00
Dongjoon Hyun 258a243419 [SPARK-14282][SQL] CodeFormatter should handle oneline comment with /* */ properly
## What changes were proposed in this pull request?

This PR improves `CodeFormatter` to fix the following malformed indentations.
```java
/* 019 */   public java.lang.Object apply(java.lang.Object _i) {
/* 020 */     InternalRow i = (InternalRow) _i;
/* 021 */     /* createexternalrow(if (isnull(input[0, double])) null else input[0, double], if (isnull(input[1, int])) null else input[1, int], ... */
/* 022 */       boolean isNull = false;
/* 023 */       final Object[] values = new Object[2];
/* 024 */       /* if (isnull(input[0, double])) null else input[0, double] */
/* 025 */     /* isnull(input[0, double]) */
...
/* 053 */     if (!false && false) {
/* 054 */       /* null */
/* 055 */     final int value9 = -1;
/* 056 */     isNull6 = true;
/* 057 */     value6 = value9;
/* 058 */   } else {
...
/* 077 */   return mutableRow;
/* 078 */ }
/* 079 */ }
/* 080 */
```

After this PR, the code will be formatted like the following.
```java
/* 019 */   public java.lang.Object apply(java.lang.Object _i) {
/* 020 */     InternalRow i = (InternalRow) _i;
/* 021 */     /* createexternalrow(if (isnull(input[0, double])) null else input[0, double], if (isnull(input[1, int])) null else input[1, int], ... */
/* 022 */     boolean isNull = false;
/* 023 */     final Object[] values = new Object[2];
/* 024 */     /* if (isnull(input[0, double])) null else input[0, double] */
/* 025 */     /* isnull(input[0, double]) */
...
/* 053 */     if (!false && false) {
/* 054 */       /* null */
/* 055 */       final int value9 = -1;
/* 056 */       isNull6 = true;
/* 057 */       value6 = value9;
/* 058 */     } else {
...
/* 077 */     return mutableRow;
/* 078 */   }
/* 079 */ }
/* 080 */
```

Also, this issue fixes the following too. (Similar with [SPARK-14185](https://issues.apache.org/jira/browse/SPARK-14185))
```java
16/03/30 12:39:24 DEBUG WholeStageCodegen: /* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
```
```java
16/03/30 12:46:32 DEBUG WholeStageCodegen:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
```

## How was this patch tested?

Pass the Jenkins tests (including new CodeFormatterSuite testcases.)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12072 from dongjoon-hyun/SPARK-14282.
2016-03-30 16:15:37 -07:00
Wenchen Fan d46c71b39d [SPARK-14268][SQL] rename toRowExpressions and fromRowExpression to serializer and deserializer in ExpressionEncoder
## What changes were proposed in this pull request?

In `ExpressionEncoder`, we use `constructorFor` to build `fromRowExpression` as the `deserializer` in `ObjectOperator`. It's kind of confusing, we should make the name consistent.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12058 from cloud-fan/rename.
2016-03-30 11:03:15 -07:00
gatorsmile b66b97cd04 [SPARK-14124][SQL] Implement Database-related DDL Commands
#### What changes were proposed in this pull request?
This PR is to implement the following four Database-related DDL commands:
 - `CREATE DATABASE|SCHEMA [IF NOT EXISTS] database_name`
 - `DROP DATABASE [IF EXISTS] database_name [RESTRICT|CASCADE]`
 - `DESCRIBE DATABASE [EXTENDED] db_name`
 - `ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...)`

Another PR will be submitted to handle the unsupported commands. In the Database-related DDL commands, we will issue an error exception for `ALTER (DATABASE|SCHEMA) database_name SET OWNER [USER|ROLE] user_or_role`.

cc yhuai andrewor14 rxin Could you review the changes? Is it in the right direction? Thanks!

#### How was this patch tested?
Added a few test cases in `command/DDLSuite.scala` for testing DDL command execution in `SQLContext`. Since `HiveContext` also shares the same implementation, the existing test cases in `\hive` also verifies the correctness of these commands.

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

Closes #12009 from gatorsmile/dbDDL.
2016-03-29 17:39:52 -07:00
Sameer Agarwal 366cac6fb0 [SPARK-14225][SQL] Cap the length of toCommentSafeString at 128 chars
## What changes were proposed in this pull request?

Builds on https://github.com/apache/spark/pull/12022 and (a) appends "..." to truncated comment strings and (b) fixes indentation in lines after the commented strings if they happen to have a `(`, `{`, `)` or `}`

## How was this patch tested?

Manually examined the generated code.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12044 from sameeragarwal/comment.
2016-03-29 16:46:45 -07:00
Dongjoon Hyun d612228eff [MINOR][SQL] Fix typos by replacing 'much' with 'match'.
## What changes were proposed in this pull request?

This PR fixes two trivial typos: 'does not **much**' --> 'does not **match**'.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12042 from dongjoon-hyun/fix_typo_by_replacing_much_with_match.
2016-03-29 12:45:43 -07:00
Herman van Hovell 27d4ef0c61 [SPARK-14213][SQL] Migrate HiveQl parsing to ANTLR4 parser
### What changes were proposed in this pull request?

This PR migrates all HiveQl parsing to the new ANTLR4 parser. This PR is build on top of https://github.com/apache/spark/pull/12011, and we should wait with merging until that one is in (hence the WIP tag).

As soon as this PR is merged we can start removing much of the old parser infrastructure.

### How was this patch tested?

Exisiting Hive unit tests.

cc rxin andrewor14 yhuai

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

Closes #12015 from hvanhovell/SPARK-14213.
2016-03-28 20:19:21 -07:00
Andrew Or 27aab80695 [SPARK-14013][SQL] Proper temp function support in catalog
## What changes were proposed in this pull request?

Session catalog was added in #11750. However, it doesn't really support temporary functions properly; right now we only store the metadata in the form of `CatalogFunction`, but this doesn't make sense for temporary functions because there is no class name.

This patch moves the `FunctionRegistry` into the `SessionCatalog`. With this, the user can call `catalog.createTempFunction` and `catalog.lookupFunction` to use the function they registered previously. This is currently still dead code, however.

## How was this patch tested?

`SessionCatalogSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #11972 from andrewor14/temp-functions.
2016-03-28 16:45:02 -07:00
Reynold Xin b7836492bb [SPARK-14155][SQL] Hide UserDefinedType interface in Spark 2.0
## What changes were proposed in this pull request?
UserDefinedType is a developer API in Spark 1.x. With very high probability we will create a new API for user-defined type that also works well with column batches as well as encoders (datasets). In Spark 2.0, let's make `UserDefinedType` `private[spark]` first.

## How was this patch tested?
Existing unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11955 from rxin/SPARK-14155.
2016-03-28 16:26:32 -07:00
Andrew Or eebc8c1c95 [SPARK-13923][SPARK-14014][SQL] Session catalog follow-ups
## What changes were proposed in this pull request?

This patch addresses the remaining comments left in #11750 and #11918 after they are merged. For a full list of changes in this patch, just trace the commits.

## How was this patch tested?

`SessionCatalogSuite` and `CatalogTestCases`

Author: Andrew Or <andrew@databricks.com>

Closes #12006 from andrewor14/session-catalog-followup.
2016-03-28 16:25:15 -07:00
Herman van Hovell 600c0b69ca [SPARK-13713][SQL] Migrate parser from ANTLR3 to ANTLR4
### What changes were proposed in this pull request?
The current ANTLR3 parser is quite complex to maintain and suffers from code blow-ups. This PR introduces a new parser that is based on ANTLR4.

This parser is based on the [Presto's SQL parser](https://github.com/facebook/presto/blob/master/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4). The current implementation can parse and create Catalyst and SQL plans. Large parts of the HiveQl DDL and some of the DML functionality is currently missing, the plan is to add this in follow-up PRs.

This PR is a work in progress, and work needs to be done in the following area's:

- [x] Error handling should be improved.
- [x] Documentation should be improved.
- [x] Multi-Insert needs to be tested.
- [ ] Naming and package locations.

### How was this patch tested?

Catalyst and SQL unit tests.

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

Closes #11557 from hvanhovell/ngParser.
2016-03-28 12:31:12 -07:00
Kazuaki Ishizaki 4a7636f2da [SPARK-13844] [SQL] Generate better code for filters with a non-nullable column
## What changes were proposed in this pull request?

This PR simplifies generated code with a non-nullable column. This PR addresses three items:
1. Generate simplified code for and / or
2. Generate better code for divide and remainder with non-zero dividend
3. Pass nullable information into BoundReference at WholeStageCodegen

I have attached the generated code with and without this PR

## How was this patch tested?

Tested by existing test suites in sql/core

Here is a motivating example
````
(0 to 6).map(i => (i.toString, i.toInt)).toDF("k", "v")
  .filter("v % 2 == 0").filter("v <= 4").filter("v > 1").show()
````

Generated code without this PR
````java
/* 032 */   protected void processNext() throws java.io.IOException {
/* 033 */     /*** PRODUCE: Project [_1#0 AS k#3,_2#1 AS v#4] */
/* 034 */
/* 035 */     /*** PRODUCE: Filter ((isnotnull((_2#1 % 2)) && ((_2#1 % 2) = 0)) && ((_2#1 <= 4) && (_2#1 > 1))) */
/* 036 */
/* 037 */     /*** PRODUCE: INPUT */
/* 038 */
/* 039 */     while (!shouldStop() && inputadapter_input.hasNext()) {
/* 040 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 041 */       /*** CONSUME: Filter ((isnotnull((_2#1 % 2)) && ((_2#1 % 2) = 0)) && ((_2#1 <= 4) && (_2#1 > 1))) */
/* 042 */       /* input[1, int] */
/* 043 */       int filter_value1 = inputadapter_row.getInt(1);
/* 044 */
/* 045 */       /* isnotnull((input[1, int] % 2)) */
/* 046 */       /* (input[1, int] % 2) */
/* 047 */       boolean filter_isNull3 = false;
/* 048 */       int filter_value3 = -1;
/* 049 */       if (false || 2 == 0) {
/* 050 */         filter_isNull3 = true;
/* 051 */       } else {
/* 052 */         if (false) {
/* 053 */           filter_isNull3 = true;
/* 054 */         } else {
/* 055 */           filter_value3 = (int)(filter_value1 % 2);
/* 056 */         }
/* 057 */       }
/* 058 */       if (!(!(filter_isNull3))) continue;
/* 059 */
/* 060 */       /* ((input[1, int] % 2) = 0) */
/* 061 */       boolean filter_isNull6 = true;
/* 062 */       boolean filter_value6 = false;
/* 063 */       /* (input[1, int] % 2) */
/* 064 */       boolean filter_isNull7 = false;
/* 065 */       int filter_value7 = -1;
/* 066 */       if (false || 2 == 0) {
/* 067 */         filter_isNull7 = true;
/* 068 */       } else {
/* 069 */         if (false) {
/* 070 */           filter_isNull7 = true;
/* 071 */         } else {
/* 072 */           filter_value7 = (int)(filter_value1 % 2);
/* 073 */         }
/* 074 */       }
/* 075 */       if (!filter_isNull7) {
/* 076 */         filter_isNull6 = false; // resultCode could change nullability.
/* 077 */         filter_value6 = filter_value7 == 0;
/* 078 */
/* 079 */       }
/* 080 */       if (filter_isNull6 || !filter_value6) continue;
/* 081 */
/* 082 */       /* (input[1, int] <= 4) */
/* 083 */       boolean filter_value11 = false;
/* 084 */       filter_value11 = filter_value1 <= 4;
/* 085 */       if (!filter_value11) continue;
/* 086 */
/* 087 */       /* (input[1, int] > 1) */
/* 088 */       boolean filter_value14 = false;
/* 089 */       filter_value14 = filter_value1 > 1;
/* 090 */       if (!filter_value14) continue;
/* 091 */
/* 092 */       filter_metricValue.add(1);
/* 093 */
/* 094 */       /*** CONSUME: Project [_1#0 AS k#3,_2#1 AS v#4] */
/* 095 */
/* 096 */       /* input[0, string] */
/* 097 */       /* input[0, string] */
/* 098 */       boolean filter_isNull = inputadapter_row.isNullAt(0);
/* 099 */       UTF8String filter_value = filter_isNull ? null : (inputadapter_row.getUTF8String(0));
/* 100 */       project_holder.reset();
/* 101 */
/* 102 */       project_rowWriter.zeroOutNullBytes();
/* 103 */
/* 104 */       if (filter_isNull) {
/* 105 */         project_rowWriter.setNullAt(0);
/* 106 */       } else {
/* 107 */         project_rowWriter.write(0, filter_value);
/* 108 */       }
/* 109 */
/* 110 */       project_rowWriter.write(1, filter_value1);
/* 111 */       project_result.setTotalSize(project_holder.totalSize());
/* 112 */       append(project_result.copy());
/* 113 */     }
/* 114 */   }
/* 115 */ }
````

Generated code with this PR
````java
/* 032 */   protected void processNext() throws java.io.IOException {
/* 033 */     /*** PRODUCE: Project [_1#0 AS k#3,_2#1 AS v#4] */
/* 034 */
/* 035 */     /*** PRODUCE: Filter (((_2#1 % 2) = 0) && ((_2#1 <= 5) && (_2#1 > 1))) */
/* 036 */
/* 037 */     /*** PRODUCE: INPUT */
/* 038 */
/* 039 */     while (!shouldStop() && inputadapter_input.hasNext()) {
/* 040 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 041 */       /*** CONSUME: Filter (((_2#1 % 2) = 0) && ((_2#1 <= 5) && (_2#1 > 1))) */
/* 042 */       /* input[1, int] */
/* 043 */       int filter_value1 = inputadapter_row.getInt(1);
/* 044 */
/* 045 */       /* ((input[1, int] % 2) = 0) */
/* 046 */       /* (input[1, int] % 2) */
/* 047 */       int filter_value3 = (int)(filter_value1 % 2);
/* 048 */
/* 049 */       boolean filter_value2 = false;
/* 050 */       filter_value2 = filter_value3 == 0;
/* 051 */       if (!filter_value2) continue;
/* 052 */
/* 053 */       /* (input[1, int] <= 5) */
/* 054 */       boolean filter_value7 = false;
/* 055 */       filter_value7 = filter_value1 <= 5;
/* 056 */       if (!filter_value7) continue;
/* 057 */
/* 058 */       /* (input[1, int] > 1) */
/* 059 */       boolean filter_value10 = false;
/* 060 */       filter_value10 = filter_value1 > 1;
/* 061 */       if (!filter_value10) continue;
/* 062 */
/* 063 */       filter_metricValue.add(1);
/* 064 */
/* 065 */       /*** CONSUME: Project [_1#0 AS k#3,_2#1 AS v#4] */
/* 066 */
/* 067 */       /* input[0, string] */
/* 068 */       /* input[0, string] */
/* 069 */       boolean filter_isNull = inputadapter_row.isNullAt(0);
/* 070 */       UTF8String filter_value = filter_isNull ? null : (inputadapter_row.getUTF8String(0));
/* 071 */       project_holder.reset();
/* 072 */
/* 073 */       project_rowWriter.zeroOutNullBytes();
/* 074 */
/* 075 */       if (filter_isNull) {
/* 076 */         project_rowWriter.setNullAt(0);
/* 077 */       } else {
/* 078 */         project_rowWriter.write(0, filter_value);
/* 079 */       }
/* 080 */
/* 081 */       project_rowWriter.write(1, filter_value1);
/* 082 */       project_result.setTotalSize(project_holder.totalSize());
/* 083 */       append(project_result.copy());
/* 084 */     }
/* 085 */   }
/* 086 */ }
````

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

Closes #11684 from kiszk/SPARK-13844.
2016-03-28 10:35:48 -07:00
Kousuke Saruta aac13fb48c [SPARK-14185][SQL][MINOR] Make indentation of debug log for generated code proper
## What changes were proposed in this pull request?

The indentation of debug log output by `CodeGenerator` is weird.
The first line of the generated code should be put on the next line of the first line of the log message.

```
16/03/28 11:10:24 DEBUG CodeGenerator: /* 001 */
/* 002 */ public java.lang.Object generate(Object[] references) {
/* 003 */   return new SpecificSafeProjection(references);
...
```

After this patch is applied, we get debug log like as follows.

```
16/03/28 10:45:50 DEBUG CodeGenerator:
/* 001 */
/* 002 */ public java.lang.Object generate(Object[] references) {
/* 003 */   return new SpecificSafeProjection(references);
...
```
## How was this patch tested?

Ran some jobs and checked debug logs.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #11990 from sarutak/fix-debuglog-indentation.
2016-03-27 23:50:23 -07:00
Dongjoon Hyun 1808465855 [MINOR] Fix newly added java-lint errors
## What changes were proposed in this pull request?

This PR fixes some newly added java-lint errors(unused-imports, line-lengsth).

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11968 from dongjoon-hyun/SPARK-14167.
2016-03-26 11:55:49 +00:00
Sameer Agarwal afd0debe07 [SPARK-14137] [SPARK-14150] [SQL] Infer IsNotNull constraints from non-nullable attributes
## What changes were proposed in this pull request?

This PR adds support for automatically inferring `IsNotNull` constraints from any non-nullable attributes that are part of an operator's output. This also fixes the issue that causes the optimizer to hit the maximum number of iterations for certain queries in https://github.com/apache/spark/pull/11828.

## How was this patch tested?

Unit test in `ConstraintPropagationSuite`

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11953 from sameeragarwal/infer-isnotnull.
2016-03-25 12:57:26 -07:00
Liang-Chi Hsieh ca003354da [SPARK-12443][SQL] encoderFor should support Decimal
## What changes were proposed in this pull request?

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

`constructorFor` will call `dataTypeFor` to determine if a type is `ObjectType` or not. If there is not case for `Decimal`, it will be recognized as `ObjectType` and causes the bug.

## How was this patch tested?

Test is added into `ExpressionEncoderSuite`.

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

Closes #10399 from viirya/fix-encoder-decimal.
2016-03-25 12:07:56 -07:00
Wenchen Fan 43b15e01c4 [SPARK-14061][SQL] implement CreateMap
## What changes were proposed in this pull request?

As we have `CreateArray` and `CreateStruct`, we should also have `CreateMap`.  This PR adds the `CreateMap` expression, and the DataFrame API, and python API.

## How was this patch tested?

various new tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11879 from cloud-fan/create_map.
2016-03-25 09:50:06 -07:00
Davies Liu 6603d9f7e2 [SPARK-13919] [SQL] fix column pruning through filter
## What changes were proposed in this pull request?

This PR fix the conflict between ColumnPruning and PushPredicatesThroughProject, because ColumnPruning will try to insert a Project before Filter, but PushPredicatesThroughProject will move the Filter before Project.This is fixed by remove the Project before Filter, if the Project only do column pruning.

The RuleExecutor will fail the test if reached max iterations.

Closes #11745

## How was this patch tested?

Existing tests.

This is a test case still failing, disabled for now, will be fixed by https://issues.apache.org/jira/browse/SPARK-14137

Author: Davies Liu <davies@databricks.com>

Closes #11828 from davies/fail_rule.
2016-03-25 09:05:23 -07:00
Wenchen Fan e9b6e7d857 [SPARK-13456][SQL][FOLLOW-UP] lazily generate the outer pointer for case class defined in REPL
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/11410, we missed a corner case: define the inner class and use it in `Dataset` at the same time by using paste mode. For this case, the inner class and the `Dataset` are inside same line object, when we build the `Dataset`, we try to get outer pointer from line object, and it will fail because the line object is not initialized yet.

https://issues.apache.org/jira/browse/SPARK-13456?focusedCommentId=15209174&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15209174 is an example for this corner case.

This PR make the process of getting outer pointer from line object lazy, so that we can successfully build the `Dataset` and finish initializing the line object.

## How was this patch tested?

new test in repl suite.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11931 from cloud-fan/repl.
2016-03-25 20:19:04 +08:00
Andrew Or 20ddf5fddf [SPARK-14014][SQL] Integrate session catalog (attempt #2)
## What changes were proposed in this pull request?

This reopens #11836, which was merged but promptly reverted because it introduced flaky Hive tests.

## How was this patch tested?

See `CatalogTestCases`, `SessionCatalogSuite` and `HiveContextSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #11938 from andrewor14/session-catalog-again.
2016-03-24 22:59:35 -07:00
Reynold Xin 3619fec1ec [SPARK-14142][SQL] Replace internal use of unionAll with union
## What changes were proposed in this pull request?
unionAll has been deprecated in SPARK-14088.

## How was this patch tested?
Should be covered by all existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11946 from rxin/SPARK-14142.
2016-03-24 22:34:55 -07:00
gatorsmile 05f652d6c2 [SPARK-13957][SQL] Support Group By Ordinal in SQL
#### What changes were proposed in this pull request?
This PR is to support group by position in SQL. For example, when users input the following query
```SQL
select c1 as a, c2, c3, sum(*) from tbl group by 1, 3, c4
```
The ordinals are recognized as the positions in the select list. Thus, `Analyzer` converts it to
```SQL
select c1, c2, c3, sum(*) from tbl group by c1, c3, c4
```

This is controlled by the config option `spark.sql.groupByOrdinal`.
- When true, the ordinal numbers in group by clauses are treated as the position in the select list.
- When false, the ordinal numbers are ignored.
- Only convert integer literals (not foldable expressions). If found foldable expressions, ignore them.
- When the positions specified in the group by clauses correspond to the aggregate functions in select list, output an exception message.
- star is not allowed to use in the select list when users specify ordinals in group by

Note: This PR is taken from https://github.com/apache/spark/pull/10731. When merging this PR, please give the credit to zhichao-li

Also cc all the people who are involved in the previous discussion:  rxin cloud-fan marmbrus yhuai hvanhovell adrian-wang chenghao-intel tejasapatil

#### How was this patch tested?

Added a few test cases for both positive and negative test cases.

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

Closes #11846 from gatorsmile/groupByOrdinal.
2016-03-25 12:55:58 +08:00
Andrew Or c44d140cae Revert "[SPARK-14014][SQL] Replace existing catalog with SessionCatalog"
This reverts commit 5dfc01976b.
2016-03-23 22:21:15 -07:00
gatorsmile f42eaf42bd [SPARK-14085][SQL] Star Expansion for Hash
#### What changes were proposed in this pull request?

This PR is to support star expansion in hash. For example,
```SQL
val structDf = testData2.select("a", "b").as("record")
structDf.select(hash($"*")
```

In addition, it refactors the codes for the rule `ResolveStar` and fixes a regression for star expansion in group by when using SQL API. For example,
```SQL
SELECT * FROM testData2 group by a, b
```

cc cloud-fan Now, the code for star resolution is much cleaner. The coverage is better. Could you check if this refactoring is good? Thanks!

#### How was this patch tested?
Added a few test cases to cover it.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11904 from gatorsmile/starResolution.
2016-03-24 11:13:36 +08:00
Andrew Or 5dfc01976b [SPARK-14014][SQL] Replace existing catalog with SessionCatalog
## What changes were proposed in this pull request?

`SessionCatalog`, introduced in #11750, is a catalog that keeps track of temporary functions and tables, and delegates metastore operations to `ExternalCatalog`. This functionality overlaps a lot with the existing `analysis.Catalog`.

As of this commit, `SessionCatalog` and `ExternalCatalog` will no longer be dead code. There are still things that need to be done after this patch, namely:
- SPARK-14013: Properly implement temporary functions in `SessionCatalog`
- SPARK-13879: Decide which DDL/DML commands to support natively in Spark
- SPARK-?????: Implement the ones we do want to support through `SessionCatalog`.
- SPARK-?????: Merge SQL/HiveContext

## How was this patch tested?

This is largely a refactoring task so there are no new tests introduced. The particularly relevant tests are `SessionCatalogSuite` and `ExternalCatalogSuite`.

Author: Andrew Or <andrew@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #11836 from andrewor14/use-session-catalog.
2016-03-23 13:34:22 -07:00
Herman van Hovell 919bf32198 [SPARK-13325][SQL] Create a 64-bit hashcode expression
This PR introduces a 64-bit hashcode expression. Such an expression is especially usefull for HyperLogLog++ and other probabilistic datastructures.

I have implemented xxHash64 which is a 64-bit hashing algorithm created by Yann Colet and Mathias Westerdahl. This is a high speed (C implementation runs at memory bandwidth) and high quality hashcode. It exploits both Instruction Level Parralellism (for speed) and the multiplication and rotation techniques (for quality) like MurMurHash does.

The initial results are promising. I have added a CG'ed test to the `HashBenchmark`, and this results in the following results (running from SBT):

    Running benchmark: Hash For simple
      Running case: interpreted version
      Running case: codegen version
      Running case: codegen version 64-bit

    Intel(R) Core(TM) i7-4750HQ CPU  2.00GHz
    Hash For simple:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    interpreted version                      1011 / 1016        132.8           7.5       1.0X
    codegen version                          1864 / 1869         72.0          13.9       0.5X
    codegen version 64-bit                   1614 / 1644         83.2          12.0       0.6X

    Running benchmark: Hash For normal
      Running case: interpreted version
      Running case: codegen version
      Running case: codegen version 64-bit

    Intel(R) Core(TM) i7-4750HQ CPU  2.00GHz
    Hash For normal:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    interpreted version                      2467 / 2475          0.9        1176.1       1.0X
    codegen version                          2008 / 2115          1.0         957.5       1.2X
    codegen version 64-bit                    728 /  758          2.9         347.0       3.4X

    Running benchmark: Hash For array
      Running case: interpreted version
      Running case: codegen version
      Running case: codegen version 64-bit

    Intel(R) Core(TM) i7-4750HQ CPU  2.00GHz
    Hash For array:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    interpreted version                      1544 / 1707          0.1       11779.6       1.0X
    codegen version                          2728 / 2745          0.0       20815.5       0.6X
    codegen version 64-bit                   2508 / 2549          0.1       19132.8       0.6X

    Running benchmark: Hash For map
      Running case: interpreted version
      Running case: codegen version
      Running case: codegen version 64-bit

    Intel(R) Core(TM) i7-4750HQ CPU  2.00GHz
    Hash For map:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    interpreted version                      1819 / 1826          0.0      444014.3       1.0X
    codegen version                           183 /  194          0.0       44642.9       9.9X
    codegen version 64-bit                    173 /  174          0.0       42120.9      10.5X

This shows that algorithm is consistently faster than MurMurHash32 in all cases and up to 3x (!) in the normal case.

I have also added this to HyperLogLog++ and it cuts the processing time of the following code in half:

    val df = sqlContext.range(1<<25).agg(approxCountDistinct("id"))
    df.explain()
    val t = System.nanoTime()
    df.show()
    val ns = System.nanoTime() - t

    // Before
    ns: Long = 5821524302

    // After
    ns: Long = 2836418963

cc cloud-fan (you have been working on hashcodes) / rxin

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

Closes #11209 from hvanhovell/xxHash.
2016-03-23 20:51:01 +01:00
Josh Rosen 3de24ae2ed [SPARK-14075] Refactor MemoryStore to be testable independent of BlockManager
This patch refactors the `MemoryStore` so that it can be tested without needing to construct / mock an entire `BlockManager`.

- The block manager's serialization- and compression-related methods have been moved from `BlockManager` to `SerializerManager`.
- `BlockInfoManager `is now passed directly to classes that need it, rather than being passed via the `BlockManager`.
- The `MemoryStore` now calls `dropFromMemory` via a new `BlockEvictionHandler` interface rather than directly calling the `BlockManager`. This change helps to enforce a narrow interface between the `MemoryStore` and `BlockManager` functionality and makes this interface easier to mock in tests.
- Several of the block unrolling tests have been moved from `BlockManagerSuite` into a new `MemoryStoreSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11899 from JoshRosen/reduce-memorystore-blockmanager-coupling.
2016-03-23 10:15:23 -07:00
gatorsmile 6ce008ba46 [SPARK-13549][SQL] Refactor the Optimizer Rule CollapseProject
#### What changes were proposed in this pull request?

The PR https://github.com/apache/spark/pull/10541 changed the rule `CollapseProject` by enabling collapsing `Project` into `Aggregate`. It leaves a to-do item to remove the duplicate code. This PR is to finish this to-do item. Also added a test case for covering this change.

#### How was this patch tested?

Added a new test case.

liancheng Could you check if the code refactoring is fine? Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11427 from gatorsmile/collapseProjectRefactor.
2016-03-24 00:51:31 +08:00
Dongjoon Hyun 1a22cf1e9b [MINOR][SQL][DOCS] Update sql/README.md and remove some unused imports in sql module.
## What changes were proposed in this pull request?

This PR updates `sql/README.md` according to the latest console output and removes some unused imports in `sql` module. This is done by manually, so there is no guarantee to remove all unused imports.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11907 from dongjoon-hyun/update_sql_module.
2016-03-22 23:07:49 -07:00
Davies Liu 4700adb98e [SPARK-13806] [SQL] fix rounding mode of negative float/double
## What changes were proposed in this pull request?

Round() in database usually round the number up (away from zero), it's different than Math.round() in Java.

For example:
```
scala> java.lang.Math.round(-3.5)
res3: Long = -3
```
In Database, we should return -4.0 in this cases.

This PR remove the buggy special case for scale=0.

## How was this patch tested?

Add tests for negative values with tie.

Author: Davies Liu <davies@databricks.com>

Closes #11894 from davies/fix_round.
2016-03-22 16:45:20 -07:00
Dongjoon Hyun c632bdc01f [SPARK-14029][SQL] Improve BooleanSimplification optimization by implementing Not canonicalization.
## What changes were proposed in this pull request?

Currently, **BooleanSimplification** optimization can handle the following cases.
* a && (!a || b ) ==> a && b
* a && (b || !a ) ==> a && b

However, it can not handle the followings cases since those equations fail at the comparisons between their canonicalized forms.
* a < 1 && (!(a < 1) || b)     ==> (a < 1) && b
* a <= 1 && (!(a <= 1) || b) ==> (a <= 1) && b
* a > 1 && (!(a > 1) || b)     ==> (a > 1) && b
* a >= 1 && (!(a >= 1) || b) ==> (a >= 1) && b

This PR implements the above cases and also the followings, too.
* a < 1 && ((a >= 1) || b )   ==> (a < 1) && b
* a <= 1 && ((a > 1) || b )   ==> (a <= 1) && b
* a > 1 && ((a <= 1) || b)  ==> (a > 1) && b
* a >= 1 && ((a < 1) || b)  ==> (a >= 1) && b

## How was this patch tested?

Pass the Jenkins tests including new test cases in BooleanSimplicationSuite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11851 from dongjoon-hyun/SPARK-14029.
2016-03-22 10:17:08 -07:00
Cheng Lian f2e855fba8 [SPARK-13473][SQL] Simplifies PushPredicateThroughProject
## What changes were proposed in this pull request?

This is a follow-up of PR #11348.

After PR #11348, a predicate is never pushed through a project as long as the project contains any non-deterministic fields. Thus, it's impossible that the candidate filter condition can reference any non-deterministic projected fields, and related logic can be safely cleaned up.

To be more specific, the following optimization is allowed:

```scala
// From:
df.select('a, 'b).filter('c > rand(42))
// To:
df.filter('c > rand(42)).select('a, 'b)
```

while this isn't:

```scala
// From:
df.select('a, rand('b) as 'rb, 'c).filter('c > 'rb)
// To:
df.filter('c > rand('b)).select('a, rand('b) as 'rb, 'c)
```

## How was this patch tested?

Existing test cases should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #11864 from liancheng/spark-13473-cleanup.
2016-03-22 19:20:56 +08:00
gatorsmile 3f49e0766f [SPARK-13320][SQL] Support Star in CreateStruct/CreateArray and Error Handling when DataFrame/DataSet Functions using Star
This PR resolves two issues:

First, expanding * inside aggregate functions of structs when using Dataframe/Dataset APIs. For example,
```scala
structDf.groupBy($"a").agg(min(struct($"record.*")))
```

Second, it improves the error messages when having invalid star usage when using Dataframe/Dataset APIs. For example,
```scala
pagecounts4PartitionsDS
  .map(line => (line._1, line._3))
  .toDF()
  .groupBy($"_1")
  .agg(sum("*") as "sumOccurances")
```
Before the fix, the invalid usage will issue a confusing error message, like:
```
org.apache.spark.sql.AnalysisException: cannot resolve '_1' given input columns _1, _2;
```
After the fix, the message is like:
```
org.apache.spark.sql.AnalysisException: Invalid usage of '*' in function 'sum'
```
cc: rxin nongli cloud-fan

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11208 from gatorsmile/sumDataSetResolution.
2016-03-22 08:21:02 +08:00
Wenchen Fan f3717fc7c9 [SPARK-14004][FOLLOW-UP] Implementations of NonSQLExpression should not override sql method
## What changes were proposed in this pull request?

There is only one exception: `PythonUDF`. However, I don't think the `PythonUDF#` prefix is useful, as we can only create python udf under python context. This PR removes the `PythonUDF#` prefix from `PythonUDF.toString`, so that it doesn't need to overrde `sql`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11859 from cloud-fan/tmp.
2016-03-21 15:24:18 -07:00
Cheng Lian 5d8de16e71 [SPARK-14004][SQL] NamedExpressions should have at most one qualifier
## What changes were proposed in this pull request?

This is a more aggressive version of PR #11820, which not only fixes the original problem, but also does the following updates to enforce the at-most-one-qualifier constraint:

- Renames `NamedExpression.qualifiers` to `NamedExpression.qualifier`
- Uses `Option[String]` rather than `Seq[String]` for `NamedExpression.qualifier`

Quoted PR description of #11820 here:

> Current implementations of `AttributeReference.sql` and `Alias.sql` joins all available qualifiers, which is logically wrong. But this implementation mistake doesn't cause any real SQL generation bugs though, since there is always at most one qualifier for any given `AttributeReference` or `Alias`.

## How was this patch tested?

Existing tests should be enough.

Author: Cheng Lian <lian@databricks.com>

Closes #11822 from liancheng/spark-14004-aggressive.
2016-03-21 11:00:09 -07:00
Wenchen Fan 43ebf7a9cb [SPARK-13456][SQL] fix creating encoders for case classes defined in Spark shell
## What changes were proposed in this pull request?

case classes defined in REPL are wrapped by line classes, and we have a trick for scala 2.10 REPL to automatically register the wrapper classes to `OuterScope` so that we can use when create encoders.
However, this trick doesn't work right after we upgrade to scala 2.11, and unfortunately the tests are only in scala 2.10, which makes this bug hidden until now.

This PR moves the encoder tests to scala 2.11  `ReplSuite`, and fixes this bug by another approach(the previous trick can't port to scala 2.11 REPL): make `OuterScope` smarter that can detect classes defined in REPL and load the singleton of line wrapper classes automatically.

## How was this patch tested?

the migrated encoder tests in `ReplSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11410 from cloud-fan/repl.
2016-03-21 10:37:24 -07:00
Wenchen Fan 17a3f00676 [SPARK-14000][SQL] case class with a tuple field can't work in Dataset
## What changes were proposed in this pull request?

When we validate an encoder, we may call `dataType` on unresolved expressions. This PR fix the validation so that we will resolve attributes first.

## How was this patch tested?

a new test in `DatasetSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11816 from cloud-fan/encoder.
2016-03-21 22:22:15 +08:00
gatorsmile 2c5b18fb0f [SPARK-12789][SQL] Support Order By Ordinal in SQL
#### What changes were proposed in this pull request?
This PR is to support order by position in SQL, e.g.
```SQL
select c1, c2, c3 from tbl order by 1 desc, 3
```
should be equivalent to
```SQL
select c1, c2, c3 from tbl order by c1 desc, c3 asc
```

This is controlled by config option `spark.sql.orderByOrdinal`.
- When true, the ordinal numbers are treated as the position in the select list.
- When false, the ordinal number in order/sort By clause are ignored.

- Only convert integer literals (not foldable expressions). If found foldable expressions, ignore them
- This also works with select *.

**Question**: Do we still need sort by columns that contain zero reference? In this case, it will have no impact on the sorting results. IMO, we should not allow users do it. rxin cloud-fan marmbrus yhuai hvanhovell
-- Update: In these cases, they are ignored in this case.

**Note**: This PR is taken from https://github.com/apache/spark/pull/10731. When merging this PR, please give the credit to zhichao-li

Also cc all the people who are involved in the previous discussion: adrian-wang chenghao-intel tejasapatil

#### How was this patch tested?
Added a few test cases for both positive and negative test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11815 from gatorsmile/orderByPosition.
2016-03-21 18:08:41 +08:00
Dongjoon Hyun 20fd254101 [SPARK-14011][CORE][SQL] Enable LineLength Java checkstyle rule
## What changes were proposed in this pull request?

[Spark Coding Style Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide) has 100-character limit on lines, but it's disabled for Java since 11/09/15. This PR enables **LineLength** checkstyle again. To help that, this also introduces **RedundantImport** and **RedundantModifier**, too. The following is the diff on `checkstyle.xml`.

```xml
-        <!-- TODO: 11/09/15 disabled - the lengths are currently > 100 in many places -->
-        <!--
         <module name="LineLength">
             <property name="max" value="100"/>
             <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://"/>
         </module>
-        -->
         <module name="NoLineWrap"/>
         <module name="EmptyBlock">
             <property name="option" value="TEXT"/>
 -167,5 +164,7
         </module>
         <module name="CommentsIndentation"/>
         <module name="UnusedImports"/>
+        <module name="RedundantImport"/>
+        <module name="RedundantModifier"/>
```

## How was this patch tested?

Currently, `lint-java` is disabled in Jenkins. It needs a manual test.
After passing the Jenkins tests, `dev/lint-java` should passes locally.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11831 from dongjoon-hyun/SPARK-14011.
2016-03-21 07:58:57 +00:00
gatorsmile f58319a24f [SPARK-14019][SQL] Remove noop SortOrder in Sort
#### What changes were proposed in this pull request?

This PR is to add a new Optimizer rule for pruning Sort if its SortOrder is no-op. In the phase of **Optimizer**, if a specific `SortOrder` does not have any reference, it has no effect on the sorting results. If `Sort` is empty, remove the whole `Sort`.

For example, in the following SQL query
```SQL
SELECT * FROM t ORDER BY NULL + 5
```

Before the fix, the plan is like
```
== Analyzed Logical Plan ==
a: int, b: int
Sort [(cast(null as int) + 5) ASC], true
+- Project [a#92,b#93]
   +- SubqueryAlias t
      +- Project [_1#89 AS a#92,_2#90 AS b#93]
         +- LocalRelation [_1#89,_2#90], [[1,2],[1,2]]

== Optimized Logical Plan ==
Sort [null ASC], true
+- LocalRelation [a#92,b#93], [[1,2],[1,2]]

== Physical Plan ==
WholeStageCodegen
:  +- Sort [null ASC], true, 0
:     +- INPUT
+- Exchange rangepartitioning(null ASC, 5), None
   +- LocalTableScan [a#92,b#93], [[1,2],[1,2]]
```

After the fix, the plan is like
```
== Analyzed Logical Plan ==
a: int, b: int
Sort [(cast(null as int) + 5) ASC], true
+- Project [a#92,b#93]
   +- SubqueryAlias t
      +- Project [_1#89 AS a#92,_2#90 AS b#93]
         +- LocalRelation [_1#89,_2#90], [[1,2],[1,2]]

== Optimized Logical Plan ==
LocalRelation [a#92,b#93], [[1,2],[1,2]]

== Physical Plan ==
LocalTableScan [a#92,b#93], [[1,2],[1,2]]
```

cc rxin cloud-fan marmbrus Thanks!

#### How was this patch tested?
Added a test suite for covering this rule

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11840 from gatorsmile/sortElimination.
2016-03-21 10:34:54 +08:00
Cheng Lian 14c7236dc6 [SPARK-14004][SQL][MINOR] AttributeReference and Alias should only use the first qualifier to generate SQL strings
## What changes were proposed in this pull request?

Current implementations of `AttributeReference.sql` and `Alias.sql` joins all available qualifiers, which is logically wrong. But this implementation mistake doesn't cause any real SQL generation bugs though, since there is always at most one qualifier for any given `AttributeReference` or `Alias`.

This PR fixes this issue by only picking the first qualifiers.

## How was this patch tested?

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

Existing tests should be enough.

Author: Cheng Lian <lian@databricks.com>

Closes #11820 from liancheng/spark-14004-single-qualifier.
2016-03-19 00:22:17 +08:00
Liang-Chi Hsieh 5f3bda6fe2 [SPARK-13838] [SQL] Clear variable code to prevent it to be re-evaluated in BoundAttribute
JIRA: https://issues.apache.org/jira/browse/SPARK-13838
## What changes were proposed in this pull request?

We should also clear the variable code in `BoundReference.genCode` to prevent it  to be evaluated twice, as we did in `evaluateVariables`.

## How was this patch tested?

Existing tests.

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

Closes #11674 from viirya/avoid-reevaluate.
2016-03-17 10:08:42 -07:00
Dilip Biswal 637a78f1d3 [SPARK-13427][SQL] Support USING clause in JOIN.
## What changes were proposed in this pull request?

Support queries that JOIN tables with USING clause.
SELECT * from table1 JOIN table2 USING <column_list>

USING clause can be used as a means to simplify the join condition
when :

1) Equijoin semantics is desired and
2) The column names in the equijoin have the same name.

We already have the support for Natural Join in Spark. This PR makes
use of the already existing infrastructure for natural join to
form the join condition and also the projection list.

## How was the this patch tested?

Have added unit tests in SQLQuerySuite, CatalystQlSuite, ResolveNaturalJoinSuite

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

Closes #11297 from dilipbiswal/spark-13427.
2016-03-17 10:01:41 -07:00
Wenchen Fan 8ef3399aff [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging
## What changes were proposed in this pull request?

Logging was made private in Spark 2.0. If we move it, then users would be able to create a Logging trait themselves to avoid changing their own code.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11764 from cloud-fan/logger.
2016-03-17 19:23:38 +08:00
Davies Liu 30c18841e4 Revert "[SPARK-13840][SQL] Split Optimizer Rule ColumnPruning to ColumnPruning and EliminateOperator"
This reverts commit 99bd2f0e94.
2016-03-16 23:11:13 -07:00
Jakob Odersky 7eef2463ad [SPARK-13118][SQL] Expression encoding for optional synthetic classes
## What changes were proposed in this pull request?

Fix expression generation for optional types.
Standard Java reflection causes issues when dealing with synthetic Scala objects (things that do not map to Java and thus contain a dollar sign in their name). This patch introduces Scala reflection in such cases.

This patch also adds a regression test for Dataset's handling of classes defined in package objects (which was the initial purpose of this PR).

## How was this patch tested?
A new test in ExpressionEncoderSuite that tests optional inner classes and a regression test for Dataset's handling of package objects.

Author: Jakob Odersky <jakob@odersky.com>

Closes #11708 from jodersky/SPARK-13118-package-objects.
2016-03-16 21:53:16 -07:00
Davies Liu c100d31ddc [SPARK-13873] [SQL] Avoid copy of UnsafeRow when there is no join in whole stage codegen
## What changes were proposed in this pull request?

We need to copy the UnsafeRow since a Join could produce multiple rows from single input rows. We could avoid that if there is no join (or the join will not produce multiple rows) inside WholeStageCodegen.

Updated the benchmark for `collect`, we could see 20-30% speedup.

## How was this patch tested?

existing unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11740 from davies/avoid_copy2.
2016-03-16 21:46:04 -07:00
Andrew Or ca9ef86c84 [SPARK-13923][SQL] Implement SessionCatalog
## What changes were proposed in this pull request?

As part of the effort to merge `SQLContext` and `HiveContext`, this patch implements an internal catalog called `SessionCatalog` that handles temporary functions and tables and delegates metastore operations to `ExternalCatalog`. Currently, this is still dead code, but in the future it will be part of `SessionState` and will replace `o.a.s.sql.catalyst.analysis.Catalog`.

A recent patch #11573 parses Hive commands ourselves in Spark, but still passes the entire query text to Hive. In a future patch, we will use `SessionCatalog` to implement the parsed commands.

## How was this patch tested?

800+ lines of tests in `SessionCatalogSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #11750 from andrewor14/temp-catalog.
2016-03-16 18:02:43 -07:00
Jakob Odersky d4d84936fb [SPARK-11011][SQL] Narrow type of UDT serialization
## What changes were proposed in this pull request?

Narrow down the parameter type of `UserDefinedType#serialize()`. Currently, the parameter type is `Any`, however it would logically make more sense to narrow it down to the type of the actual user defined type.

## How was this patch tested?

Existing tests were successfully run on local machine.

Author: Jakob Odersky <jakob@odersky.com>

Closes #11379 from jodersky/SPARK-11011-udt-types.
2016-03-16 16:59:36 -07:00
Sameer Agarwal 77ba3021c1 [SPARK-13869][SQL] Remove redundant conditions while combining filters
## What changes were proposed in this pull request?

**[I'll link it to the JIRA once ASF JIRA is back online]**

This PR modifies the existing `CombineFilters` rule to remove redundant conditions while combining individual filter predicates. For instance, queries of the form `table.where('a === 1 && 'b === 1).where('a === 1 && 'c === 1)` will now be optimized to ` table.where('a === 1 && 'b === 1 && 'c === 1)` (instead of ` table.where('a === 1 && 'a === 1 && 'b === 1 && 'c === 1)`)

## How was this patch tested?

Unit test in `FilterPushdownSuite`

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11670 from sameeragarwal/combine-filters.
2016-03-16 16:27:46 -07:00
Sameer Agarwal f96997ba24 [SPARK-13871][SQL] Support for inferring filters from data constraints
## What changes were proposed in this pull request?

This PR generalizes the `NullFiltering` optimizer rule in catalyst to `InferFiltersFromConstraints` that can automatically infer all relevant filters based on an operator's constraints while making sure of 2 things:

(a) no redundant filters are generated, and
(b) filters that do not contribute to any further optimizations are not generated.

## How was this patch tested?

Extended all tests in `InferFiltersFromConstraintsSuite` (that were initially based on `NullFilteringSuite` to test filter inference in `Filter` and `Join` operators.

In particular the 2 tests ( `single inner join with pre-existing filters: filter out values on either side` and `multiple inner joins: filter out values on all sides on equi-join keys` attempts to highlight/test the real potential of this rule for join optimization.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11665 from sameeragarwal/infer-filters.
2016-03-16 16:26:51 -07:00
Wenchen Fan 1d1de28a3c [SPARK-13827][SQL] Can't add subquery to an operator with same-name outputs while generate SQL string
## What changes were proposed in this pull request?

This PR tries to solve a fundamental issue in the `SQLBuilder`. When we want to turn a logical plan into SQL string and put it after FROM clause, we need to wrap it with a sub-query. However, a logical plan is allowed to have same-name outputs with different qualifiers(e.g. the `Join` operator), and this kind of plan can't be put under a subquery as we will erase and assign a new qualifier to all outputs and make it impossible to distinguish same-name outputs.

To solve this problem, this PR renames all attributes with globally unique names(using exprId), so that we don't need qualifiers to resolve ambiguity anymore.

For example, `SELECT x.key, MAX(y.key) OVER () FROM t x JOIN t y`, we will parse this SQL to a Window operator and a Project operator, and add a sub-query between them. The generated SQL looks like:
```
SELECT sq_1.key, sq_1.max
FROM (
    SELECT sq_0.key, sq_0.key, MAX(sq_0.key) OVER () AS max
    FROM (
        SELECT x.key, y.key FROM t1 AS x JOIN t2 AS y
    ) AS sq_0
) AS sq_1
```
You can see, the `key` columns become ambiguous after `sq_0`.

After this PR, it will generate something like:
```
SELECT attr_30 AS key, attr_37 AS max
FROM (
    SELECT attr_30, attr_37
    FROM (
        SELECT attr_30, attr_35, MAX(attr_35) AS attr_37
        FROM (
            SELECT attr_30, attr_35 FROM
                (SELECT key AS attr_30 FROM t1) AS sq_0
            INNER JOIN
                (SELECT key AS attr_35 FROM t1) AS sq_1
        ) AS sq_2
    ) AS sq_3
) AS sq_4
```
The outermost SELECT is used to turn the generated named to real names back, and the innermost SELECT is used to alias real columns to our generated names. Between them, there is no name ambiguity anymore.

## How was this patch tested?

existing tests and new tests in LogicalPlanToSQLSuite.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11658 from cloud-fan/gensql.
2016-03-16 11:57:28 -07:00
Wenchen Fan d9e8f26d03 [SPARK-13924][SQL] officially support multi-insert
## What changes were proposed in this pull request?

There is a feature of hive SQL called multi-insert. For example:
```
FROM src
INSERT OVERWRITE TABLE dest1
SELECT key + 1
INSERT OVERWRITE TABLE dest2
SELECT key WHERE key > 2
INSERT OVERWRITE TABLE dest3
SELECT col EXPLODE(arr) exp AS col
...
```

We partially support it currently, with some limitations: 1) WHERE can't reference columns produced by LATERAL VIEW. 2) It's not executed eagerly, i.e. `sql("...multi-insert clause...")` won't take place right away like other commands, e.g. CREATE TABLE.

This PR removes these limitations and make us fully support multi-insert.

## How was this patch tested?

new tests in `SQLQuerySuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11754 from cloud-fan/lateral-view.
2016-03-16 10:52:36 -07:00
Sean Owen 3b461d9ecd [SPARK-13823][SPARK-13397][SPARK-13395][CORE] More warnings, StandardCharset follow up
## What changes were proposed in this pull request?

Follow up to https://github.com/apache/spark/pull/11657

- Also update `String.getBytes("UTF-8")` to use `StandardCharsets.UTF_8`
- And fix one last new Coverity warning that turned up (use of unguarded `wait()` replaced by simpler/more robust `java.util.concurrent` classes in tests)
- And while we're here cleaning up Coverity warnings, just fix about 15 more build warnings

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #11725 from srowen/SPARK-13823.2.
2016-03-16 09:36:34 +00:00
Yucai Yu 52b6a899be [MINOR][TEST][SQL] Remove wrong "expected" parameter in checkNaNWithoutCodegen
## What changes were proposed in this pull request?

Remove the wrong "expected" parameter in MathFunctionsSuite.scala's checkNaNWithoutCodegen.
This function is to check NaN value, so the "expected" parameter is useless. The Callers do not pass "expected" value and the similar function like checkNaNWithGeneratedProjection and checkNaNWithOptimization do not use it also.

Author: Yucai Yu <yucai.yu@intel.com>

Closes #11718 from yucai/unused_expected.
2016-03-15 21:44:58 -07:00
gatorsmile 99bd2f0e94 [SPARK-13840][SQL] Split Optimizer Rule ColumnPruning to ColumnPruning and EliminateOperator
#### What changes were proposed in this pull request?

Before this PR, two Optimizer rules `ColumnPruning` and `PushPredicateThroughProject` reverse each other's effects. Optimizer always reaches the max iteration when optimizing some queries. Extra `Project` are found in the plan. For example, below is the optimized plan after reaching 100 iterations:

```
Join Inner, Some((cast(id1#16 as bigint) = id1#18L))
:- Project [id1#16]
:  +- Filter isnotnull(cast(id1#16 as bigint))
:     +- Project [id1#16]
:        +- Relation[id1#16,newCol#17] JSON part: struct<>, data: struct<id1:int,newCol:int>
+- Filter isnotnull(id1#18L)
   +- Relation[id1#18L] JSON part: struct<>, data: struct<id1:bigint>
```

This PR splits the optimizer rule `ColumnPruning` to `ColumnPruning` and `EliminateOperators`

The issue becomes worse when having another rule `NullFiltering`, which could add extra Filters for `IsNotNull`. We have to be careful when introducing extra `Filter` if the benefit is not large enough. Another PR will be submitted by sameeragarwal to handle this issue.

cc sameeragarwal marmbrus

In addition, `ColumnPruning` should not push `Project` through non-deterministic `Filter`. This could cause wrong results. This will be put in a separate PR.

cc davies cloud-fan yhuai

#### How was this patch tested?

Modified the existing test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11682 from gatorsmile/viewDuplicateNames.
2016-03-15 00:30:14 -07:00
Michael Armbrust 17eec0a71b [SPARK-13664][SQL] Add a strategy for planning partitioned and bucketed scans of files
This PR adds a new strategy, `FileSourceStrategy`, that can be used for planning scans of collections of files that might be partitioned or bucketed.

Compared with the existing planning logic in `DataSourceStrategy` this version has the following desirable properties:
 - It removes the need to have `RDD`, `broadcastedHadoopConf` and other distributed concerns  in the public API of `org.apache.spark.sql.sources.FileFormat`
 - Partition column appending is delegated to the format to avoid an extra copy / devectorization when appending partition columns
 - It minimizes the amount of data that is shipped to each executor (i.e. it does not send the whole list of files to every worker in the form of a hadoop conf)
 - it natively supports bucketing files into partitions, and thus does not require coalescing / creating a `UnionRDD` with the correct partitioning.
 - Small files are automatically coalesced into fewer tasks using an approximate bin-packing algorithm.

Currently only a testing source is planned / tested using this strategy.  In follow-up PRs we will port the existing formats to this API.

A stub for `FileScanRDD` is also added, but most methods remain unimplemented.

Other minor cleanups:
 - partition pruning is pushed into `FileCatalog` so both the new and old code paths can use this logic.  This will also allow future implementations to use indexes or other tricks (i.e. a MySQL metastore)
 - The partitions from the `FileCatalog` now propagate information about file sizes all the way up to the planner so we can intelligently spread files out.
 - `Array` -> `Seq` in some internal APIs to avoid unnecessary `toArray` calls
 - Rename `Partition` to `PartitionDirectory` to differentiate partitions used earlier in pruning from those where we have already enumerated the files and their sizes.

Author: Michael Armbrust <michael@databricks.com>

Closes #11646 from marmbrus/fileStrategy.
2016-03-14 19:21:12 -07:00
Liang-Chi Hsieh 6a4bfcd62b [SPARK-13658][SQL] BooleanSimplification rule is slow with large boolean expressions
JIRA: https://issues.apache.org/jira/browse/SPARK-13658

## What changes were proposed in this pull request?

Quoted from JIRA description: When run TPCDS Q3 [1] with lots predicates to filter out the partitions, the optimizer rule BooleanSimplification take about 2 seconds (it use lots of sematicsEqual, which require copy the whole tree).

It will great if we could speedup it.

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

How to speed up it:

When we ask the canonicalized expression in `Expression`, it calls `Canonicalize.execute` on itself. `Canonicalize.execute` basically transforms up all expressions included in this expression. However, we don't keep the canonicalized versions for these children expressions. So in next time we ask the canonicalized expressions for the children expressions (e.g., `BooleanSimplification`), we will rerun `Canonicalize.execute` on each of them. It wastes much time.

By forcing the children expressions to get and keep their canonicalized versions first, we can avoid re-canonicalize these expressions.

I simply benchmark it with an expression which is part of the where clause in TPCDS Q3:

    val testRelation = LocalRelation('ss_sold_date_sk.int, 'd_moy.int, 'i_manufact_id.int, 'ss_item_sk.string, 'i_item_sk.string, 'd_date_sk.int)

    val input = ('d_date_sk === 'ss_sold_date_sk) && ('ss_item_sk === 'i_item_sk) && ('i_manufact_id === 436) && ('d_moy === 12) && (('ss_sold_date_sk > 2415355 && 'ss_sold_date_sk < 2415385) || ('ss_sold_date_sk > 2415720 && 'ss_sold_date_sk < 2415750) || ('ss_sold_date_sk > 2416085 && 'ss_sold_date_sk < 2416115) || ('ss_sold_date_sk > 2416450 && 'ss_sold_date_sk < 2416480) || ('ss_sold_date_sk > 2416816 && 'ss_sold_date_sk < 2416846) || ('ss_sold_date_sk > 2417181 && 'ss_sold_date_sk < 2417211) || ('ss_sold_date_sk > 2417546 && 'ss_sold_date_sk < 2417576) || ('ss_sold_date_sk > 2417911 && 'ss_sold_date_sk < 2417941) || ('ss_sold_date_sk > 2418277 && 'ss_sold_date_sk < 2418307) || ('ss_sold_date_sk > 2418642 && 'ss_sold_date_sk < 2418672) || ('ss_sold_date_sk > 2419007 && 'ss_sold_date_sk < 2419037) || ('ss_sold_date_sk > 2419372 && 'ss_sold_date_sk < 2419402) || ('ss_sold_date_sk > 2419738 && 'ss_sold_date_sk < 2419768) || ('ss_sold_date_sk > 2420103 && 'ss_sold_date_sk < 2420133) || ('ss_sold_date_sk > 2420468 && 'ss_sold_date_sk < 2420498) || ('ss_sold_date_sk > 2420833 && 'ss_sold_date_sk < 2420863) || ('ss_sold_date_sk > 2421199 && 'ss_sold_date_sk < 2421229) || ('ss_sold_date_sk > 2421564 && 'ss_sold_date_sk < 2421594) || ('ss_sold_date_sk > 2421929 && 'ss_sold_date_sk < 2421959) || ('ss_sold_date_sk > 2422294 && 'ss_sold_date_sk < 2422324) || ('ss_sold_date_sk > 2422660 && 'ss_sold_date_sk < 2422690) || ('ss_sold_date_sk > 2423025 && 'ss_sold_date_sk < 2423055) || ('ss_sold_date_sk > 2423390 && 'ss_sold_date_sk < 2423420) || ('ss_sold_date_sk > 2423755 && 'ss_sold_date_sk < 2423785) || ('ss_sold_date_sk > 2424121 && 'ss_sold_date_sk < 2424151) || ('ss_sold_date_sk > 2424486 && 'ss_sold_date_sk < 2424516) || ('ss_sold_date_sk > 2424851 && 'ss_sold_date_sk < 2424881) || ('ss_sold_date_sk > 2425216 && 'ss_sold_date_sk < 2425246) || ('ss_sold_date_sk > 2425582 && 'ss_sold_date_sk < 2425612) || ('ss_sold_date_sk > 2425947 && 'ss_sold_date_sk < 2425977) || ('ss_sold_date_sk > 2426312 && 'ss_sold_date_sk < 2426342) || ('ss_sold_date_sk > 2426677 && 'ss_sold_date_sk < 2426707) || ('ss_sold_date_sk > 2427043 && 'ss_sold_date_sk < 2427073) || ('ss_sold_date_sk > 2427408 && 'ss_sold_date_sk < 2427438) || ('ss_sold_date_sk > 2427773 && 'ss_sold_date_sk < 2427803) || ('ss_sold_date_sk > 2428138 && 'ss_sold_date_sk < 2428168) || ('ss_sold_date_sk > 2428504 && 'ss_sold_date_sk < 2428534) || ('ss_sold_date_sk > 2428869 && 'ss_sold_date_sk < 2428899) || ('ss_sold_date_sk > 2429234 && 'ss_sold_date_sk < 2429264) || ('ss_sold_date_sk > 2429599 && 'ss_sold_date_sk < 2429629) || ('ss_sold_date_sk > 2429965 && 'ss_sold_date_sk < 2429995) || ('ss_sold_date_sk > 2430330 && 'ss_sold_date_sk < 2430360) || ('ss_sold_date_sk > 2430695 && 'ss_sold_date_sk < 2430725) || ('ss_sold_date_sk > 2431060 && 'ss_sold_date_sk < 2431090) || ('ss_sold_date_sk > 2431426 && 'ss_sold_date_sk < 2431456) || ('ss_sold_date_sk > 2431791 && 'ss_sold_date_sk < 2431821) || ('ss_sold_date_sk > 2432156 && 'ss_sold_date_sk < 2432186) || ('ss_sold_date_sk > 2432521 && 'ss_sold_date_sk < 2432551) || ('ss_sold_date_sk > 2432887 && 'ss_sold_date_sk < 2432917) || ('ss_sold_date_sk > 2433252 && 'ss_sold_date_sk < 2433282) || ('ss_sold_date_sk > 2433617 && 'ss_sold_date_sk < 2433647) || ('ss_sold_date_sk > 2433982 && 'ss_sold_date_sk < 2434012) || ('ss_sold_date_sk > 2434348 && 'ss_sold_date_sk < 2434378) || ('ss_sold_date_sk > 2434713 && 'ss_sold_date_sk < 2434743)))

    val plan = testRelation.where(input).analyze
    val actual = Optimize.execute(plan)

With this patch:

    352 milliseconds
    346 milliseconds
    340 milliseconds

Without this patch:

    585 milliseconds
    880 milliseconds
    677 milliseconds

## How was this patch tested?

Existing tests should pass.

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

Closes #11647 from viirya/improve-expr-canonicalize.
2016-03-14 11:23:29 -07:00
Dongjoon Hyun acdf219703 [MINOR][DOCS] Fix more typos in comments/strings.
## What changes were proposed in this pull request?

This PR fixes 135 typos over 107 files:
* 121 typos in comments
* 11 typos in testcase name
* 3 typos in log messages

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11689 from dongjoon-hyun/fix_more_typos.
2016-03-14 09:07:39 +00:00
Sean Owen 1840852841 [SPARK-13823][CORE][STREAMING][SQL] Always specify Charset in String <-> byte[] conversions (and remaining Coverity items)
## What changes were proposed in this pull request?

- Fixes calls to `new String(byte[])` or `String.getBytes()` that rely on platform default encoding, to use UTF-8
- Same for `InputStreamReader` and `OutputStreamWriter` constructors
- Standardizes on UTF-8 everywhere
- Standardizes specifying the encoding with `StandardCharsets.UTF-8`, not the Guava constant or "UTF-8" (which means handling `UnuspportedEncodingException`)
- (also addresses the other remaining Coverity scan issues, which are pretty trivial; these are separated into commit 1deecd8d9c )

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #11657 from srowen/SPARK-13823.
2016-03-13 21:03:49 -07:00
Davies Liu ba8c86d06f [SPARK-13671] [SPARK-13311] [SQL] Use different physical plans for RDD and data sources
## What changes were proposed in this pull request?

This PR split the PhysicalRDD into two classes, PhysicalRDD and PhysicalScan. PhysicalRDD is used for DataFrames that is created from existing RDD. PhysicalScan is used for DataFrame that is created from data sources. This enable use to apply different optimization on both of them.

Also fix the problem for sameResult() on two DataSourceScan.

Also fix the equality check to toString for `In`. It's better to use Seq there, but we can't break this public API (sad).

## How was this patch tested?

Existing tests. Manually tested with TPCDS query Q59 and Q64, all those duplicated exchanges can be re-used now, also saw there are 40+% performance improvement (saving half of the scan).

Author: Davies Liu <davies@databricks.com>

Closes #11514 from davies/existing_rdd.
2016-03-12 00:48:36 -08:00
Andrew Or 66d9d0edfe [SPARK-13139][SQL] Parse Hive DDL commands ourselves
## What changes were proposed in this pull request?

This patch is ported over from viirya's changes in #11048. Currently for most DDLs we just pass the query text directly to Hive. Instead, we should parse these commands ourselves and in the future (not part of this patch) use the `HiveCatalog` to process these DDLs. This is a pretext to merging `SQLContext` and `HiveContext`.

Note: As of this patch we still pass the query text to Hive. The difference is that we now parse the commands ourselves so in the future we can just use our own catalog.

## How was this patch tested?

Jenkins, new `DDLCommandSuite`, which comprises of about 40% of the changes here.

Author: Andrew Or <andrew@databricks.com>

Closes #11573 from andrewor14/parser-plus-plus.
2016-03-11 15:13:48 -08:00
Wenchen Fan 6871cc8f3e [SPARK-12718][SPARK-13720][SQL] SQL generation support for window functions
## What changes were proposed in this pull request?

Add SQL generation support for window functions. The idea is simple, just treat `Window` operator like `Project`, i.e. add subquery to its child when necessary, generate a `SELECT ... FROM ...` SQL string, implement `sql` method for window related expressions, e.g. `WindowSpecDefinition`, `WindowFrame`, etc.

This PR also fixed SPARK-13720 by improving the process of adding extra `SubqueryAlias`(the `RecoverScopingInfo` rule). Before this PR, we update the qualifiers in project list while adding the subquery. However, this is incomplete as we need to update qualifiers in all ancestors that refer attributes here. In this PR, we split `RecoverScopingInfo` into 2 rules: `AddSubQuery` and `UpdateQualifier`. `AddSubQuery` only add subquery if necessary, and `UpdateQualifier` will re-propagate and update qualifiers bottom up.

Ideally we should put the bug fix part in an individual PR, but this bug also blocks the window stuff, so I put them together here.

Many thanks to gatorsmile for the initial discussion and test cases!

## How was this patch tested?

new tests in `LogicalPlanToSQLSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11555 from cloud-fan/window.
2016-03-11 13:22:34 +08:00
gatorsmile 560489f4e1 [SPARK-13732][SPARK-13797][SQL] Remove projectList from Window and Eliminate useless Window
#### What changes were proposed in this pull request?

`projectList` is useless. Its value is always the same as the child.output. Remove it from the class `Window`. Removal can simplify the codes in Analyzer and Optimizer.

This PR is based on the discussion started by cloud-fan in a separate PR:
https://github.com/apache/spark/pull/5604#discussion_r55140466

This PR also eliminates useless `Window`.

cloud-fan yhuai

#### How was this patch tested?

Existing test cases cover it.

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

Closes #11565 from gatorsmile/removeProjListWindow.
2016-03-11 11:59:18 +08:00
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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 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
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
Michael Armbrust 2b042577fb [SPARK-13092][SQL] Add ExpressionSet for constraint tracking
This PR adds a new abstraction called an `ExpressionSet` which attempts to canonicalize expressions to remove cosmetic differences.  Deterministic expressions that are in the set after canonicalization will always return the same answer given the same input (i.e. false positives should not be possible). However, it is possible that two canonical expressions that are not equal will in fact return the same answer given any input (i.e. false negatives are possible).

```scala
val set = AttributeSet('a + 1 :: 1 + 'a :: Nil)

set.iterator => Iterator('a + 1)
set.contains('a + 1) => true
set.contains(1 + 'a) => true
set.contains('a + 2) => false
```

Other relevant changes include:
 - Since this concept overlaps with the existing `semanticEquals` and `semanticHash`, those functions are also ported to this new infrastructure.
 - A memoized `canonicalized` version of the expression is added as a `lazy val` to `Expression` and is used by both `semanticEquals` and `ExpressionSet`.
 - A set of unit tests for `ExpressionSet` are added
 - Tests which expect `semanticEquals` to be less intelligent than it now is are updated.

As a followup, we should consider auditing the places where we do `O(n)` `semanticEquals` operations and replace them with `ExpressionSet`.  We should also consider consolidating `AttributeSet` as a specialized factory for an `ExpressionSet.`

Author: Michael Armbrust <michael@databricks.com>

Closes #11338 from marmbrus/expressionSet.
2016-02-24 19:43:00 -08:00
Yin Huai cbb0b65ad5 [SPARK-13383][SQL] Fix test
## What changes were proposed in this pull request?

Reverting SPARK-13376 (d563c8fa01) affects the test added by SPARK-13383. So, I am fixing the test.

Author: Yin Huai <yhuai@databricks.com>

Closes #11355 from yhuai/SPARK-13383-fix-test.
2016-02-24 16:13:55 -08:00
Reynold Xin f92f53faee Revert "[SPARK-13321][SQL] Support nested UNION in parser"
This reverts commit 55d6fdf22d.
2016-02-24 12:25:02 -08:00
Reynold Xin 65805ab6ea Revert "Revert "[SPARK-13383][SQL] Keep broadcast hint after column pruning""
This reverts commit 382b27babf.
2016-02-24 12:03:45 -08:00
Reynold Xin d563c8fa01 Revert "[SPARK-13376] [SQL] improve column pruning"
This reverts commit e9533b419e.
2016-02-24 11:58:32 -08:00
Reynold Xin 382b27babf Revert "[SPARK-13383][SQL] Keep broadcast hint after column pruning"
This reverts commit f373986997.
2016-02-24 11:58:12 -08:00
Liang-Chi Hsieh f373986997 [SPARK-13383][SQL] Keep broadcast hint after column pruning
JIRA: https://issues.apache.org/jira/browse/SPARK-13383

## What changes were proposed in this pull request?

When we do column pruning in Optimizer, we put additional Project on top of a logical plan. However, when we already wrap a BroadcastHint on a logical plan, the added Project will hide BroadcastHint after later execution.

We should take care of BroadcastHint when we do column pruning.

## How was the this patch tested?

Unit test is added.

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

Closes #11260 from viirya/keep-broadcasthint.
2016-02-24 10:22:40 -08:00
Davies Liu 86c852cf2e [SPARK-13431] [SQL] [test-maven] split keywords from ExpressionParser.g
## What changes were proposed in this pull request?

This PR pull all the keywords (and some others) from ExpressionParser.g as KeywordParser.g, because ExpressionParser is too large to compile.

## How was the this patch tested?

unit test, maven build

Closes #11329

Author: Davies Liu <davies@databricks.com>

Closes #11331 from davies/split_expr.
2016-02-23 21:22:00 -08:00
Davies Liu e9533b419e [SPARK-13376] [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).

## How was the 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 #11256 from davies/fix_column_pruning.
2016-02-23 18:19:22 -08:00
Davies Liu c481bdf512 [SPARK-13329] [SQL] considering output for statistics of logical plan
The current implementation of statistics of UnaryNode does not considering output (for example, Project may product much less columns than it's child), we should considering it to have a better guess.

We usually only join with few columns from a parquet table, the size of projected plan could be much smaller than the original parquet files. Having a better guess of size help we choose between broadcast join or sort merge join.

After this PR, I saw a few queries choose broadcast join other than sort merge join without turning spark.sql.autoBroadcastJoinThreshold for every query, ended up with about 6-8X improvements on end-to-end time.

We use `defaultSize` of DataType to estimate the size of a column, currently For DecimalType/StringType/BinaryType and UDT, we are over-estimate too much (4096 Bytes), so this PR change them to some more reasonable values. Here are the new defaultSize for them:

DecimalType:  8 or 16 bytes, based on the precision
StringType:  20 bytes
BinaryType: 100 bytes
UDF: default size of SQL type

These numbers are not perfect (hard to have a perfect number for them), but should be better than 4096.

Author: Davies Liu <davies@databricks.com>

Closes #11210 from davies/statics.
2016-02-23 12:55:44 -08:00
Michael Armbrust c5bfe5d2a2 [SPARK-13440][SQL] ObjectType should accept any ObjectType, If should not care about nullability
The type checking functions of `If` and `UnwrapOption` are fixed to eliminate spurious failures.  `UnwrapOption` was checking for an input of `ObjectType` but `ObjectType`'s accept function was hard coded to return `false`.  `If`'s type check was returning a false negative in the case that the two options differed only by nullability.

Tests added:
 -  an end-to-end regression test is added to `DatasetSuite` for the reported failure.
 - all the unit tests in `ExpressionEncoderSuite` are augmented to also confirm successful analysis.  These tests are actually what pointed out the additional issues with `If` resolution.

Author: Michael Armbrust <michael@databricks.com>

Closes #11316 from marmbrus/datasetOptions.
2016-02-23 11:20:27 -08:00
gatorsmile 87250580f2 [SPARK-13263][SQL] SQL Generation Support for Tablesample
In the parser, tableSample clause is part of tableSource.
```
tableSource
init { gParent.pushMsg("table source", state); }
after { gParent.popMsg(state); }
    : tabname=tableName
    ((tableProperties) => props=tableProperties)?
    ((tableSample) => ts=tableSample)?
    ((KW_AS) => (KW_AS alias=Identifier)
    |
    (Identifier) => (alias=Identifier))?
    -> ^(TOK_TABREF $tabname $props? $ts? $alias?)
    ;
```

Two typical query samples using TABLESAMPLE are:
```
    "SELECT s.id FROM t0 TABLESAMPLE(10 PERCENT) s"
    "SELECT * FROM t0 TABLESAMPLE(0.1 PERCENT)"
```

FYI, the logical plan of a TABLESAMPLE query:
```
sql("SELECT * FROM t0 TABLESAMPLE(0.1 PERCENT)").explain(true)

== Analyzed Logical Plan ==
id: bigint
Project [id#16L]
+- Sample 0.0, 0.001, false, 381
   +- Subquery t0
      +- Relation[id#16L] ParquetRelation
```

Thanks! cc liancheng

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

This patch had conflicts when merged, resolved by
Committer: Cheng Lian <lian@databricks.com>

Closes #11148 from gatorsmile/tablesplitsample.
2016-02-23 16:13:09 +08:00
gatorsmile 9dd5399d78 [SPARK-12723][SQL] Comprehensive Verification and Fixing of SQL Generation Support for Expressions
#### What changes were proposed in this pull request?

Ensure that all built-in expressions can be mapped to its SQL representation if there is one (e.g. ScalaUDF doesn't have a SQL representation). The function lists are from the expression list in `FunctionRegistry`.

window functions, grouping sets functions (`cube`, `rollup`, `grouping`, `grouping_id`), generator functions (`explode` and `json_tuple`) are covered by separate JIRA and PRs. Thus, this PR does not cover them. Except these functions, all the built-in expressions are covered. For details, see the list in `ExpressionToSQLSuite`.

Fixed a few issues. For example, the `prettyName` of `approx_count_distinct` is not right. The `sql` of `hash` function is not right, since the `hash` function does not accept `seed`.

Additionally, also correct the order of expressions in `FunctionRegistry` so that people are easier to find which functions are missing.

cc liancheng

#### How was the this patch tested?
Added two test cases in LogicalPlanToSQLSuite for covering `not like` and `not in`.

Added a new test suite `ExpressionToSQLSuite` to cover the functions:

1. misc non-aggregate functions + complex type creators + null expressions
2. math functions
3. aggregate functions
4. string functions
5. date time functions + calendar interval
6. collection functions
7. misc functions

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11314 from gatorsmile/expressionToSQL.
2016-02-22 22:17:56 -08:00
Dongjoon Hyun 024482bf51 [MINOR][DOCS] Fix all typos in markdown files of doc and similar patterns in other comments
## What changes were proposed in this pull request?

This PR tries to fix all typos in all markdown files under `docs` module,
and fixes similar typos in other comments, too.

## How was the this patch tested?

manual tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11300 from dongjoon-hyun/minor_fix_typos.
2016-02-22 09:52:07 +00:00
Reynold Xin 9bf6a926a1 [HOTFIX] Fix compilation break 2016-02-21 19:37:35 -08:00
Liang-Chi Hsieh 55d6fdf22d [SPARK-13321][SQL] Support nested UNION in parser
JIRA: https://issues.apache.org/jira/browse/SPARK-13321

The following SQL can not be parsed with current parser:

    SELECT  `u_1`.`id` FROM (((SELECT  `t0`.`id` FROM `default`.`t0`) UNION ALL (SELECT  `t0`.`id` FROM `default`.`t0`)) UNION ALL (SELECT  `t0`.`id` FROM `default`.`t0`)) AS u_1

We should fix it.

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

Closes #11204 from viirya/nested-union.
2016-02-21 19:10:17 -08:00
Franklyn D'souza 0f90f4e6ac [SPARK-13410][SQL] Support unionAll for DataFrames with UDT columns.
## What changes were proposed in this pull request?

This PR adds equality operators to UDT classes so that they can be correctly tested for dataType equality during union operations.

This was previously causing `"AnalysisException: u"unresolved operator 'Union;""` when trying to unionAll two dataframes with UDT columns as below.

```
from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
from pyspark.sql import types

schema = types.StructType([types.StructField("point", PythonOnlyUDT(), True)])

a = sqlCtx.createDataFrame([[PythonOnlyPoint(1.0, 2.0)]], schema)
b = sqlCtx.createDataFrame([[PythonOnlyPoint(3.0, 4.0)]], schema)

c = a.unionAll(b)
```

## How was the this patch tested?

Tested using two unit tests in sql/test.py and the DataFrameSuite.

Additional information here : https://issues.apache.org/jira/browse/SPARK-13410

Author: Franklyn D'souza <franklynd@gmail.com>

Closes #11279 from damnMeddlingKid/udt-union-all.
2016-02-21 16:58:17 -08:00
Andrew Or 6c3832b26e [SPARK-13080][SQL] Implement new Catalog API using Hive
## What changes were proposed in this pull request?

This is a step towards merging `SQLContext` and `HiveContext`. A new internal Catalog API was introduced in #10982 and extended in #11069. This patch introduces an implementation of this API using `HiveClient`, an existing interface to Hive. It also extends `HiveClient` with additional calls to Hive that are needed to complete the catalog implementation.

*Where should I start reviewing?* The new catalog introduced is `HiveCatalog`. This class is relatively simple because it just calls `HiveClientImpl`, where most of the new logic is. I would not start with `HiveClient`, `HiveQl`, or `HiveMetastoreCatalog`, which are modified mainly because of a refactor.

*Why is this patch so big?* I had to refactor HiveClient to remove an intermediate representation of databases, tables, partitions etc. After this refactor `CatalogTable` convert directly to and from `HiveTable` (etc.). Otherwise we would have to first convert `CatalogTable` to the intermediate representation and then convert that to HiveTable, which is messy.

The new class hierarchy is as follows:
```
org.apache.spark.sql.catalyst.catalog.Catalog
  - org.apache.spark.sql.catalyst.catalog.InMemoryCatalog
  - org.apache.spark.sql.hive.HiveCatalog
```

Note that, as of this patch, none of these classes are currently used anywhere yet. This will come in the future before the Spark 2.0 release.

## How was the this patch tested?
All existing unit tests, and HiveCatalogSuite that extends CatalogTestCases.

Author: Andrew Or <andrew@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #11293 from rxin/hive-catalog.
2016-02-21 15:00:24 -08:00
Herman van Hovell b6a873d6d4 [SPARK-13136][SQL] Create a dedicated Broadcast exchange operator
Quite a few Spark SQL join operators broadcast one side of the join to all nodes. The are a few problems with this:

- This conflates broadcasting (a data exchange) with joining. Data exchanges should be managed by a different operator.
- All these nodes implement their own (duplicate) broadcasting logic.
- Re-use of indices is quite hard.

This PR defines both a ```BroadcastDistribution``` and ```BroadcastPartitioning```, these contain a `BroadcastMode`. The `BroadcastMode` defines the way in which we transform the Array of `InternalRow`'s into an index. We currently support the following `BroadcastMode`'s:

- IdentityBroadcastMode: This broadcasts the rows in their original form.
- HashSetBroadcastMode: This applies a projection to the input rows, deduplicates these rows and broadcasts the resulting `Set`.
- HashedRelationBroadcastMode: This transforms the input rows into a `HashedRelation`, and broadcasts this index.

To match this distribution we implement a ```BroadcastExchange``` operator which will perform the broadcast for us, and have ```EnsureRequirements``` plan this operator. The old Exchange operator has been renamed into ShuffleExchange in order to clearly separate between Shuffled and Broadcasted exchanges. Finally the classes in Exchange.scala have been moved to a dedicated package.

cc rxin davies

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

Closes #11083 from hvanhovell/SPARK-13136.
2016-02-21 12:32:31 -08:00
Reynold Xin af441ddbd1 [SPARK-13306][SQL] Addendum to uncorrelated scalar subquery
## What changes were proposed in this pull request?
This pull request fixes some minor issues (documentation, test flakiness, test organization) with #11190, which was merged earlier tonight.

## How was the this patch tested?
unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11285 from rxin/subquery.
2016-02-21 12:27:02 -08:00
Reynold Xin 0947f0989b [SPARK-13420][SQL] Rename Subquery logical plan to SubqueryAlias
## What changes were proposed in this pull request?
This patch renames logical.Subquery to logical.SubqueryAlias, which is a more appropriate name for this operator (versus subqueries as expressions).

## How was the this patch tested?
Unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11288 from rxin/SPARK-13420.
2016-02-21 11:31:46 -08:00
Cheng Lian d9efe63ecd [SPARK-12799] Simplify various string output for expressions
This PR introduces several major changes:

1. Replacing `Expression.prettyString` with `Expression.sql`

   The `prettyString` method is mostly an internal, developer faced facility for debugging purposes, and shouldn't be exposed to users.

1. Using SQL-like representation as column names for selected fields that are not named expression (back-ticks and double quotes should be removed)

   Before, we were using `prettyString` as column names when possible, and sometimes the result column names can be weird.  Here are several examples:

   Expression         | `prettyString` | `sql`      | Note
   ------------------ | -------------- | ---------- | ---------------
   `a && b`           | `a && b`       | `a AND b`  |
   `a.getField("f")`  | `a[f]`         | `a.f`      | `a` is a struct

1. Adding trait `NonSQLExpression` extending from `Expression` for expressions that don't have a SQL representation (e.g. Scala UDF/UDAF and Java/Scala object expressions used for encoders)

   `NonSQLExpression.sql` may return an arbitrary user facing string representation of the expression.

Author: Cheng Lian <lian@databricks.com>

Closes #10757 from liancheng/spark-12799.simplify-expression-string-methods.
2016-02-21 22:53:15 +08:00
Davies Liu 7925071280 [SPARK-13306] [SQL] uncorrelated scalar subquery
A scalar subquery is a subquery that only generate single row and single column, could be used as part of expression. Uncorrelated scalar subquery means it does not has a reference to external table.

All the uncorrelated scalar subqueries will be executed during prepare() of SparkPlan.

The plans for query
```sql
select 1 + (select 2 + (select 3))
```
looks like this
```
== Parsed Logical Plan ==
'Project [unresolvedalias((1 + subquery#1),None)]
:- OneRowRelation$
+- 'Subquery subquery#1
   +- 'Project [unresolvedalias((2 + subquery#0),None)]
      :- OneRowRelation$
      +- 'Subquery subquery#0
         +- 'Project [unresolvedalias(3,None)]
            +- OneRowRelation$

== Analyzed Logical Plan ==
_c0: int
Project [(1 + subquery#1) AS _c0#4]
:- OneRowRelation$
+- Subquery subquery#1
   +- Project [(2 + subquery#0) AS _c0#3]
      :- OneRowRelation$
      +- Subquery subquery#0
         +- Project [3 AS _c0#2]
            +- OneRowRelation$

== Optimized Logical Plan ==
Project [(1 + subquery#1) AS _c0#4]
:- OneRowRelation$
+- Subquery subquery#1
   +- Project [(2 + subquery#0) AS _c0#3]
      :- OneRowRelation$
      +- Subquery subquery#0
         +- Project [3 AS _c0#2]
            +- OneRowRelation$

== Physical Plan ==
WholeStageCodegen
:  +- Project [(1 + subquery#1) AS _c0#4]
:     :- INPUT
:     +- Subquery subquery#1
:        +- WholeStageCodegen
:           :  +- Project [(2 + subquery#0) AS _c0#3]
:           :     :- INPUT
:           :     +- Subquery subquery#0
:           :        +- WholeStageCodegen
:           :           :  +- Project [3 AS _c0#2]
:           :           :     +- INPUT
:           :           +- Scan OneRowRelation[]
:           +- Scan OneRowRelation[]
+- Scan OneRowRelation[]
```

Author: Davies Liu <davies@databricks.com>

Closes #11190 from davies/scalar_subquery.
2016-02-20 21:01:51 -08:00
gatorsmile f88c641bc8 [SPARK-13310] [SQL] Resolve Missing Sorting Columns in Generate
```scala
// case 1: missing sort columns are resolvable if join is true
sql("SELECT explode(a) AS val, b FROM data WHERE b < 2 order by val, c")
// case 2: missing sort columns are not resolvable if join is false. Thus, issue an error message in this case
sql("SELECT explode(a) AS val FROM data order by val, c")
```

When sort columns are not in `Generate`, we can resolve them when `join` is equal to `true`. Still trying to add more test cases for the other `UnaryNode` types.

Could you review the changes? davies cloud-fan Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11198 from gatorsmile/missingInSort.
2016-02-20 13:53:23 -08:00
Reynold Xin 6624a588c1 Revert "[SPARK-12567] [SQL] Add aes_{encrypt,decrypt} UDFs"
This reverts commit 4f9a664818.
2016-02-19 22:44:20 -08:00
Kai Jiang 4f9a664818 [SPARK-12567] [SQL] Add aes_{encrypt,decrypt} UDFs
Author: Kai Jiang <jiangkai@gmail.com>

Closes #10527 from vectorijk/spark-12567.
2016-02-19 22:28:47 -08:00
gatorsmile ec7a1d6e42 [SPARK-12594] [SQL] Outer Join Elimination by Filter Conditions
Conversion of outer joins, if the predicates in filter conditions can restrict the result sets so that all null-supplying rows are eliminated.

- `full outer` -> `inner` if both sides have such predicates
- `left outer` -> `inner` if the right side has such predicates
- `right outer` -> `inner` if the left side has such predicates
- `full outer` -> `left outer` if only the left side has such predicates
- `full outer` -> `right outer` if only the right side has such predicates

If applicable, this can greatly improve the performance, since outer join is much slower than inner join, full outer join is much slower than left/right outer join.

The original PR is https://github.com/apache/spark/pull/10542

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

Closes #10567 from gatorsmile/outerJoinEliminationByFilterCond.
2016-02-19 22:27:10 -08:00
Sameer Agarwal 091f6a7830 [SPARK-13091][SQL] Rewrite/Propagate constraints for Aliases
This PR adds support for rewriting constraints if there are aliases in the query plan. For e.g., if there is a query of form `SELECT a, a AS b`, any constraints on `a` now also apply to `b`.

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

cc marmbrus

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11144 from sameeragarwal/alias.
2016-02-19 14:48:34 -08:00
Liang-Chi Hsieh c7c55637bf [SPARK-13384][SQL] Keep attribute qualifiers after dedup in Analyzer
JIRA: https://issues.apache.org/jira/browse/SPARK-13384

## What changes were proposed in this pull request?

When we de-duplicate attributes in Analyzer, we create new attributes. However, we don't keep original qualifiers. Some plans will be failed to analysed. We should keep original qualifiers in new attributes.

## How was the this patch tested?

Unit test is added.

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

Closes #11261 from viirya/keep-attr-qualifiers.
2016-02-19 12:22:22 -08:00
gatorsmile c776fce99b [SPARK-13380][SQL][DOCUMENT] Document Rand(seed) and Randn(seed) Return Indeterministic Results When Data Partitions are not fixed.
`rand` and `randn` functions with a `seed` argument are commonly used. Based on the common sense, the results of `rand` and `randn` should be deterministic if the `seed` parameter value is provided. For example, in MS SQL Server, it also has a function `rand`. Regarding the parameter `seed`, the description is like: ```Seed is an integer expression (tinyint, smallint, or int) that gives the seed value. If seed is not specified, the SQL Server Database Engine assigns a seed value at random. For a specified seed value, the result returned is always the same.```

Update: the current implementation is unable to generate deterministic results when the partitions are not fixed. This PR documents this issue in the function descriptions.

jkbradley hit an issue and provided an example in the following JIRA: https://issues.apache.org/jira/browse/SPARK-13333

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11232 from gatorsmile/randSeed.
2016-02-18 21:19:36 -08:00
Davies Liu 26f38bb83c [SPARK-13351][SQL] fix column pruning on Expand
Currently, the columns in projects of Expand that are not used by Aggregate are not pruned, this PR fix that.

Author: Davies Liu <davies@databricks.com>

Closes #11225 from davies/fix_pruning_expand.
2016-02-18 13:07:41 -08:00
Takuya UESHIN 19dc69de79 [SPARK-12976][SQL] Add LazilyGenerateOrdering and use it for RangePartitioner of Exchange.
Add `LazilyGenerateOrdering` to support generated ordering for `RangePartitioner` of `Exchange` instead of `InterpretedOrdering`.

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

Closes #10894 from ueshin/issues/SPARK-12976.
2016-02-16 10:54:44 -08:00
gatorsmile fee739f07b [SPARK-13221] [SQL] Fixing GroupingSets when Aggregate Functions Containing GroupBy Columns
Using GroupingSets will generate a wrong result when Aggregate Functions containing GroupBy columns.

This PR is to fix it. Since the code changes are very small. Maybe we also can merge it to 1.6

For example, the following query returns a wrong result:
```scala
sql("select course, sum(earnings) as sum from courseSales group by course, earnings" +
     " grouping sets((), (course), (course, earnings))" +
     " order by course, sum").show()
```
Before the fix, the results are like
```
[null,null]
[Java,null]
[Java,20000.0]
[Java,30000.0]
[dotNET,null]
[dotNET,5000.0]
[dotNET,10000.0]
[dotNET,48000.0]
```
After the fix, the results become correct:
```
[null,113000.0]
[Java,20000.0]
[Java,30000.0]
[Java,50000.0]
[dotNET,5000.0]
[dotNET,10000.0]
[dotNET,48000.0]
[dotNET,63000.0]
```

UPDATE:  This PR also deprecated the external column: GROUPING__ID.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11100 from gatorsmile/groupingSets.
2016-02-15 23:16:58 -08:00
Josh Rosen a8bbc4f50e [SPARK-12503][SPARK-12505] Limit pushdown in UNION ALL and OUTER JOIN
This patch adds a new optimizer rule for performing limit pushdown. Limits will now be pushed down in two cases:

- If a limit is on top of a `UNION ALL` operator, then a partition-local limit operator will be pushed to each of the union operator's children.
- If a limit is on top of an `OUTER JOIN` then a partition-local limit will be pushed to one side of the join. For `LEFT OUTER` and `RIGHT OUTER` joins, the limit will be pushed to the left and right side, respectively. For `FULL OUTER` join, we will only push limits when at most one of the inputs is already limited: if one input is limited we will push a smaller limit on top of it and if neither input is limited then we will limit the input which is estimated to be larger.

These optimizations were proposed previously by gatorsmile in #10451 and #10454, but those earlier PRs were closed and deferred for later because at that time Spark's physical `Limit` operator would trigger a full shuffle to perform global limits so there was a chance that pushdowns could actually harm performance by causing additional shuffles/stages. In #7334, we split the `Limit` operator into separate `LocalLimit` and `GlobalLimit` operators, so we can now push down only local limits (which don't require extra shuffles). This patch is based on both of gatorsmile's patches, with changes and simplifications due to partition-local-limiting.

When we push down the limit, we still keep the original limit in place, so we need a mechanism to ensure that the optimizer rule doesn't keep pattern-matching once the limit has been pushed down. In order to handle this, this patch adds a `maxRows` method to `SparkPlan` which returns the maximum number of rows that the plan can compute, then defines the pushdown rules to only push limits to children if the children's maxRows are greater than the limit's maxRows. This idea is carried over from #10451; see that patch for additional discussion.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11121 from JoshRosen/limit-pushdown-2.
2016-02-14 17:32:21 -08:00
Carson Wang 7cb4d74c98 [SPARK-13185][SQL] Reuse Calendar object in DateTimeUtils.StringToDate method to improve performance
The java `Calendar` object is expensive to create. I have a sub query like this `SELECT a, b, c FROM table UV WHERE (datediff(UV.visitDate, '1997-01-01')>=0 AND datediff(UV.visitDate, '2015-01-01')<=0))`

The table stores `visitDate` as String type and has 3 billion records. A `Calendar` object is created every time `DateTimeUtils.stringToDate` is called. By reusing the `Calendar` object, I saw about 20 seconds performance improvement for this stage.

Author: Carson Wang <carson.wang@intel.com>

Closes #11090 from carsonwang/SPARK-13185.
2016-02-14 16:00:20 -08:00
Sean Owen 388cd9ea8d [SPARK-13172][CORE][SQL] Stop using RichException.getStackTrace it is deprecated
Replace `getStackTraceString` with `Utils.exceptionString`

Author: Sean Owen <sowen@cloudera.com>

Closes #11182 from srowen/SPARK-13172.
2016-02-13 21:05:48 -08:00
Davies Liu 5b805df279 [SPARK-12705] [SQL] push missing attributes for Sort
The current implementation of ResolveSortReferences can only push one missing attributes into it's child, it failed to analyze TPCDS Q98, because of there are two missing attributes in that (one from Window, another from Aggregate).

Author: Davies Liu <davies@databricks.com>

Closes #11153 from davies/resolve_sort.
2016-02-12 09:34:18 -08:00
Liang-Chi Hsieh e31c80737b [SPARK-13277][SQL] ANTLR ignores other rule using the USING keyword
JIRA: https://issues.apache.org/jira/browse/SPARK-13277

There is an ANTLR warning during compilation:

    warning(200): org/apache/spark/sql/catalyst/parser/SparkSqlParser.g:938:7:
    Decision can match input such as "KW_USING Identifier" using multiple alternatives: 2, 3

    As a result, alternative(s) 3 were disabled for that input

This patch is to fix it.

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

Closes #11168 from viirya/fix-parser-using.
2016-02-11 21:09:44 +01:00
Nong Li 18bcbbdd84 [SPARK-13270][SQL] Remove extra new lines in whole stage codegen and include pipeline plan in comments.
Author: Nong Li <nong@databricks.com>

Closes #11155 from nongli/spark-13270.
2016-02-10 23:52:19 -08:00
gatorsmile e88bff1279 [SPARK-13235][SQL] Removed an Extra Distinct from the Plan when Using Union in SQL
Currently, the parser added two `Distinct` operators in the plan if we are using `Union` or `Union Distinct` in the SQL. This PR is to remove the extra `Distinct` from the plan.

For example, before the fix, the following query has a plan with two `Distinct`
```scala
sql("select * from t0 union select * from t0").explain(true)
```
```
== Parsed Logical Plan ==
'Project [unresolvedalias(*,None)]
+- 'Subquery u_2
   +- 'Distinct
      +- 'Project [unresolvedalias(*,None)]
         +- 'Subquery u_1
            +- 'Distinct
               +- 'Union
                  :- 'Project [unresolvedalias(*,None)]
                  :  +- 'UnresolvedRelation `t0`, None
                  +- 'Project [unresolvedalias(*,None)]
                     +- 'UnresolvedRelation `t0`, None

== Analyzed Logical Plan ==
id: bigint
Project [id#16L]
+- Subquery u_2
   +- Distinct
      +- Project [id#16L]
         +- Subquery u_1
            +- Distinct
               +- Union
                  :- Project [id#16L]
                  :  +- Subquery t0
                  :     +- Relation[id#16L] ParquetRelation
                  +- Project [id#16L]
                     +- Subquery t0
                        +- Relation[id#16L] ParquetRelation

== Optimized Logical Plan ==
Aggregate [id#16L], [id#16L]
+- Aggregate [id#16L], [id#16L]
   +- Union
      :- Project [id#16L]
      :  +- Relation[id#16L] ParquetRelation
      +- Project [id#16L]
         +- Relation[id#16L] ParquetRelation
```
After the fix, the plan is changed without the extra `Distinct` as follows:
```
== Parsed Logical Plan ==
'Project [unresolvedalias(*,None)]
+- 'Subquery u_1
   +- 'Distinct
      +- 'Union
         :- 'Project [unresolvedalias(*,None)]
         :  +- 'UnresolvedRelation `t0`, None
         +- 'Project [unresolvedalias(*,None)]
           +- 'UnresolvedRelation `t0`, None

== Analyzed Logical Plan ==
id: bigint
Project [id#17L]
+- Subquery u_1
   +- Distinct
      +- Union
        :- Project [id#16L]
        :  +- Subquery t0
        :     +- Relation[id#16L] ParquetRelation
        +- Project [id#16L]
          +- Subquery t0
          +- Relation[id#16L] ParquetRelation

== Optimized Logical Plan ==
Aggregate [id#17L], [id#17L]
+- Union
  :- Project [id#16L]
  :  +- Relation[id#16L] ParquetRelation
  +- Project [id#16L]
    +- Relation[id#16L] ParquetRelation
```

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11120 from gatorsmile/unionDistinct.
2016-02-11 08:40:27 +01:00
Herman van Hovell 1842c55d89 [SPARK-13276] Catch bad characters at the end of a Table Identifier/Expression string
The parser currently parses the following strings without a hitch:
* Table Identifier:
  * `a.b.c` should fail, but results in the following table identifier `a.b`
  * `table!#` should fail, but results in the following table identifier `table`
* Expression
  * `1+2 r+e` should fail, but results in the following expression `1 + 2`

This PR fixes this by adding terminated rules for both expression parsing and table identifier parsing.

cc cloud-fan (we discussed this in https://github.com/apache/spark/pull/10649) jayadevanmurali (this causes your PR https://github.com/apache/spark/pull/11051 to fail)

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

Closes #11159 from hvanhovell/SPARK-13276.
2016-02-11 08:30:58 +01:00
Davies Liu b5761d150b [SPARK-12706] [SQL] grouping() and grouping_id()
Grouping() returns a column is aggregated or not, grouping_id() returns the aggregation levels.

grouping()/grouping_id() could be used with window function, but does not work in having/sort clause, will be fixed by another PR.

The GROUPING__ID/grouping_id() in Hive is wrong (according to docs), we also did it wrongly, this PR change that to match the behavior in most databases (also the docs of Hive).

Author: Davies Liu <davies@databricks.com>

Closes #10677 from davies/grouping.
2016-02-10 20:13:38 -08:00
gatorsmile 663cc400f3 [SPARK-12725][SQL] Resolving Name Conflicts in SQL Generation and Name Ambiguity Caused by Internally Generated Expressions
Some analysis rules generate aliases or auxiliary attribute references with the same name but different expression IDs. For example, `ResolveAggregateFunctions` introduces `havingCondition` and `aggOrder`, and `DistinctAggregationRewriter` introduces `gid`.

This is OK for normal query execution since these attribute references get expression IDs. However, it's troublesome when converting resolved query plans back to SQL query strings since expression IDs are erased.

Here's an example Spark 1.6.0 snippet for illustration:
```scala
sqlContext.range(10).select('id as 'a, 'id as 'b).registerTempTable("t")
sqlContext.sql("SELECT SUM(a) FROM t GROUP BY a, b ORDER BY COUNT(a), COUNT(b)").explain(true)
```
The above code produces the following resolved plan:
```
== Analyzed Logical Plan ==
_c0: bigint
Project [_c0#101L]
+- Sort [aggOrder#102L ASC,aggOrder#103L ASC], true
   +- Aggregate [a#47L,b#48L], [(sum(a#47L),mode=Complete,isDistinct=false) AS _c0#101L,(count(a#47L),mode=Complete,isDistinct=false) AS aggOrder#102L,(count(b#48L),mode=Complete,isDistinct=false) AS aggOrder#103L]
      +- Subquery t
         +- Project [id#46L AS a#47L,id#46L AS b#48L]
            +- LogicalRDD [id#46L], MapPartitionsRDD[44] at range at <console>:26
```
Here we can see that both aggregate expressions in `ORDER BY` are extracted into an `Aggregate` operator, and both of them are named `aggOrder` with different expression IDs.

The solution is to automatically add the expression IDs into the attribute name for the Alias and AttributeReferences that are generated by Analyzer in SQL Generation.

In this PR, it also resolves another issue. Users could use the same name as the internally generated names. The duplicate names should not cause name ambiguity. When resolving the column, Catalyst should not pick the column that is internally generated.

Could you review the solution? marmbrus liancheng

I did not set the newly added flag for all the alias and attribute reference generated by Analyzers. Please let me know if I should do it? Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11050 from gatorsmile/namingConflicts.
2016-02-11 10:44:39 +08:00
Gábor Lipták 9269036d8c [SPARK-11565] Replace deprecated DigestUtils.shaHex call
Author: Gábor Lipták <gliptak@gmail.com>

Closes #9532 from gliptak/SPARK-11565.
2016-02-10 09:52:35 +00:00
Davies Liu 0e5ebac3c1 [SPARK-12950] [SQL] Improve lookup of BytesToBytesMap in aggregate
This PR improve the lookup of BytesToBytesMap by:

1. Generate code for calculate the hash code of grouping keys.

2. Do not use MemoryLocation, fetch the baseObject and offset for key and value directly (remove the indirection).

Author: Davies Liu <davies@databricks.com>

Closes #11010 from davies/gen_map.
2016-02-09 16:41:21 -08:00
Wenchen Fan 7fe4fe630a [SPARK-12888] [SQL] [FOLLOW-UP] benchmark the new hash expression
Adds the benchmark results as comments.

The codegen version is slower than the interpreted version for `simple` case becasue of 3 reasons:

1. codegen version use a more complex hash algorithm than interpreted version, i.e. `Murmur3_x86_32.hashInt` vs [simple multiplication and addition](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala#L153).
2. codegen version will write the hash value to a row first and then read it out. I tried to create a `GenerateHasher` that can generate code to return hash value directly and got about 60% speed up for the `simple` case, does it worth?
3. the row in `simple` case only has one int field, so the runtime reflection may be removed because of branch prediction, which makes the interpreted version faster.

The `array` case is also slow for similar reasons, e.g. array elements are of same type, so interpreted version can probably get rid of runtime reflection by branch prediction.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10917 from cloud-fan/hash-benchmark.
2016-02-09 13:06:36 -08:00
Wenchen Fan 8e4d15f707 [SPARK-13101][SQL] nullability of array type element should not fail analysis of encoder
nullability should only be considered as an optimization rather than part of the type system, so instead of failing analysis for mismatch nullability, we should pass analysis and add runtime null check.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11035 from cloud-fan/ignore-nullability.
2016-02-08 12:06:00 -08:00
Josh Rosen 06f0df6df2 [SPARK-8964] [SQL] Use Exchange to perform shuffle in Limit
This patch changes the implementation of the physical `Limit` operator so that it relies on the `Exchange` operator to perform data movement rather than directly using `ShuffledRDD`. In addition to improving efficiency, this lays the necessary groundwork for further optimization of limit, such as limit pushdown or whole-stage codegen.

At a high-level, this replaces the old physical `Limit` operator with two new operators, `LocalLimit` and `GlobalLimit`. `LocalLimit` performs per-partition limits, while `GlobalLimit` applies the final limit to a single partition; `GlobalLimit`'s declares that its `requiredInputDistribution` is `SinglePartition`, which will cause the planner to use an `Exchange` to perform the appropriate shuffles. Thus, a logical `Limit` appearing in the middle of a query plan will be expanded into `LocalLimit -> Exchange to one partition -> GlobalLimit`.

In the old code, calling `someDataFrame.limit(100).collect()` or `someDataFrame.take(100)` would actually skip the shuffle and use a fast-path which used `executeTake()` in order to avoid computing all partitions in case only a small number of rows were requested. This patch preserves this optimization by treating logical `Limit` operators specially when they appear as the terminal operator in a query plan: if a `Limit` is the final operator, then we will plan a special `CollectLimit` physical operator which implements the old `take()`-based logic.

In order to be able to match on operators only at the root of the query plan, this patch introduces a special `ReturnAnswer` logical operator which functions similar to `BroadcastHint`: this dummy operator is inserted at the root of the optimized logical plan before invoking the physical planner, allowing the planner to pattern-match on it.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7334 from JoshRosen/remove-copy-in-limit.
2016-02-08 11:38:21 -08:00
Jakob Odersky 6883a5120c [SPARK-13171][CORE] Replace future calls with Future
Trivial search-and-replace to eliminate deprecation warnings in Scala 2.11.
Also works with 2.10

Author: Jakob Odersky <jakob@odersky.com>

Closes #11085 from jodersky/SPARK-13171.
2016-02-05 19:00:12 -08:00
Wenchen Fan 1ed354a536 [SPARK-12939][SQL] migrate encoder resolution logic to Analyzer
https://issues.apache.org/jira/browse/SPARK-12939

Now we will catch `ObjectOperator` in `Analyzer` and resolve the `fromRowExpression/deserializer` inside it.  Also update the `MapGroups` and `CoGroup` to pass in `dataAttributes`, so that we can correctly resolve value deserializer(the `child.output` contains both groupking key and values, which may mess things up if they have same-name attribtues). End-to-end tests are added.

follow-ups:

* remove encoders from typed aggregate expression.
* completely remove resolve/bind in `ExpressionEncoder`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10852 from cloud-fan/bug.
2016-02-05 14:34:12 -08:00
Andrew Or bd38dd6f75 [SPARK-13079][SQL] InMemoryCatalog follow-ups
This patch incorporates review feedback from #11069, which is already merged.

Author: Andrew Or <andrew@databricks.com>

Closes #11080 from andrewor14/catalog-follow-ups.
2016-02-04 12:20:18 -08:00
Josh Rosen 33212cb9a1 [SPARK-13168][SQL] Collapse adjacent repartition operators
Spark SQL should collapse adjacent `Repartition` operators and only keep the last one.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11064 from JoshRosen/collapse-repartition.
2016-02-04 11:08:50 -08:00
Reynold Xin dee801adb7 [SPARK-12828][SQL] Natural join follow-up
This is a small addendum to #10762 to make the code more robust again future changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #11070 from rxin/SPARK-12828-natural-join.
2016-02-03 23:43:48 -08:00
Daoyuan Wang 0f81318ae2 [SPARK-12828][SQL] add natural join support
Jira:
https://issues.apache.org/jira/browse/SPARK-12828

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

Closes #10762 from adrian-wang/naturaljoin.
2016-02-03 21:05:53 -08:00
Andrew Or a64831124c [SPARK-13079][SQL] Extend and implement InMemoryCatalog
This is a step towards consolidating `SQLContext` and `HiveContext`.

This patch extends the existing Catalog API added in #10982 to include methods for handling table partitions. In particular, a partition is identified by `PartitionSpec`, which is just a `Map[String, String]`. The Catalog is still not used by anything yet, but its API is now more or less complete and an implementation is fully tested.

About 200 lines are test code.

Author: Andrew Or <andrew@databricks.com>

Closes #11069 from andrewor14/catalog.
2016-02-03 19:32:41 -08:00
Herman van Hovell 9dd2741ebe [SPARK-13157] [SQL] Support any kind of input for SQL commands.
The ```SparkSqlLexer``` currently swallows characters which have not been defined in the grammar. This causes problems with SQL commands, such as: ```add jar file:///tmp/ab/TestUDTF.jar```. In this example the `````` is swallowed.

This PR adds an extra Lexer rule to handle such input, and makes a tiny modification to the ```ASTNode```.

cc davies liancheng

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

Closes #11052 from hvanhovell/SPARK-13157.
2016-02-03 12:31:30 -08:00
Sameer Agarwal 138c300f97 [SPARK-12957][SQL] Initial support for constraint propagation in SparkSQL
Based on the semantics of a query, we can derive a number of data constraints on output of each (logical or physical) operator. For instance, if a filter defines `‘a > 10`, we know that the output data of this filter satisfies 2 constraints:

1. `‘a > 10`
2. `isNotNull(‘a)`

This PR proposes a possible way of keeping track of these constraints and propagating them in the logical plan, which can then help us build more advanced optimizations (such as pruning redundant filters, optimizing joins, among others). We define constraints as a set of (implicitly conjunctive) expressions. For e.g., if a filter operator has constraints = `Set(‘a > 10, ‘b < 100)`, it’s implied that the outputs satisfy both individual constraints (i.e., `‘a > 10` AND `‘b < 100`).

Design Document: https://docs.google.com/a/databricks.com/document/d/1WQRgDurUBV9Y6CWOBS75PQIqJwT-6WftVa18xzm7nCo/edit?usp=sharing

Author: Sameer Agarwal <sameer@databricks.com>

Closes #10844 from sameeragarwal/constraints.
2016-02-02 22:22:50 -08:00
Davies Liu e86f8f63bf [SPARK-13147] [SQL] improve readability of generated code
1. try to avoid the suffix (unique id)
2. remove the comment if there is no code generated.
3. re-arrange the order of functions
4. trop the new line for inlined blocks.

Author: Davies Liu <davies@databricks.com>

Closes #11032 from davies/better_suffix.
2016-02-02 22:13:10 -08:00
Wenchen Fan 672032d0ab [SPARK-13020][SQL][TEST] fix random generator for map type
when we generate map, we first randomly pick a length, then create a seq of key value pair with the expected length, and finally call `toMap`. However, `toMap` will remove all duplicated keys, which makes the actual map size much less than we expected.

This PR fixes this problem by put keys in a set first, to guarantee we have enough keys to build a map with expected length.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10930 from cloud-fan/random-generator.
2016-02-03 08:26:35 +08:00
Kevin (Sangwoo) Kim b377b03531 [DOCS] Update StructType.scala
The example will throw error like
<console>:20: error: not found: value StructType

Need to add this line:
import org.apache.spark.sql.types._

Author: Kevin (Sangwoo) Kim <sangwookim.me@gmail.com>

Closes #10141 from swkimme/patch-1.
2016-02-02 13:24:21 -08:00
Davies Liu be5dd881f1 [SPARK-12913] [SQL] Improve performance of stat functions
As benchmarked and discussed here: https://github.com/apache/spark/pull/10786/files#r50038294, benefits from codegen, the declarative aggregate function could be much faster than imperative one.

Author: Davies Liu <davies@databricks.com>

Closes #10960 from davies/stddev.
2016-02-02 11:50:14 -08:00
Daoyuan Wang 358300c795 [SPARK-13056][SQL] map column would throw NPE if value is null
Jira:
https://issues.apache.org/jira/browse/SPARK-13056

Create a map like
{ "a": "somestring", "b": null}
Query like
SELECT col["b"] FROM t1;
NPE would be thrown.

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

Closes #10964 from adrian-wang/npewriter.
2016-02-02 11:09:40 -08:00
Reynold Xin be7a2fc071 [SPARK-13078][SQL] API and test cases for internal catalog
This pull request creates an internal catalog API. The creation of this API is the first step towards consolidating SQLContext and HiveContext. I envision we will have two different implementations in Spark 2.0: (1) a simple in-memory implementation, and (2) an implementation based on the current HiveClient (ClientWrapper).

I took a look at what Hive's internal metastore interface/implementation, and then created this API based on it. I believe this is the minimal set needed in order to achieve all the needed functionality.

Author: Reynold Xin <rxin@databricks.com>

Closes #10982 from rxin/SPARK-13078.
2016-02-01 14:11:52 -08:00
Nong Li 064b029c6a [SPARK-13043][SQL] Implement remaining catalyst types in ColumnarBatch.
This includes: float, boolean, short, decimal and calendar interval.

Decimal is mapped to long or byte array depending on the size and calendar
interval is mapped to a struct of int and long.

The only remaining type is map. The schema mapping is straightforward but
we might want to revisit how we deal with this in the rest of the execution
engine.

Author: Nong Li <nong@databricks.com>

Closes #10961 from nongli/spark-13043.
2016-02-01 13:56:14 -08:00
gatorsmile 8f26eb5ef6 [SPARK-12705][SPARK-10777][SQL] Analyzer Rule ResolveSortReferences
JIRA: https://issues.apache.org/jira/browse/SPARK-12705

**Scope:**
This PR is a general fix for sorting reference resolution when the child's `outputSet` does not have the order-by attributes (called, *missing attributes*):
  - UnaryNode support is limited to `Project`, `Window`, `Aggregate`, `Distinct`, `Filter`, `RepartitionByExpression`.
  - We will not try to resolve the missing references inside a subquery, unless the outputSet of this subquery contains it.

**General Reference Resolution Rules:**
  - Jump over the nodes with the following types: `Distinct`, `Filter`, `RepartitionByExpression`. Do not need to add missing attributes. The reason is their `outputSet` is decided by their `inputSet`, which is the `outputSet` of their children.
  - Group-by expressions in `Aggregate`: missing order-by attributes are not allowed to be added into group-by expressions since it will change the query result. Thus, in RDBMS, it is not allowed.
  - Aggregate expressions in `Aggregate`: if the group-by expressions in `Aggregate` contains the missing attributes but aggregate expressions do not have it, just add them into the aggregate expressions. This can resolve the analysisExceptions thrown by the three TCPDS queries.
  - `Project` and `Window` are special. We just need to add the missing attributes to their `projectList`.

**Implementation:**
  1. Traverse the whole tree in a pre-order manner to find all the resolvable missing order-by attributes.
  2. Traverse the whole tree in a post-order manner to add the found missing order-by attributes to the node if their `inputSet` contains the attributes.
  3. If the origins of the missing order-by attributes are different nodes, each pass only resolves the missing attributes that are from the same node.

**Risk:**
Low. This rule will be trigger iff ```!s.resolved && child.resolved``` is true. Thus, very few cases are affected.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10678 from gatorsmile/sortWindows.
2016-02-01 11:57:13 -08:00
gatorsmile 33c8a490f7 [SPARK-12989][SQL] Delaying Alias Cleanup after ExtractWindowExpressions
JIRA: https://issues.apache.org/jira/browse/SPARK-12989

In the rule `ExtractWindowExpressions`, we simply replace alias by the corresponding attribute. However, this will cause an issue exposed by the following case:

```scala
val data = Seq(("a", "b", "c", 3), ("c", "b", "a", 3)).toDF("A", "B", "C", "num")
  .withColumn("Data", struct("A", "B", "C"))
  .drop("A")
  .drop("B")
  .drop("C")

val winSpec = Window.partitionBy("Data.A", "Data.B").orderBy($"num".desc)
data.select($"*", max("num").over(winSpec) as "max").explain(true)
```
In this case, both `Data.A` and `Data.B` are `alias` in `WindowSpecDefinition`. If we replace these alias expression by their alias names, we are unable to know what they are since they will not be put in `missingExpr` too.

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

Closes #10963 from gatorsmile/seletStarAfterColDrop.
2016-02-01 11:22:02 -08:00
Wenchen Fan c1da4d421a [SPARK-13093] [SQL] improve null check in nullSafeCodeGen for unary, binary and ternary expression
The current implementation is sub-optimal:

* If an expression is always nullable, e.g. `Unhex`, we can still remove null check for children if they are not nullable.
* If an expression has some non-nullable children, we can still remove null check for these children and keep null check for others.

This PR improves this by making the null check elimination more fine-grained.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10987 from cloud-fan/null-check.
2016-01-31 22:43:03 -08:00
Liang-Chi Hsieh 0e6d92d042 [SPARK-12689][SQL] Migrate DDL parsing to the newly absorbed parser
JIRA: https://issues.apache.org/jira/browse/SPARK-12689

DDLParser processes three commands: createTable, describeTable and refreshTable.
This patch migrates the three commands to newly absorbed parser.

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

Closes #10723 from viirya/migrate-ddl-describe.
2016-01-30 23:05:29 -08:00
Cheng Lian a1303de0a0 [SPARK-13070][SQL] Better error message when Parquet schema merging fails
Make sure we throw better error messages when Parquet schema merging fails.

Author: Cheng Lian <lian@databricks.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #10979 from viirya/schema-merging-failure-message.
2016-01-30 23:02:49 -08:00
wangyang de28371998 [SPARK-13100][SQL] improving the performance of stringToDate method in DateTimeUtils.scala
In jdk1.7 TimeZone.getTimeZone() is synchronized, so use an instance variable to hold an GMT TimeZone object instead of instantiate it every time.

Author: wangyang <wangyang@haizhi.com>

Closes #10994 from wangyang1992/datetimeUtil.
2016-01-30 15:20:57 -08:00
Wenchen Fan dab246f7e4 [SPARK-13098] [SQL] remove GenericInternalRowWithSchema
This class is only used for serialization of Python DataFrame. However, we don't require internal row there, so `GenericRowWithSchema` can also do the job.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10992 from cloud-fan/python.
2016-01-29 23:37:51 -08:00
Davies Liu e6a02c66d5 [SPARK-12914] [SQL] generate aggregation with grouping keys
This PR add support for grouping keys for generated TungstenAggregate.

Spilling and performance improvements for BytesToBytesMap will be done by followup PR.

Author: Davies Liu <davies@databricks.com>

Closes #10855 from davies/gen_keys.
2016-01-29 20:16:11 -08:00
gatorsmile 5f686cc8b7 [SPARK-12656] [SQL] Implement Intersect with Left-semi Join
Our current Intersect physical operator simply delegates to RDD.intersect. We should remove the Intersect physical operator and simply transform a logical intersect into a semi-join with distinct. This way, we can take advantage of all the benefits of join implementations (e.g. managed memory, code generation, broadcast joins).

After a search, I found one of the mainstream RDBMS did the same. In their query explain, Intersect is replaced by Left-semi Join. Left-semi Join could help outer-join elimination in Optimizer, as shown in the PR: https://github.com/apache/spark/pull/10566

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

Closes #10630 from gatorsmile/IntersectBySemiJoin.
2016-01-29 11:22:12 -08:00
Wenchen Fan c5f745ede0 [SPARK-13072] [SQL] simplify and improve murmur3 hash expression codegen
simplify(remove several unnecessary local variables) the generated code of hash expression, and avoid null check if possible.

generated code comparison for `hash(int, double, string, array<string>)`:
**before:**
```
  public UnsafeRow apply(InternalRow i) {
    /* hash(input[0, int],input[1, double],input[2, string],input[3, array<int>],42) */
    int value1 = 42;
    /* input[0, int] */
    int value3 = i.getInt(0);
    if (!false) {
      value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashInt(value3, value1);
    }
    /* input[1, double] */
    double value5 = i.getDouble(1);
    if (!false) {
      value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(Double.doubleToLongBits(value5), value1);
    }
    /* input[2, string] */
    boolean isNull6 = i.isNullAt(2);
    UTF8String value7 = isNull6 ? null : (i.getUTF8String(2));
    if (!isNull6) {
      value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value7.getBaseObject(), value7.getBaseOffset(), value7.numBytes(), value1);
    }
    /* input[3, array<int>] */
    boolean isNull8 = i.isNullAt(3);
    ArrayData value9 = isNull8 ? null : (i.getArray(3));
    if (!isNull8) {
      int result10 = value1;
      for (int index11 = 0; index11 < value9.numElements(); index11++) {
        if (!value9.isNullAt(index11)) {
          final int element12 = value9.getInt(index11);
          result10 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashInt(element12, result10);
        }
      }
      value1 = result10;
    }
  }
```
**after:**
```
  public UnsafeRow apply(InternalRow i) {
    /* hash(input[0, int],input[1, double],input[2, string],input[3, array<int>],42) */
    int value1 = 42;
    /* input[0, int] */
    int value3 = i.getInt(0);
    value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashInt(value3, value1);
    /* input[1, double] */
    double value5 = i.getDouble(1);
    value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(Double.doubleToLongBits(value5), value1);
    /* input[2, string] */
    boolean isNull6 = i.isNullAt(2);
    UTF8String value7 = isNull6 ? null : (i.getUTF8String(2));

    if (!isNull6) {
      value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value7.getBaseObject(), value7.getBaseOffset(), value7.numBytes(), value1);
    }

    /* input[3, array<int>] */
    boolean isNull8 = i.isNullAt(3);
    ArrayData value9 = isNull8 ? null : (i.getArray(3));
    if (!isNull8) {
      for (int index10 = 0; index10 < value9.numElements(); index10++) {
        final int element11 = value9.getInt(index10);
        value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashInt(element11, value1);
      }
    }

    rowWriter14.write(0, value1);
    return result12;
  }
```

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10974 from cloud-fan/codegen.
2016-01-29 10:24:23 -08:00
Davies Liu 55561e7693 [SPARK-13031][SQL] cleanup codegen and improve test coverage
1. enable whole stage codegen during tests even there is only one operator supports that.
2. split doProduce() into two APIs: upstream() and doProduce()
3. generate prefix for fresh names of each operator
4. pass UnsafeRow to parent directly (avoid getters and create UnsafeRow again)
5. fix bugs and tests.

This PR re-open #10944 and fix the bug.

Author: Davies Liu <davies@databricks.com>

Closes #10977 from davies/gen_refactor.
2016-01-29 01:59:59 -08:00
Wenchen Fan 721ced28b5 [SPARK-13067] [SQL] workaround for a weird scala reflection problem
A simple workaround to avoid getting parameter types when convert a
logical plan to json.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10970 from cloud-fan/reflection.
2016-01-28 22:43:03 -08:00
Liang-Chi Hsieh 66449b8dcd [SPARK-12968][SQL] Implement command to set current database
JIRA: https://issues.apache.org/jira/browse/SPARK-12968

Implement command to set current database.

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

Closes #10916 from viirya/ddl-use-database.
2016-01-28 22:20:52 -08:00
Davies Liu b9dfdcc63b Revert "[SPARK-13031] [SQL] cleanup codegen and improve test coverage"
This reverts commit cc18a71992.
2016-01-28 17:01:12 -08:00
Liang-Chi Hsieh 4637fc08a3 [SPARK-11955][SQL] Mark optional fields in merging schema for safely pushdowning filters in Parquet
JIRA: https://issues.apache.org/jira/browse/SPARK-11955

Currently we simply skip pushdowning filters in parquet if we enable schema merging.

However, we can actually mark particular fields in merging schema for safely pushdowning filters in parquet.

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

Closes #9940 from viirya/safe-pushdown-parquet-filters.
2016-01-28 16:25:21 -08:00
Davies Liu cc18a71992 [SPARK-13031] [SQL] cleanup codegen and improve test coverage
1. enable whole stage codegen during tests even there is only one operator supports that.
2. split doProduce() into two APIs: upstream() and doProduce()
3. generate prefix for fresh names of each operator
4. pass UnsafeRow to parent directly (avoid getters and create UnsafeRow again)
5. fix bugs and tests.

Author: Davies Liu <davies@databricks.com>

Closes #10944 from davies/gen_refactor.
2016-01-28 13:51:55 -08:00
Herman van Hovell ef96cd3c52 [SPARK-12865][SPARK-12866][SQL] Migrate SparkSQLParser/ExtendedHiveQlParser commands to new Parser
This PR moves all the functionality provided by the SparkSQLParser/ExtendedHiveQlParser to the new Parser hierarchy (SparkQl/HiveQl). This also improves the current SET command parsing: the current implementation swallows ```set role ...``` and ```set autocommit ...``` commands, this PR respects these commands (and passes them on to Hive).

This PR and https://github.com/apache/spark/pull/10723 end the use of Parser-Combinator parsers for SQL parsing. As a result we can also remove the ```AbstractSQLParser``` in Catalyst.

The PR is marked WIP as long as it doesn't pass all tests.

cc rxin viirya winningsix (this touches https://github.com/apache/spark/pull/10144)

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

Closes #10905 from hvanhovell/SPARK-12866.
2016-01-27 13:45:00 -08:00
Jason Lee edd473751b [SPARK-10847][SQL][PYSPARK] Pyspark - DataFrame - Optional Metadata with None triggers cryptic failure
The error message is now changed from "Do not support type class scala.Tuple2." to "Do not support type class org.json4s.JsonAST$JNull$" to be more informative about what is not supported. Also, StructType metadata now handles JNull correctly, i.e., {'a': None}. test_metadata_null is added to tests.py to show the fix works.

Author: Jason Lee <cjlee@us.ibm.com>

Closes #8969 from jasoncl/SPARK-10847.
2016-01-27 09:55:10 -08:00
Nong Li 555127387a [SPARK-12854][SQL] Implement complex types support in ColumnarBatch
This patch adds support for complex types for ColumnarBatch. ColumnarBatch supports structs
and arrays. There is a simple mapping between the richer catalyst types to these two. Strings
are treated as an array of bytes.

ColumnarBatch will contain a column for each node of the schema. Non-complex schemas consists
of just leaf nodes. Structs represent an internal node with one child for each field. Arrays
are internal nodes with one child. Structs just contain nullability. Arrays contain offsets
and lengths into the child array. This structure is able to handle arbitrary nesting. It has
the key property that we maintain columnar throughout and that primitive types are only stored
in the leaf nodes and contiguous across rows. For example, if the schema is
```
array<array<int>>
```
There are three columns in the schema. The internal nodes each have one children. The leaf node contains all the int data stored consecutively.

As part of this, this patch adds append APIs in addition to the Put APIs (e.g. putLong(rowid, v)
vs appendLong(v)). These APIs are necessary when the batch contains variable length elements.
The vectors are not fixed length and will grow as necessary. This should make the usage a lot
simpler for the writer.

Author: Nong Li <nong@databricks.com>

Closes #10820 from nongli/spark-12854.
2016-01-26 17:34:01 -08:00
Cheng Lian 83507fea9f [SQL] Minor Scaladoc format fix
Otherwise the `^` character is always marked as error in IntelliJ since it represents an unclosed superscript markup tag.

Author: Cheng Lian <lian@databricks.com>

Closes #10926 from liancheng/agg-doc-fix.
2016-01-26 14:29:29 -08:00
Wenchen Fan be375fcbd2 [SPARK-12879] [SQL] improve the unsafe row writing framework
As we begin to use unsafe row writing framework(`BufferHolder` and `UnsafeRowWriter`) in more and more places(`UnsafeProjection`, `UnsafeRowParquetRecordReader`, `GenerateColumnAccessor`, etc.), we should add more doc to it and make it easier to use.

This PR abstract the technique used in `UnsafeRowParquetRecordReader`: avoid unnecessary operatition as more as possible. For example, do not always point the row to the buffer at the end, we only need to update the size of row. If all fields are of primitive type, we can even save the row size updating. Then we can apply this technique to more places easily.

a local benchmark shows `UnsafeProjection` is up to 1.7x faster after this PR:
**old version**
```
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
unsafe projection:                 Avg Time(ms)    Avg Rate(M/s)  Relative Rate
-------------------------------------------------------------------------------
single long                             2616.04           102.61         1.00 X
single nullable long                    3032.54            88.52         0.86 X
primitive types                         9121.05            29.43         0.29 X
nullable primitive types               12410.60            21.63         0.21 X
```

**new version**
```
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
unsafe projection:                 Avg Time(ms)    Avg Rate(M/s)  Relative Rate
-------------------------------------------------------------------------------
single long                             1533.34           175.07         1.00 X
single nullable long                    2306.73           116.37         0.66 X
primitive types                         8403.93            31.94         0.18 X
nullable primitive types               12448.39            21.56         0.12 X
```

For single non-nullable long(the best case), we can have about 1.7x speed up. Even it's nullable, we can still have 1.3x speed up. For other cases, it's not such a boost as the saved operations only take a little proportion of the whole process.  The benchmark code is included in this PR.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10809 from cloud-fan/unsafe-projection.
2016-01-25 16:23:59 -08:00
Andy Grove d8e480521e [SPARK-12932][JAVA API] improved error message for java type inference failure
Author: Andy Grove <andygrove73@gmail.com>

Closes #10865 from andygrove/SPARK-12932.
2016-01-25 09:22:10 +00:00
Reynold Xin 423783a08b [SPARK-12904][SQL] Strength reduction for integral and decimal literal comparisons
This pull request implements strength reduction for comparing integral expressions and decimal literals, which is more common now because we switch to parsing fractional literals as decimal types (rather than doubles). I added the rules to the existing DecimalPrecision rule with some refactoring to simplify the control flow. I also moved DecimalPrecision rule into its own file due to the growing size.

Author: Reynold Xin <rxin@databricks.com>

Closes #10882 from rxin/SPARK-12904-1.
2016-01-23 12:13:05 -08:00
Herman van Hovell 1017327930 [SPARK-12848][SQL] Change parsed decimal literal datatype from Double to Decimal
The current parser turns a decimal literal, for example ```12.1```, into a Double. The problem with this approach is that we convert an exact literal into a non-exact ```Double```. The PR changes this behavior, a Decimal literal is now converted into an extact ```BigDecimal```.

The behavior for scientific decimals, for example ```12.1e01```, is unchanged. This will be converted into a Double.

This PR replaces the ```BigDecimal``` literal by a ```Double``` literal, because the ```BigDecimal``` is the default now. You can use the double literal by appending a 'D' to the value, for instance: ```3.141527D```

cc davies rxin

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

Closes #10796 from hvanhovell/SPARK-12848.
2016-01-20 15:13:01 -08:00
Wenchen Fan f3934a8d65 [SPARK-12888][SQL] benchmark the new hash expression
Benchmark it on 4 different schemas, the result:
```
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
Hash For simple:                   Avg Time(ms)    Avg Rate(M/s)  Relative Rate
-------------------------------------------------------------------------------
interpreted version                       31.47           266.54         1.00 X
codegen version                           64.52           130.01         0.49 X
```

```
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
Hash For normal:                   Avg Time(ms)    Avg Rate(M/s)  Relative Rate
-------------------------------------------------------------------------------
interpreted version                     4068.11             0.26         1.00 X
codegen version                         1175.92             0.89         3.46 X
```

```
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
Hash For array:                    Avg Time(ms)    Avg Rate(M/s)  Relative Rate
-------------------------------------------------------------------------------
interpreted version                     9276.70             0.06         1.00 X
codegen version                        14762.23             0.04         0.63 X
```

```
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
Hash For map:                      Avg Time(ms)    Avg Rate(M/s)  Relative Rate
-------------------------------------------------------------------------------
interpreted version                    58869.79             0.01         1.00 X
codegen version                         9285.36             0.06         6.34 X
```

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10816 from cloud-fan/hash-benchmark.
2016-01-20 15:08:27 -08:00
gatorsmile 8f90c15187 [SPARK-12616][SQL] Making Logical Operator Union Support Arbitrary Number of Children
The existing `Union` logical operator only supports two children. Thus, adding a new logical operator `Unions` which can have arbitrary number of children to replace the existing one.

`Union` logical plan is a binary node. However, a typical use case for union is to union a very large number of input sources (DataFrames, RDDs, or files). It is not uncommon to union hundreds of thousands of files. In this case, our optimizer can become very slow due to the large number of logical unions. We should change the Union logical plan to support an arbitrary number of children, and add a single rule in the optimizer to collapse all adjacent `Unions` into a single `Unions`. Note that this problem doesn't exist in physical plan, because the physical `Unions` already supports arbitrary number of children.

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

Closes #10577 from gatorsmile/unionAllMultiChildren.
2016-01-20 14:59:30 -08:00
Davies Liu 8e4f894e98 [SPARK-12881] [SQL] subexpress elimination in mutable projection
Author: Davies Liu <davies@databricks.com>

Closes #10814 from davies/mutable_subexpr.
2016-01-20 10:02:40 -08:00
Reynold Xin 753b194511 [SPARK-12912][SQL] Add a test suite for EliminateSubQueries
Also updated documentation to explain why ComputeCurrentTime and EliminateSubQueries are in the optimizer rather than analyzer.

Author: Reynold Xin <rxin@databricks.com>

Closes #10837 from rxin/optimizer-analyzer-comment.
2016-01-20 00:00:28 -08:00
Reynold Xin 3e84ef0a54 [SPARK-12770][SQL] Implement rules for branch elimination for CaseWhen
The three optimization cases are:

1. If the first branch's condition is a true literal, remove the CaseWhen and use the value from that branch.
2. If a branch's condition is a false or null literal, remove that branch.
3. If only the else branch is left, remove the CaseWhen and use the value from the else branch.

Author: Reynold Xin <rxin@databricks.com>

Closes #10827 from rxin/SPARK-12770.
2016-01-19 16:14:41 -08:00
Jakob Odersky c78e2080e0 [SPARK-12816][SQL] De-alias type when generating schemas
Call `dealias` on local types to fix schema generation for abstract type members, such as

```scala
type KeyValue = (Int, String)
```

Add simple test

Author: Jakob Odersky <jodersky@gmail.com>

Closes #10749 from jodersky/aliased-schema.
2016-01-19 12:31:03 -08:00
gatorsmile b72e01e821 [SPARK-12867][SQL] Nullability of Intersect can be stricter
JIRA: https://issues.apache.org/jira/browse/SPARK-12867

When intersecting one nullable column with one non-nullable column, the result will not contain any null. Thus, we can make nullability of `intersect` stricter.

liancheng Could you please check if the code changes are appropriate? Also added test cases to verify the results. Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10812 from gatorsmile/nullabilityIntersect.
2016-01-19 11:35:58 -08:00
Reynold Xin 39ac56fc60 [SPARK-12889][SQL] Rename ParserDialect -> ParserInterface.
Based on discussions in #10801, I'm submitting a pull request to rename ParserDialect to ParserInterface.

Author: Reynold Xin <rxin@databricks.com>

Closes #10817 from rxin/SPARK-12889.
2016-01-18 17:10:32 -08:00
Wenchen Fan 4f11e3f2aa [SPARK-12841][SQL] fix cast in filter
In SPARK-10743 we wrap cast with `UnresolvedAlias` to give `Cast` a better alias if possible. However, for cases like `filter`, the `UnresolvedAlias` can't be resolved and actually we don't need a better alias for this case.  This PR move the cast wrapping logic to `Column.named` so that we will only do it when we need a alias name.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10781 from cloud-fan/bug.
2016-01-18 14:15:27 -08:00
Reynold Xin 38c3c0e31a [SPARK-12855][SQL] Remove parser dialect developer API
This pull request removes the public developer parser API for external parsers. Given everything a parser depends on (e.g. logical plans and expressions) are internal and not stable, external parsers will break with every release of Spark. It is a bad idea to create the illusion that Spark actually supports pluggable parsers. In addition, this also reduces incentives for 3rd party projects to contribute parse improvements back to Spark.

Author: Reynold Xin <rxin@databricks.com>

Closes #10801 from rxin/SPARK-12855.
2016-01-18 13:55:42 -08:00
Reynold Xin 44fcf992aa [SPARK-12873][SQL] Add more comment in HiveTypeCoercion for type widening
I was reading this part of the analyzer code again and got confused by the difference between findWiderTypeForTwo and findTightestCommonTypeOfTwo.

I also simplified WidenSetOperationTypes to make it a lot simpler. The easiest way to review this one is to just read the original code, and the new code. The logic is super simple.

Author: Reynold Xin <rxin@databricks.com>

Closes #10802 from rxin/SPARK-12873.
2016-01-18 11:08:44 -08:00
Wenchen Fan cede7b2a11 [SPARK-12860] [SQL] speed up safe projection for primitive types
The idea is simple, use `SpecificMutableRow` instead of `GenericMutableRow` as result row for safe projection.

A simple benchmark shows about 1.5x speed up for primitive types, code: https://gist.github.com/cloud-fan/fa77713ccebf0823b2ab#file-safeprojectionbenchmark-scala

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10790 from cloud-fan/safe-projection.
2016-01-17 09:11:43 -08:00
Davies Liu 3c0d2365d5 [SPARK-12796] [SQL] Whole stage codegen
This is the initial work for whole stage codegen, it support Projection/Filter/Range, we will continue work on this to support more physical operators.

A micro benchmark show that a query with range, filter and projection could be 3X faster then before.

It's turned on by default. For a tree that have at least two chained plans, a WholeStageCodegen will be inserted into it, for example, the following plan
```
Limit 10
+- Project [(id#5L + 1) AS (id + 1)#6L]
   +- Filter ((id#5L & 1) = 1)
      +- Range 0, 1, 4, 10, [id#5L]
```
will be translated into
```
Limit 10
+- WholeStageCodegen
      +- Project [(id#1L + 1) AS (id + 1)#2L]
         +- Filter ((id#1L & 1) = 1)
            +- Range 0, 1, 4, 10, [id#1L]
```

Here is the call graph to generate Java source for A and B (A  support codegen, but B does not):

```
  *   WholeStageCodegen       Plan A               FakeInput        Plan B
  * =========================================================================
  *
  * -> execute()
  *     |
  *  doExecute() -------->   produce()
  *                             |
  *                          doProduce()  -------> produce()
  *                                                   |
  *                                                doProduce() ---> execute()
  *                                                   |
  *                                                consume()
  *                          doConsume()  ------------|
  *                             |
  *  doConsume()  <-----    consume()
```

A SparkPlan that support codegen need to implement doProduce() and doConsume():

```
def doProduce(ctx: CodegenContext): (RDD[InternalRow], String)
def doConsume(ctx: CodegenContext, child: SparkPlan, input: Seq[ExprCode]): String
```

Author: Davies Liu <davies@databricks.com>

Closes #10735 from davies/whole2.
2016-01-16 10:29:27 -08:00
Wenchen Fan 2f7d0b68a2 [SPARK-12856] [SQL] speed up hashCode of unsafe array
We iterate the bytes to calculate hashCode before, but now we have `Murmur3_x86_32.hashUnsafeBytes` that don't require the bytes to be word algned, we should use that instead.

A simple benchmark shows it's about 3 X faster, benchmark code: https://gist.github.com/cloud-fan/fa77713ccebf0823b2ab#file-arrayhashbenchmark-scala

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10784 from cloud-fan/array-hashcode.
2016-01-16 00:38:17 -08:00
Davies Liu 242efb7546 [SPARK-12840] [SQL] Support passing arbitrary objects (not just expressions) into code generated classes
This is a refactor to support codegen for aggregation and broadcast join.

Author: Davies Liu <davies@databricks.com>

Closes #10777 from davies/rename2.
2016-01-15 19:07:42 -08:00
Herman van Hovell 7cd7f22025 [SPARK-12575][SQL] Grammar parity with existing SQL parser
In this PR the new CatalystQl parser stack reaches grammar parity with the old Parser-Combinator based SQL Parser. This PR also replaces all uses of the old Parser, and removes it from the code base.

Although the existing Hive and SQL parser dialects were mostly the same, some kinks had to be worked out:
- The SQL Parser allowed syntax like ```APPROXIMATE(0.01) COUNT(DISTINCT a)```. In order to make this work we needed to hardcode approximate operators in the parser, or we would have to create an approximate expression. ```APPROXIMATE_COUNT_DISTINCT(a, 0.01)``` would also do the job and is much easier to maintain. So, this PR **removes** this keyword.
- The old SQL Parser supports ```LIMIT``` clauses in nested queries. This is **not supported** anymore. See https://github.com/apache/spark/pull/10689 for the rationale for this.
- Hive has a charset name char set literal combination it supports, for instance the following expression ```_ISO-8859-1 0x4341464562616265``` would yield this string: ```CAFEbabe```. Hive will only allow charset names to start with an underscore. This is quite annoying in spark because as soon as you use a tuple names will start with an underscore. In this PR we **remove** this feature from the parser. It would be quite easy to implement such a feature as an Expression later on.
- Hive and the SQL Parser treat decimal literals differently. Hive will turn any decimal into a ```Double``` whereas the SQL Parser would convert a non-scientific decimal into a ```BigDecimal```, and would turn a scientific decimal into a Double. We follow Hive's behavior here. The new parser supports a big decimal literal, for instance: ```81923801.42BD```, which can be used when a big decimal is needed.

cc rxin viirya marmbrus yhuai cloud-fan

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

Closes #10745 from hvanhovell/SPARK-12575-2.
2016-01-15 15:19:10 -08:00
Wenchen Fan 3f1c58d60b [SQL][MINOR] BoundReference do not need to be NamedExpression
We made it a `NamedExpression` to workaroud some hacky cases long time ago, and now seems it's safe to remove it.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10765 from cloud-fan/minor.
2016-01-15 14:20:22 -08:00
Davies Liu c5e7076da7 [MINOR] [SQL] GeneratedExpressionCode -> ExprCode
GeneratedExpressionCode is too long

Author: Davies Liu <davies@databricks.com>

Closes #10767 from davies/renaming.
2016-01-15 08:26:20 -08:00
Michael Armbrust cc7af86afd [SPARK-12813][SQL] Eliminate serialization for back to back operations
The goal of this PR is to eliminate unnecessary translations when there are back-to-back `MapPartitions` operations.  In order to achieve this I also made the following simplifications:

 - Operators no longer have hold encoders, instead they have only the expressions that they need.  The benefits here are twofold: the expressions are visible to transformations so go through the normal resolution/binding process.  now that they are visible we can change them on a case by case basis.
 - Operators no longer have type parameters.  Since the engine is responsible for its own type checking, having the types visible to the complier was an unnecessary complication.  We still leverage the scala compiler in the companion factory when constructing a new operator, but after this the types are discarded.

Deferred to a follow up PR:
 - Remove as much of the resolution/binding from Dataset/GroupedDataset as possible. We should still eagerly check resolution and throw an error though in the case of mismatches for an `as` operation.
 - Eliminate serializations in more cases by adding more cases to `EliminateSerialization`

Author: Michael Armbrust <michael@databricks.com>

Closes #10747 from marmbrus/encoderExpressions.
2016-01-14 17:44:56 -08:00
Reynold Xin 902667fd27 [SPARK-12771][SQL] Simplify CaseWhen code generation
The generated code for CaseWhen uses a control variable "got" to make sure we do not evaluate more branches once a branch is true. Changing that to generate just simple "if / else" would be slightly more efficient.

This closes #10737.

Author: Reynold Xin <rxin@databricks.com>

Closes #10755 from rxin/SPARK-12771.
2016-01-14 10:09:03 -08:00
Wenchen Fan 962e9bcf94 [SPARK-12756][SQL] use hash expression in Exchange
This PR makes bucketing and exchange share one common hash algorithm, so that we can guarantee the data distribution is same between shuffle and bucketed data source, which enables us to only shuffle one side when join a bucketed table and a normal one.

This PR also fixes the tests that are broken by the new hash behaviour in shuffle.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10703 from cloud-fan/use-hash-expr-in-shuffle.
2016-01-13 22:43:28 -08:00
Reynold Xin cbbcd8e425 [SPARK-12791][SQL] Simplify CaseWhen by breaking "branches" into "conditions" and "values"
This pull request rewrites CaseWhen expression to break the single, monolithic "branches" field into a sequence of tuples (Seq[(condition, value)]) and an explicit optional elseValue field.

Prior to this pull request, each even position in "branches" represents the condition for each branch, and each odd position represents the value for each branch. The use of them have been pretty confusing with a lot sliding windows or grouped(2) calls.

Author: Reynold Xin <rxin@databricks.com>

Closes #10734 from rxin/simplify-case.
2016-01-13 12:44:35 -08:00
Wenchen Fan c2ea79f96a [SPARK-12642][SQL] improve the hash expression to be decoupled from unsafe row
https://issues.apache.org/jira/browse/SPARK-12642

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10694 from cloud-fan/hash-expr.
2016-01-13 12:29:02 -08:00
Liang-Chi Hsieh 63eee86cc6 [SPARK-9297] [SQL] Add covar_pop and covar_samp
JIRA: https://issues.apache.org/jira/browse/SPARK-9297

Add two aggregation functions: covar_pop and covar_samp.

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

Closes #10029 from viirya/covar-funcs.
2016-01-13 10:26:55 -08:00
Kousuke Saruta cb7b864a24 [SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before ",")
Fix the style violation (space before , and :).
This PR is a followup for #10643 and rework of #10685 .

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #10732 from sarutak/SPARK-12692-followup-sql.
2016-01-12 22:25:20 -08:00
Reynold Xin b3b9ad23cf [SPARK-12788][SQL] Simplify BooleanEquality by using casts.
Author: Reynold Xin <rxin@databricks.com>

Closes #10730 from rxin/SPARK-12788.
2016-01-12 18:45:55 -08:00
Reynold Xin 0d543b98f3 Revert "[SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before "," or ":")"
This reverts commit 8cfa218f4f.
2016-01-12 12:56:52 -08:00
Reynold Xin 0ed430e315 [SPARK-12768][SQL] Remove CaseKeyWhen expression
This patch removes CaseKeyWhen expression and replaces it with a factory method that generates the equivalent CaseWhen. This reduces the amount of code we'd need to maintain in the future for both code generation and optimizer.

Note that we introduced CaseKeyWhen to avoid duplicate evaluations of the key. This is no longer a problem because we now have common subexpression elimination.

Author: Reynold Xin <rxin@databricks.com>

Closes #10722 from rxin/SPARK-12768.
2016-01-12 11:13:08 -08:00
Reynold Xin 1d88879530 [SPARK-12762][SQL] Add unit test for SimplifyConditionals optimization rule
This pull request does a few small things:

1. Separated if simplification from BooleanSimplification and created a new rule SimplifyConditionals. In the future we can also simplify other conditional expressions here.

2. Added unit test for SimplifyConditionals.

3. Renamed SimplifyCaseConversionExpressionsSuite to SimplifyStringCaseConversionSuite

Author: Reynold Xin <rxin@databricks.com>

Closes #10716 from rxin/SPARK-12762.
2016-01-12 10:58:57 -08:00
Kousuke Saruta 8cfa218f4f [SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before "," or ":")
Fix the style violation (space before , and :).
This PR is a followup for #10643.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #10718 from sarutak/SPARK-12692-followup-sql.
2016-01-12 00:51:00 -08:00
Cheng Lian 36d493509d [SPARK-12498][SQL][MINOR] BooleanSimplication simplification
Scala syntax allows binary case classes to be used as infix operator in pattern matching. This PR makes use of this syntax sugar to make `BooleanSimplification` more readable.

Author: Cheng Lian <lian@databricks.com>

Closes #10445 from liancheng/boolean-simplification-simplification.
2016-01-11 18:42:26 -08:00
Herman van Hovell fe9eb0b0ce [SPARK-12576][SQL] Enable expression parsing in CatalystQl
The PR allows us to use the new SQL parser to parse SQL expressions such as: ```1 + sin(x*x)```

We enable this functionality in this PR, but we will not start using this actively yet. This will be done as soon as we have reached grammar parity with the existing parser stack.

cc rxin

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

Closes #10649 from hvanhovell/SPARK-12576.
2016-01-11 16:29:37 -08:00
Marcelo Vanzin 6439a82503 [SPARK-3873][BUILD] Enable import ordering error checking.
Turn import ordering violations into build errors, plus a few adjustments
to account for how the checker behaves. I'm a little on the fence about
whether the existing code is right, but it's easier to appease the checker
than to discuss what's the more correct order here.

Plus a few fixes to imports that cropped in since my recent cleanups.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10612 from vanzin/SPARK-3873-enable.
2016-01-10 20:04:50 -08:00
Liang-Chi Hsieh 95cd5d95ce [SPARK-12577] [SQL] Better support of parentheses in partition by and order by clause of window function's over clause
JIRA: https://issues.apache.org/jira/browse/SPARK-12577

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

Closes #10620 from viirya/fix-parentheses.
2016-01-08 21:48:06 -08:00
Cheng Lian d9447cac74 [SPARK-12593][SQL] Converts resolved logical plan back to SQL
This PR tries to enable Spark SQL to convert resolved logical plans back to SQL query strings.  For now, the major use case is to canonicalize Spark SQL native view support.  The major entry point is `SQLBuilder.toSQL`, which returns an `Option[String]` if the logical plan is recognized.

The current version is still in WIP status, and is quite limited.  Known limitations include:

1.  The logical plan must be analyzed but not optimized

    The optimizer erases `Subquery` operators, which contain necessary scope information for SQL generation.  Future versions should be able to recover erased scope information by inserting subqueries when necessary.

1.  The logical plan must be created using HiveQL query string

    Query plans generated by composing arbitrary DataFrame API combinations are not supported yet.  Operators within these query plans need to be rearranged into a canonical form that is more suitable for direct SQL generation.  For example, the following query plan

    ```
    Filter (a#1 < 10)
     +- MetastoreRelation default, src, None
    ```

    need to be canonicalized into the following form before SQL generation:

    ```
    Project [a#1, b#2, c#3]
     +- Filter (a#1 < 10)
         +- MetastoreRelation default, src, None
    ```

    Otherwise, the SQL generation process will have to handle a large number of special cases.

1.  Only a fraction of expressions and basic logical plan operators are supported in this PR

    Currently, 95.7% (1720 out of 1798) query plans in `HiveCompatibilitySuite` can be successfully converted to SQL query strings.

    Known unsupported components are:

    - Expressions
      - Part of math expressions
      - Part of string expressions (buggy?)
      - Null expressions
      - Calendar interval literal
      - Part of date time expressions
      - Complex type creators
      - Special `NOT` expressions, e.g. `NOT LIKE` and `NOT IN`
    - Logical plan operators/patterns
      - Cube, rollup, and grouping set
      - Script transformation
      - Generator
      - Distinct aggregation patterns that fit `DistinctAggregationRewriter` analysis rule
      - Window functions

    Support for window functions, generators, and cubes etc. will be added in follow-up PRs.

This PR leverages `HiveCompatibilitySuite` for testing SQL generation in a "round-trip" manner:

*   For all select queries, we try to convert it back to SQL
*   If the query plan is convertible, we parse the generated SQL into a new logical plan
*   Run the new logical plan instead of the original one

If the query plan is inconvertible, the test case simply falls back to the original logic.

TODO

- [x] Fix failed test cases
- [x] Support for more basic expressions and logical plan operators (e.g. distinct aggregation etc.)
- [x] Comments and documentation

Author: Cheng Lian <lian@databricks.com>

Closes #10541 from liancheng/sql-generation.
2016-01-08 14:08:13 -08:00
Liang-Chi Hsieh cfe1ba56e4 [SPARK-12687] [SQL] Support from clause surrounded by ().
JIRA: https://issues.apache.org/jira/browse/SPARK-12687

Some queries such as `(select 1 as a) union (select 2 as a)` can't work. This patch fixes it.

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

Closes #10660 from viirya/fix-union.
2016-01-08 09:50:41 -08:00
Sean Owen b9c8353378 [SPARK-12618][CORE][STREAMING][SQL] Clean up build warnings: 2.0.0 edition
Fix most build warnings: mostly deprecated API usages. I'll annotate some of the changes below. CC rxin who is leading the charge to remove the deprecated APIs.

Author: Sean Owen <sowen@cloudera.com>

Closes #10570 from srowen/SPARK-12618.
2016-01-08 17:47:44 +00:00
Kazuaki Ishizaki 34dbc8af21 [SPARK-12580][SQL] Remove string concatenations from usage and extended in @ExpressionDescription
Use multi-line string literals for ExpressionDescription with ``// scalastyle:off line.size.limit`` and ``// scalastyle:on line.size.limit``

The policy is here, as describe at https://github.com/apache/spark/pull/10488

Let's use multi-line string literals. If we have to have a line with more than 100 characters, let's use ``// scalastyle:off line.size.limit`` and ``// scalastyle:on line.size.limit`` to just bypass the line number requirement.

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

Closes #10524 from kiszk/SPARK-12580.
2016-01-07 13:56:34 -08:00
Davies Liu fd1dcfaf26 [SPARK-12542][SQL] support except/intersect in HiveQl
Parse the SQL query with except/intersect in FROM clause for HivQL.

Author: Davies Liu <davies@databricks.com>

Closes #10622 from davies/intersect.
2016-01-06 23:46:12 -08:00
Davies Liu 6f7ba6409a [SPARK-12681] [SQL] split IdentifiersParser.g into two files
To avoid to have a huge Java source (over 64K loc), that can't be compiled.

cc hvanhovell

Author: Davies Liu <davies@databricks.com>

Closes #10624 from davies/split_ident.
2016-01-06 15:54:00 -08:00
Herman van Hovell ea489f14f1 [SPARK-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst
This PR moves a major part of the new SQL parser to Catalyst. This is a prelude to start using this parser for all of our SQL parsing. The following key changes have been made:

The ANTLR Parser & Supporting classes have been moved to the Catalyst project. They are now part of the ```org.apache.spark.sql.catalyst.parser``` package. These classes contained quite a bit of code that was originally from the Hive project, I have added aknowledgements whenever this applied. All Hive dependencies have been factored out. I have also taken this chance to clean-up the ```ASTNode``` class, and to improve the error handling.

The HiveQl object that provides the functionality to convert an AST into a LogicalPlan has been refactored into three different classes, one for every SQL sub-project:
- ```CatalystQl```: This implements Query and Expression parsing functionality.
- ```SparkQl```: This is a subclass of CatalystQL and provides SQL/Core only functionality such as Explain and Describe.
- ```HiveQl```: This is a subclass of ```SparkQl``` and this adds Hive-only functionality to the parser such as Analyze, Drop, Views, CTAS & Transforms. This class still depends on Hive.

cc rxin

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

Closes #10583 from hvanhovell/SPARK-12575.
2016-01-06 11:16:53 -08:00
Marcelo Vanzin b3ba1be3b7 [SPARK-3873][TESTS] Import ordering fixes.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10582 from vanzin/SPARK-3873-tests.
2016-01-05 19:07:39 -08:00
Marcelo Vanzin df8bd97520 [SPARK-3873][SQL] Import ordering fixes.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10573 from vanzin/SPARK-3873-sql.
2016-01-05 16:48:59 -08:00
Liang-Chi Hsieh d202ad2fc2 [SPARK-12439][SQL] Fix toCatalystArray and MapObjects
JIRA: https://issues.apache.org/jira/browse/SPARK-12439

In toCatalystArray, we should look at the data type returned by dataTypeFor instead of silentSchemaFor, to determine if the element is native type. An obvious problem is when the element is Option[Int] class, catalsilentSchemaFor will return Int, then we will wrongly recognize the element is native type.

There is another problem when using Option as array element. When we encode data like Seq(Some(1), Some(2), None) with encoder, we will use MapObjects to construct an array for it later. But in MapObjects, we don't check if the return value of lambdaFunction is null or not. That causes a bug that the decoded data for Seq(Some(1), Some(2), None) would be Seq(1, 2, -1), instead of Seq(1, 2, null).

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

Closes #10391 from viirya/fix-catalystarray.
2016-01-05 12:33:21 -08:00
Wenchen Fan 76768337be [SPARK-12480][FOLLOW-UP] use a single column vararg for hash
address comments in #10435

This makes the API easier to use if user programmatically generate the call to hash, and they will get analysis exception if the arguments of hash is empty.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10588 from cloud-fan/hash.
2016-01-05 10:23:36 -08:00
Liang-Chi Hsieh b3c48e39f4 [SPARK-12438][SQL] Add SQLUserDefinedType support for encoder
JIRA: https://issues.apache.org/jira/browse/SPARK-12438

ScalaReflection lacks the support of SQLUserDefinedType. We should add it.

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

Closes #10390 from viirya/encoder-udt.
2016-01-05 10:19:56 -08:00
Michael Armbrust 53beddc5bf [SPARK-12568][SQL] Add BINARY to Encoders
Author: Michael Armbrust <michael@databricks.com>

Closes #10516 from marmbrus/datasetCleanup.
2016-01-04 23:23:41 -08:00
Reynold Xin b634901bb2 [SPARK-12600][SQL] follow up: add range check for DecimalType
This addresses davies' code review feedback in https://github.com/apache/spark/pull/10559

Author: Reynold Xin <rxin@databricks.com>

Closes #10586 from rxin/remove-deprecated-sql-followup.
2016-01-04 21:05:27 -08:00
Wenchen Fan b1a771231e [SPARK-12480][SQL] add Hash expression that can calculate hash value for a group of expressions
just write the arguments into unsafe row and use murmur3 to calculate hash code

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10435 from cloud-fan/hash-expr.
2016-01-04 18:49:41 -08:00
Reynold Xin 77ab49b857 [SPARK-12600][SQL] Remove deprecated methods in Spark SQL
Author: Reynold Xin <rxin@databricks.com>

Closes #10559 from rxin/remove-deprecated-sql.
2016-01-04 18:02:38 -08:00
Nong Li 34de24abb5 [SPARK-12589][SQL] Fix UnsafeRowParquetRecordReader to properly set the row length.
The reader was previously not setting the row length meaning it was wrong if there were variable
length columns. This problem does not manifest usually, since the value in the column is correct and
projecting the row fixes the issue.

Author: Nong Li <nong@databricks.com>

Closes #10576 from nongli/spark-12589.
2016-01-04 14:58:24 -08:00
Davies Liu d084a2de32 [SPARK-12541] [SQL] support cube/rollup as function
This PR enable cube/rollup as function, so they can be used as this:
```
select a, b, sum(c) from t group by rollup(a, b)
```

Author: Davies Liu <davies@databricks.com>

Closes #10522 from davies/rollup.
2016-01-04 14:26:56 -08:00
Herman van Hovell 0171b71e95 [SPARK-12421][SQL] Prevent Internal/External row from exposing state.
It is currently possible to change the values of the supposedly immutable ```GenericRow``` and ```GenericInternalRow``` classes. This is caused by the fact that scala's ArrayOps ```toArray``` (returned by calling ```toSeq```) will return the backing array instead of a copy. This PR fixes this problem.

This PR was inspired by https://github.com/apache/spark/pull/10374 by apo1.

cc apo1 sarutak marmbrus cloud-fan nongli (everyone in the previous conversation).

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

Closes #10553 from hvanhovell/SPARK-12421.
2016-01-04 12:41:57 -08:00
Pete Robbins b504b6a90a [SPARK-12470] [SQL] Fix size reduction calculation
also only allocate required buffer size

Author: Pete Robbins <robbinspg@gmail.com>

Closes #10421 from robbinspg/master.
2016-01-04 10:43:21 -08:00
thomastechs c82924d564 [SPARK-12533][SQL] hiveContext.table() throws the wrong exception
Avoiding the the No such table exception and throwing analysis exception as per the bug: SPARK-12533

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

Closes #10529 from thomastechs/topic-branch.
2016-01-03 11:09:30 -08:00
Liang-Chi Hsieh c9dbfcc653 [SPARK-11743][SQL] Move the test for arrayOfUDT
A following pr for #9712. Move the test for arrayOfUDT.

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

Closes #10538 from viirya/move-udt-test.
2015-12-31 23:48:05 -08:00
Davies Liu e6c77874b9 [SPARK-12585] [SQL] move numFields to constructor of UnsafeRow
Right now, numFields will be passed in by pointTo(), then bitSetWidthInBytes is calculated, making pointTo() a little bit heavy.

It should be part of constructor of UnsafeRow.

Author: Davies Liu <davies@databricks.com>

Closes #10528 from davies/numFields.
2015-12-30 22:16:37 -08:00
Herman van Hovell f76ee109d8 [SPARK-8641][SPARK-12455][SQL] Native Spark Window functions - Follow-up (docs & tests)
This PR is a follow-up for PR https://github.com/apache/spark/pull/9819. It adds documentation for the window functions and a couple of NULL tests.

The documentation was largely based on the documentation in (the source of)  Hive and Presto:
* https://prestodb.io/docs/current/functions/window.html
* https://cwiki.apache.org/confluence/display/Hive/LanguageManual+WindowingAndAnalytics

I am not sure if we need to add the licenses of these two projects to the licenses directory. They are both under the ASL. srowen any thoughts?

cc yhuai

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

Closes #10402 from hvanhovell/SPARK-8641-docs.
2015-12-30 16:51:07 -08:00
Wenchen Fan aa48164a43 [SPARK-12495][SQL] use true as default value for propagateNull in NewInstance
Most of cases we should propagate null when call `NewInstance`, and so far there is only one case we should stop null propagation: create product/java bean. So I think it makes more sense to propagate null by dafault.

This also fixes a bug when encode null array/map, which is firstly discovered in https://github.com/apache/spark/pull/10401

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10443 from cloud-fan/encoder.
2015-12-30 10:56:08 -08:00
gatorsmile 4f75f785df [SPARK-12564][SQL] Improve missing column AnalysisException
```
org.apache.spark.sql.AnalysisException: cannot resolve 'value' given input columns text;
```

lets put a `:` after `columns` and put the columns in `[]` so that they match the toString of DataFrame.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10518 from gatorsmile/improveAnalysisExceptionMsg.
2015-12-29 22:28:59 -08:00
Reynold Xin 270a659584 [SPARK-12549][SQL] Take Option[Seq[DataType]] in UDF input type specification.
In Spark we allow UDFs to declare its expected input types in order to apply type coercion. The expected input type parameter takes a Seq[DataType] and uses Nil when no type coercion is applied. It makes more sense to take Option[Seq[DataType]] instead, so we can differentiate a no-arg function vs function with no expected input type specified.

Author: Reynold Xin <rxin@databricks.com>

Closes #10504 from rxin/SPARK-12549.
2015-12-29 16:58:23 -08:00
Kazuaki Ishizaki 8e629b10cb [SPARK-12530][BUILD] Fix build break at Spark-Master-Maven-Snapshots from #1293
Compilation error caused due to string concatenations that are not a constant
Use raw string literal to avoid string concatenations

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Thanks cloud-fan for helping investigating this issue!

Author: Cheng Lian <lian@databricks.com>

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

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

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

Author: Xiu Guo <xguo27@gmail.com>

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

Author: Cheng Lian <lian@databricks.com>

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

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

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

Thank you very much!

Author: gatorsmile <gatorsmile@gmail.com>

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

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

example json:

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

Closes #10373 from dilipbiswal/spark-12398.
2015-12-21 12:46:06 -08:00
Kousuke Saruta 6eba655259 [SPARK-12404][SQL] Ensure objects passed to StaticInvoke is Serializable
Now `StaticInvoke` receives `Any` as a object and `StaticInvoke` can be serialized but sometimes the object passed is not serializable.

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

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

I'll add test cases.

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

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

This PR fix lots of bugs about nullability.

Author: Davies Liu <davies@databricks.com>

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

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

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

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

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

cc rxin / yhuai

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

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

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

Author: Davies Liu <davies@databricks.com>

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

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

cc yhuai nongli marmbrus

Author: Davies Liu <davies@databricks.com>

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

Author: Davies Liu <davies@databricks.com>

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

Closes #10249

Author: Davies Liu <davies@databricks.com>

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

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

cc marmbrus nongli

Author: Davies Liu <davies@databricks.com>

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

Author: Josh Rosen <joshrosen@databricks.com>

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

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

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

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

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

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

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

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

cr: JoshRosen, pwendell

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

Author: Dmitry Erastov <derastov@gmail.com>

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

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

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

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

Author: Yin Huai <yhuai@databricks.com>

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

Author: Cheng Lian <lian@databricks.com>

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

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

cc yhuai

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

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

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

c.get.explain(true)
```

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

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

Author: Yin Huai <yhuai@databricks.com>

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

Author: Yin Huai <yhuai@databricks.com>

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

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

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

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

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

cc yhuai

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

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

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

Fixed the problem and also added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

whoever merge this PR, please give the credit to dilipbiswal

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

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

Author: Marcelo Vanzin <vanzin@cloudera.com>

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

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

cc nongli  yhuai

Author: Davies Liu <davies@databricks.com>

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

Author: Davies Liu <davies@databricks.com>

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

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

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

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

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

I also made similar changes for Python.

Author: Reynold Xin <rxin@databricks.com>

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

Thanks gatorsmile who discovered this bug.

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

We should add NullType support to RowEncoder.

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

Author: Nong Li <nong@databricks.com>

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

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

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

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

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

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

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

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

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

Author: Davies Liu <davies@databricks.com>

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

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

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

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

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

Author: Yin Huai <yhuai@databricks.com>

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

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

Author: Reynold Xin <rxin@databricks.com>

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

This is optionally disabled and only used for supported schemas.

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

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

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

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

Author: Nong Li <nong@databricks.com>

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

Author: Reynold Xin <rxin@databricks.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

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

Author: Reynold Xin <rxin@databricks.com>

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

Author: JihongMa <linlin200605@gmail.com>

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

Author: Wenchen Fan <wenchen@databricks.com>

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

Author: Reynold Xin <rxin@databricks.com>

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

Author: Davies Liu <davies@databricks.com>

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

Author: mayuanwen <mayuanwen@qiyi.com>

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

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

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

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

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

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

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

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

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

scala> df.registerTempTable("DF")

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

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

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

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

Also add tests for it and fix a bug.

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

Author: Yin Huai <yhuai@databricks.com>

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

Author: Yin Huai <yhuai@databricks.com>

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

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

other issues that still in investigation:

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

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

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

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

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

Author: JihongMa <linlin200605@gmail.com>

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

Author: Reynold Xin <rxin@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

Closes #7841 from aray/sql-pivot.
2015-11-11 16:23:24 -08:00
Reynold Xin a9a6b80c71 [SPARK-11645][SQL] Remove OpenHashSet for the old aggregate.
Author: Reynold Xin <rxin@databricks.com>

Closes #9621 from rxin/SPARK-11645.
2015-11-11 12:48:51 -08:00
Wenchen Fan ec2b807212 [SPARK-11564][SQL][FOLLOW-UP] clean up java tuple encoder
We need to support custom classes like java beans and combine them into tuple, and it's very hard to do it with the  TypeTag-based approach.
We should keep only the compose-based way to create tuple encoder.

This PR also move `Encoder` to `org.apache.spark.sql`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9567 from cloud-fan/java.
2015-11-11 10:52:23 -08:00
Wenchen Fan 1510c527b4 [SPARK-10371][SQL][FOLLOW-UP] fix code style
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9627 from cloud-fan/follow.
2015-11-11 09:33:41 -08:00
Herman van Hovell 21c562fa03 [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)
This PR is a 2nd follow-up for [SPARK-9241](https://issues.apache.org/jira/browse/SPARK-9241). It contains the following improvements:
* Fix for a potential bug in distinct child expression and attribute alignment.
* Improved handling of duplicate distinct child expressions.
* Added test for distinct UDAF with multiple children.

cc yhuai

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

Closes #9566 from hvanhovell/SPARK-9241-followup-2.
2015-11-10 16:28:21 -08:00
Yin Huai 3121e78168 [SPARK-9830][SPARK-11641][SQL][FOLLOW-UP] Remove AggregateExpression1 and update toString of Exchange
https://issues.apache.org/jira/browse/SPARK-9830

This is the follow-up pr for https://github.com/apache/spark/pull/9556 to address davies' comments.

Author: Yin Huai <yhuai@databricks.com>

Closes #9607 from yhuai/removeAgg1-followup.
2015-11-10 16:25:22 -08:00
Nong Li 87aedc48c0 [SPARK-10371][SQL] Implement subexpr elimination for UnsafeProjections
This patch adds the building blocks for codegening subexpr elimination and implements
it end to end for UnsafeProjection. The building blocks can be used to do the same thing
for other operators.

It introduces some utilities to compute common sub expressions. Expressions can be added to
this data structure. The expr and its children will be recursively matched against existing
expressions (ones previously added) and grouped into common groups. This is built using
the existing `semanticEquals`. It does not understand things like commutative or associative
expressions. This can be done as future work.

After building this data structure, the codegen process takes advantage of it by:
  1. Generating a helper function in the generated class that computes the common
     subexpression. This is done for all common subexpressions that have at least
     two occurrences and the expression tree is sufficiently complex.
  2. When generating the apply() function, if the helper function exists, call that
     instead of regenerating the expression tree. Repeated calls to the helper function
     shortcircuit the evaluation logic.

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

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

Closes #9480 from nongli/spark-10371.
2015-11-10 11:28:53 -08:00
Wenchen Fan 53600854c2 [SPARK-11590][SQL] use native json_tuple in lateral view
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9562 from cloud-fan/json-tuple.
2015-11-10 11:21:31 -08:00
Wenchen Fan dfcfcbcc04 [SPARK-11578][SQL][FOLLOW-UP] complete the user facing api for typed aggregation
Currently the user facing api for typed aggregation has some limitations:

* the customized typed aggregation must be the first of aggregation list
* the customized typed aggregation can only use long as buffer type
* the customized typed aggregation can only use flat type as result type

This PR tries to remove these limitations.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9599 from cloud-fan/agg.
2015-11-10 11:14:25 -08:00
Yin Huai e0701c7560 [SPARK-9830][SQL] Remove AggregateExpression1 and Aggregate Operator used to evaluate AggregateExpression1s
https://issues.apache.org/jira/browse/SPARK-9830

This PR contains the following main changes.
* Removing `AggregateExpression1`.
* Removing `Aggregate` operator, which is used to evaluate `AggregateExpression1`.
* Removing planner rule used to plan `Aggregate`.
* Linking `MultipleDistinctRewriter` to analyzer.
* Renaming `AggregateExpression2` to `AggregateExpression` and `AggregateFunction2` to `AggregateFunction`.
* Updating places where we create aggregate expression. The way to create aggregate expressions is `AggregateExpression(aggregateFunction, mode, isDistinct)`.
* Changing `val`s in `DeclarativeAggregate`s that touch children of this function to `lazy val`s (when we create aggregate expression in DataFrame API, children of an aggregate function can be unresolved).

Author: Yin Huai <yhuai@databricks.com>

Closes #9556 from yhuai/removeAgg1.
2015-11-10 11:06:29 -08:00
Wenchen Fan fcb57e9c73 [SPARK-11564][SQL][FOLLOW-UP] improve java api for GroupedDataset
created `MapGroupFunction`, `FlatMapGroupFunction`, `CoGroupFunction`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9564 from cloud-fan/map.
2015-11-09 15:16:47 -08:00
Reynold Xin 97b7080cf2 [SPARK-11564][SQL] Dataset Java API audit
A few changes:

1. Removed fold, since it can be confusing for distributed collections.
2. Created specific interfaces for each Dataset function (e.g. MapFunction, ReduceFunction, MapPartitionsFunction)
3. Added more documentation and test cases.

The other thing I'm considering doing is to have a "collector" interface for FlatMapFunction and MapPartitionsFunction, similar to MapReduce's map function.

Author: Reynold Xin <rxin@databricks.com>

Closes #9531 from rxin/SPARK-11564.
2015-11-08 20:57:09 -08:00
Wenchen Fan b2d195e137 [SPARK-11554][SQL] add map/flatMap to GroupedDataset
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9521 from cloud-fan/map.
2015-11-08 12:59:35 -08:00
Herman van Hovell 30c8ba71a7 [SPARK-11451][SQL] Support single distinct count on multiple columns.
This PR adds support for multiple column in a single count distinct aggregate to the new aggregation path.

cc yhuai

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

Closes #9409 from hvanhovell/SPARK-11451.
2015-11-08 11:06:10 -08:00
Herman van Hovell ef362846eb [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up
This PR is a follow up for PR https://github.com/apache/spark/pull/9406. It adds more documentation to the rewriting rule, removes a redundant if expression in the non-distinct aggregation path and adds a multiple distinct test to the AggregationQuerySuite.

cc yhuai marmbrus

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

Closes #9541 from hvanhovell/SPARK-9241-followup.
2015-11-07 13:37:37 -08:00
Herman van Hovell 6d0ead322e [SPARK-9241][SQL] Supporting multiple DISTINCT columns (2) - Rewriting Rule
The second PR for SPARK-9241, this adds support for multiple distinct columns to the new aggregation code path.

This PR solves the multiple DISTINCT column problem by rewriting these Aggregates into an Expand-Aggregate-Aggregate combination. See the [JIRA ticket](https://issues.apache.org/jira/browse/SPARK-9241) for some information on this. The advantages over the - competing - [first PR](https://github.com/apache/spark/pull/9280) are:
- This can use the faster TungstenAggregate code path.
- It is impossible to OOM due to an ```OpenHashSet``` allocating to much memory. However, this will multiply the number of input rows by the number of distinct clauses (plus one), and puts a lot more memory pressure on the aggregation code path itself.

The location of this Rule is a bit funny, and should probably change when the old aggregation path is changed.

cc yhuai - Could you also tell me where to add tests for this?

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

Closes #9406 from hvanhovell/SPARK-9241-rewriter.
2015-11-06 16:04:20 -08:00
Wenchen Fan 7e9a9e603a [SPARK-11269][SQL] Java API support & test cases for Dataset
This simply brings https://github.com/apache/spark/pull/9358 up-to-date.

Author: Wenchen Fan <wenchen@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #9528 from rxin/dataset-java.
2015-11-06 15:37:07 -08:00
Herman van Hovell f328fedafd [SPARK-11450] [SQL] Add Unsafe Row processing to Expand
This PR enables the Expand operator to process and produce Unsafe Rows.

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

Closes #9414 from hvanhovell/SPARK-11450.
2015-11-06 12:21:53 -08:00
Imran Rashid 49f1a82037 [SPARK-10116][CORE] XORShiftRandom.hashSeed is random in high bits
https://issues.apache.org/jira/browse/SPARK-10116

This is really trivial, just happened to notice it -- if `XORShiftRandom.hashSeed` is really supposed to have random bits throughout (as the comment implies), it needs to do something for the conversion to `long`.

mengxr mkolod

Author: Imran Rashid <irashid@cloudera.com>

Closes #8314 from squito/SPARK-10116.
2015-11-06 20:06:24 +00:00
Yin Huai 8211aab079 [SPARK-9858][SQL] Add an ExchangeCoordinator to estimate the number of post-shuffle partitions for aggregates and joins (follow-up)
https://issues.apache.org/jira/browse/SPARK-9858

This PR is the follow-up work of https://github.com/apache/spark/pull/9276. It addresses JoshRosen's comments.

Author: Yin Huai <yhuai@databricks.com>

Closes #9453 from yhuai/numReducer-followUp.
2015-11-06 11:13:51 -08:00
Liang-Chi Hsieh 574141a298 [SPARK-9162] [SQL] Implement code generation for ScalaUDF
JIRA: https://issues.apache.org/jira/browse/SPARK-9162

Currently ScalaUDF extends CodegenFallback and doesn't provide code generation implementation. This path implements code generation for ScalaUDF.

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

Closes #9270 from viirya/scalaudf-codegen.
2015-11-06 10:52:04 -08:00
Wenchen Fan 253e87e8ab [SPARK-11453][SQL][FOLLOW-UP] remove DecimalLit
A cleanup for https://github.com/apache/spark/pull/9085.

The `DecimalLit` is very similar to `FloatLit`, we can just keep one of them.
Also added low level unit test at `SqlParserSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9482 from cloud-fan/parser.
2015-11-06 06:38:49 -08:00
Michael Armbrust 363a476c3f [SPARK-11528] [SQL] Typed aggregations for Datasets
This PR adds the ability to do typed SQL aggregations.  We will likely also want to provide an interface to allow users to do aggregations on objects, but this is deferred to another PR.

```scala
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
ds.groupBy(_._1).agg(sum("_2").as[Int]).collect()

res0: Array(("a", 30), ("b", 3), ("c", 1))
```

Author: Michael Armbrust <michael@databricks.com>

Closes #9499 from marmbrus/dataset-agg.
2015-11-05 21:42:32 -08:00
Reynold Xin 6091e91fca Revert "[SPARK-11469][SQL] Allow users to define nondeterministic udfs."
This reverts commit 9cf56c96b7.
2015-11-05 17:10:35 -08:00
Davies Liu 07414afac9 [SPARK-11537] [SQL] fix negative hours/minutes/seconds
Currently, if the Timestamp is before epoch (1970/01/01), the hours, minutes and seconds will be negative (also rounding up).

Author: Davies Liu <davies@databricks.com>

Closes #9502 from davies/neg_hour.
2015-11-05 17:02:22 -08:00
Reynold Xin d19f4fda63 [SPARK-11505][SQL] Break aggregate functions into multiple files
functions.scala was getting pretty long. I broke it into multiple files.

I also added explicit data types for some public vals, and renamed aggregate function pretty names to lower case, which is more consistent with rest of the functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #9471 from rxin/SPARK-11505.
2015-11-04 13:44:07 -08:00
Reynold Xin 3bd6f5d2ae [SPARK-11490][SQL] variance should alias var_samp instead of var_pop.
stddev is an alias for stddev_samp. variance should be consistent with stddev.

Also took the chance to remove internal Stddev and Variance, and only kept StddevSamp/StddevPop and VarianceSamp/VariancePop.

Author: Reynold Xin <rxin@databricks.com>

Closes #9449 from rxin/SPARK-11490.
2015-11-04 09:34:52 -08:00
Nong e352de0db2 [SPARK-11329] [SQL] Cleanup from spark-11329 fix.
Author: Nong <nong@cloudera.com>

Closes #9442 from nongli/spark-11483.
2015-11-03 16:44:37 -08:00
Daoyuan Wang d188a67762 [SPARK-10533][SQL] handle scientific notation in sqlParser
https://issues.apache.org/jira/browse/SPARK-10533

val df = sqlContext.createDataFrame(Seq(("a",1.0),("b",2.0),("c",3.0)))
df.filter("_2 < 2.0e1").show

Scientific notation didn't work.

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

Closes #9085 from adrian-wang/scinotation.
2015-11-03 22:30:23 +08:00
Davies Liu 67e23b39ac [SPARK-10429] [SQL] make mutableProjection atomic
Right now, SQL's mutable projection updates every value of the mutable project after it evaluates the corresponding expression. This makes the behavior of MutableProjection confusing and complicate the implementation of common aggregate functions like stddev because developers need to be aware that when evaluating {{i+1}}th expression of a mutable projection, {{i}}th slot of the mutable row has already been updated.

This PR make the MutableProjection atomic, by generating all the results of expressions first, then copy them into mutableRow.

Had run a mircro-benchmark, there is no notable performance difference between using class members and local variables.

cc yhuai

Author: Davies Liu <davies@databricks.com>

Closes #9422 from davies/atomic_mutable and squashes the following commits:

bbc1758 [Davies Liu] support wide table
8a0ae14 [Davies Liu] fix bug
bec07da [Davies Liu] refactor
2891628 [Davies Liu] make mutableProjection atomic
2015-11-03 11:42:08 +01:00
Yin Huai d728d5c986 [SPARK-9858][SPARK-9859][SPARK-9861][SQL] Add an ExchangeCoordinator to estimate the number of post-shuffle partitions for aggregates and joins
https://issues.apache.org/jira/browse/SPARK-9858
https://issues.apache.org/jira/browse/SPARK-9859
https://issues.apache.org/jira/browse/SPARK-9861

Author: Yin Huai <yhuai@databricks.com>

Closes #9276 from yhuai/numReducer.
2015-11-03 00:12:49 -08:00
navis.ryu c34c27fe92 [SPARK-9034][SQL] Reflect field names defined in GenericUDTF
Hive GenericUDTF#initialize() defines field names in a returned schema though,
the current HiveGenericUDTF drops these names.
We might need to reflect these in a logical plan tree.

Author: navis.ryu <navis@apache.org>

Closes #8456 from navis/SPARK-9034.
2015-11-02 23:52:36 -08:00
Yin Huai 9cf56c96b7 [SPARK-11469][SQL] Allow users to define nondeterministic udfs.
This is the first task (https://issues.apache.org/jira/browse/SPARK-11469) of https://issues.apache.org/jira/browse/SPARK-11438

Author: Yin Huai <yhuai@databricks.com>

Closes #9393 from yhuai/udfNondeterministic.
2015-11-02 21:18:38 -08:00
Nong Li 9cb5c731da [SPARK-11329][SQL] Support star expansion for structs.
1. Supporting expanding structs in Projections. i.e.
  "SELECT s.*" where s is a struct type.
  This is fixed by allowing the expand function to handle structs in addition to tables.

2. Supporting expanding * inside aggregate functions of structs.
   "SELECT max(struct(col1, structCol.*))"
   This requires recursively expanding the expressions. In this case, it it the aggregate
   expression "max(...)" and we need to recursively expand its children inputs.

Author: Nong Li <nongli@gmail.com>

Closes #9343 from nongli/spark-11329.
2015-11-02 20:32:08 -08:00
tedyu db11ee5e56 [SPARK-11371] Make "mean" an alias for "avg" operator
From Reynold in the thread 'Exception when using some aggregate operators' (http://search-hadoop.com/m/q3RTt0xFr22nXB4/):

I don't think these are bugs. The SQL standard for average is "avg", not "mean". Similarly, a distinct count is supposed to be written as "count(distinct col)", not "countDistinct(col)".
We can, however, make "mean" an alias for "avg" to improve compatibility between DataFrame and SQL.

Author: tedyu <yuzhihong@gmail.com>

Closes #9332 from ted-yu/master.
2015-11-02 13:51:53 -08:00
Liang-Chi Hsieh 3e770a64a4 [SPARK-9298][SQL] Add pearson correlation aggregation function
JIRA: https://issues.apache.org/jira/browse/SPARK-9298

This patch adds pearson correlation aggregation function based on `AggregateExpression2`.

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

Closes #8587 from viirya/corr_aggregation.
2015-11-01 18:37:27 -08:00
Nong Li 046e32ed84 [SPARK-11410][SQL] Add APIs to provide functionality similar to Hive's DISTRIBUTE BY and SORT BY.
DISTRIBUTE BY allows the user to hash partition the data by specified exprs. It also allows for
optioning sorting within each resulting partition. There is no required relationship between the
exprs for partitioning and sorting (i.e. one does not need to be a prefix of the other).

This patch adds to APIs to DataFrames which can be used together to provide this functionality:
  1. distributeBy() which partitions the data frame into a specified number of partitions using the
     partitioning exprs.
  2. localSort() which sorts each partition using the provided sorting exprs.

To get the DISTRIBUTE BY functionality, the user simply does: df.distributeBy(...).localSort(...)

Author: Nong Li <nongli@gmail.com>

Closes #9364 from nongli/spark-11410.
2015-11-01 14:34:06 -08:00
Dilip Biswal fc27dfbf0f [SPARK-11024][SQL] Optimize NULL in <inlist-expressions> by folding it to Literal(null)
Add a rule in optimizer to convert NULL [NOT] IN (expr1,...,expr2) to
Literal(null).

This is a follow up defect to SPARK-8654

cloud-fan Can you please take a look ?

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

Closes #9348 from dilipbiswal/spark_11024.
2015-10-31 12:55:33 -07:00
Davies Liu eb59b94c45 [SPARK-11417] [SQL] no @Override in codegen
Older version of Janino (>2.7) does not support Override, we should not use that in codegen.

Author: Davies Liu <davies@databricks.com>

Closes #9372 from davies/no_override.
2015-10-30 00:36:20 -07:00
Davies Liu 56419cf11f [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPARK-10929] [SQL] Cooperative memory management
This PR introduce a mechanism to call spill() on those SQL operators that support spilling (for example, BytesToBytesMap, UnsafeExternalSorter and ShuffleExternalSorter) if there is not enough memory for execution. The preserved first page is needed anymore, so removed.

Other Spillable objects in Spark core (ExternalSorter and AppendOnlyMap) are not included in this PR, but those could benefit from this (trigger others' spilling).

The PrepareRDD may be not needed anymore, could be removed in follow up PR.

The following script will fail with OOM before this PR, finished in 150 seconds with 2G heap (also works in 1.5 branch, with similar duration).

```python
sqlContext.setConf("spark.sql.shuffle.partitions", "1")
df = sqlContext.range(1<<25).selectExpr("id", "repeat(id, 2) as s")
df2 = df.select(df.id.alias('id2'), df.s.alias('s2'))
j = df.join(df2, df.id==df2.id2).groupBy(df.id).max("id", "id2")
j.explain()
print j.count()
```

For thread-safety, here what I'm got:

1) Without calling spill(), the operators should only be used by single thread, no safety problems.

2) spill() could be triggered in two cases, triggered by itself, or by other operators. we can check trigger == this in spill(), so it's still in the same thread, so safety problems.

3) if it's triggered by other operators (right now cache will not trigger spill()), we only spill the data into disk when it's in scanning stage (building is finished), so the in-memory sorter or memory pages are read-only, we only need to synchronize the iterator and change it.

4) During scanning, the iterator will only use one record in one page, we can't free this page, because the downstream is currently using it (used by UnsafeRow or other objects). In BytesToBytesMap, we just skip the current page, and dump all others into disk. In UnsafeExternalSorter, we keep the page that is used by current record (having the same baseObject), free it when loading the next record. In ShuffleExternalSorter, the spill() will not trigger during scanning.

5) In order to avoid deadlock, we didn't call acquireMemory during spill (so we reused the pointer array in InMemorySorter).

Author: Davies Liu <davies@databricks.com>

Closes #9241 from davies/force_spill.
2015-10-29 23:38:06 -07:00
sethah a01cbf5daa [SPARK-10641][SQL] Add Skewness and Kurtosis Support
Implementing skewness and kurtosis support based on following algorithm:
https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics

Author: sethah <seth.hendrickson16@gmail.com>

Closes #9003 from sethah/SPARK-10641.
2015-10-29 11:58:39 -07:00
Wenchen Fan 87f28fc240 [SPARK-11379][SQL] ExpressionEncoder can't handle top level primitive type correctly
For inner primitive type(e.g. inside `Product`), we use `schemaFor` to get the catalyst type for it, https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala#L403.

However, for top level primitive type, we use `dataTypeFor`, which is wrong.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9337 from cloud-fan/encoder.
2015-10-29 11:17:03 +01:00
Wenchen Fan 0cb7662d86 [SPARK-11351] [SQL] support hive interval literal
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9304 from cloud-fan/interval.
2015-10-28 21:35:57 -07:00
Michael Armbrust 032748bb9a [SPARK-11377] [SQL] withNewChildren should not convert StructType to Seq
This is minor, but I ran into while writing Datasets and while it wasn't needed for the final solution, it was super confusing so we should fix it.

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

Author: Michael Armbrust <michael@databricks.com>

Closes #9334 from marmbrus/structMakeCopy.
2015-10-28 09:40:05 -07:00
Wenchen Fan 075ce4914f [SPARK-11313][SQL] implement cogroup on DataSets (support 2 datasets)
A simpler version of https://github.com/apache/spark/pull/9279, only support 2 datasets.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9324 from cloud-fan/cogroup2.
2015-10-28 13:58:52 +01:00
Michael Armbrust 5a5f65905a [SPARK-11347] [SQL] Support for joinWith in Datasets
This PR adds a new operation `joinWith` to a `Dataset`, which returns a `Tuple` for each pair where a given `condition` evaluates to true.

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

Closes #9300 from marmbrus/datasets-tuples.
2015-10-27 13:28:52 -07:00
Yanbo Liang 360ed832f5 [SPARK-11303][SQL] filter should not be pushed down into sample
When sampling and then filtering DataFrame, the SQL Optimizer will push down filter into sample and produce wrong result. This is due to the sampler is calculated based on the original scope rather than the scope after filtering.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #9294 from yanboliang/spark-11303.
2015-10-27 11:28:59 +01:00
Jia Li 958a0ec8fa [SPARK-11277][SQL] sort_array throws exception scala.MatchError
I'm new to spark. I was trying out the sort_array function then hit this exception. I looked into the spark source code. I found the root cause is that sort_array does not check for an array of NULLs. It's not meaningful to sort an array of entirely NULLs anyway.

I'm adding a check on the input array type to SortArray. If the array consists of NULLs entirely, there is no need to sort such array. I have also added a test case for this.

Please help to review my fix. Thanks!

Author: Jia Li <jiali@us.ibm.com>

Closes #9247 from jliwork/SPARK-11277.
2015-10-27 10:57:08 +01:00
Josh Rosen 85e654c5ec [SPARK-10984] Simplify *MemoryManager class structure
This patch refactors the MemoryManager class structure. After #9000, Spark had the following classes:

- MemoryManager
- StaticMemoryManager
- ExecutorMemoryManager
- TaskMemoryManager
- ShuffleMemoryManager

This is fairly confusing. To simplify things, this patch consolidates several of these classes:

- ShuffleMemoryManager and ExecutorMemoryManager were merged into MemoryManager.
- TaskMemoryManager is moved into Spark Core.

**Key changes and tasks**:

- [x] Merge ExecutorMemoryManager into MemoryManager.
  - [x] Move pooling logic into Allocator.
- [x] Move TaskMemoryManager from `spark-unsafe` to `spark-core`.
- [x] Refactor the existing Tungsten TaskMemoryManager interactions so Tungsten code use only this and not both this and ShuffleMemoryManager.
- [x] Refactor non-Tungsten code to use the TaskMemoryManager instead of ShuffleMemoryManager.
- [x] Merge ShuffleMemoryManager into MemoryManager.
  - [x] Move code
  - [x] ~~Simplify 1/n calculation.~~ **Will defer to followup, since this needs more work.**
- [x] Port ShuffleMemoryManagerSuite tests.
- [x] Move classes from `unsafe` package to `memory` package.
- [ ] Figure out how to handle the hacky use of the memory managers in HashedRelation's broadcast variable construction.
- [x] Test porting and cleanup: several tests relied on mock functionality (such as `TestShuffleMemoryManager.markAsOutOfMemory`) which has been changed or broken during the memory manager consolidation
  - [x] AbstractBytesToBytesMapSuite
  - [x] UnsafeExternalSorterSuite
  - [x] UnsafeFixedWidthAggregationMapSuite
  - [x] UnsafeKVExternalSorterSuite

**Compatiblity notes**:

- This patch introduces breaking changes in `ExternalAppendOnlyMap`, which is marked as `DevloperAPI` (likely for legacy reasons): this class now cannot be used outside of a task.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9127 from JoshRosen/SPARK-10984.
2015-10-25 21:19:52 -07:00
Alexander Slesarenko 92b9c5edd9 [SPARK-6428][SQL] Removed unnecessary typecasts in MutableInt, MutableDouble etc.
marmbrus rxin I believe these typecasts are not required in the presence of explicit return types.

Author: Alexander Slesarenko <avslesarenko@gmail.com>

Closes #9262 from aslesarenko/remove-typecasts.
2015-10-25 10:37:10 +01:00
Davies Liu 487d409e71 [SPARK-11243][SQL] zero out padding bytes in UnsafeRow
For nested StructType, the underline buffer could be used for others before, we should zero out the padding bytes for those primitive types that have less than 8 bytes.

cc cloud-fan

Author: Davies Liu <davies@databricks.com>

Closes #9217 from davies/zero_out.
2015-10-23 01:33:14 -07:00
Reynold Xin cdea0174e3 [SPARK-11273][SQL] Move ArrayData/MapData/DataTypeParser to catalyst.util package
Author: Reynold Xin <rxin@databricks.com>

Closes #9239 from rxin/types-private.
2015-10-23 00:00:21 -07:00
Michael Armbrust 53e83a3a77 [SPARK-11116][SQL] First Draft of Dataset API
*This PR adds a new experimental API to Spark, tentitively named Datasets.*

A `Dataset` is a strongly-typed collection of objects that can be transformed in parallel using functional or relational operations.  Example usage is as follows:

### Functional
```scala
> val ds: Dataset[Int] = Seq(1, 2, 3).toDS()
> ds.filter(_ % 1 == 0).collect()
res1: Array[Int] = Array(1, 2, 3)
```

### Relational
```scala
scala> ds.toDF().show()
+-----+
|value|
+-----+
|    1|
|    2|
|    3|
+-----+

> ds.select(expr("value + 1").as[Int]).collect()
res11: Array[Int] = Array(2, 3, 4)
```

## Comparison to RDDs
 A `Dataset` differs from an `RDD` in the following ways:
  - The creation of a `Dataset` requires the presence of an explicit `Encoder` that can be
    used to serialize the object into a binary format.  Encoders are also capable of mapping the
    schema of a given object to the Spark SQL type system.  In contrast, RDDs rely on runtime
    reflection based serialization.
  - Internally, a `Dataset` is represented by a Catalyst logical plan and the data is stored
    in the encoded form.  This representation allows for additional logical operations and
    enables many operations (sorting, shuffling, etc.) to be performed without deserializing to
    an object.

A `Dataset` can be converted to an `RDD` by calling the `.rdd` method.

## Comparison to DataFrames

A `Dataset` can be thought of as a specialized DataFrame, where the elements map to a specific
JVM object type, instead of to a generic `Row` container. A DataFrame can be transformed into
specific Dataset by calling `df.as[ElementType]`.  Similarly you can transform a strongly-typed
`Dataset` to a generic DataFrame by calling `ds.toDF()`.

## Implementation Status and TODOs

This is a rough cut at the least controversial parts of the API.  The primary purpose here is to get something committed so that we can better parallelize further work and get early feedback on the API.  The following is being deferred to future PRs:
 - Joins and Aggregations (prototype here f11f91e6f0)
 - Support for Java

Additionally, the responsibility for binding an encoder to a given schema is currently done in a fairly ad-hoc fashion.  This is an internal detail, and what we are doing today works for the cases we care about.  However, as we add more APIs we'll probably need to do this in a more principled way (i.e. separate resolution from binding as we do in DataFrames).

## COMPATIBILITY NOTE
Long term we plan to make `DataFrame` extend `Dataset[Row]`.  However,
making this change to che class hierarchy would break the function signatures for the existing
function operations (map, flatMap, etc).  As such, this class should be considered a preview
of the final API.  Changes will be made to the interface after Spark 1.6.

Author: Michael Armbrust <michael@databricks.com>

Closes #9190 from marmbrus/dataset-infra.
2015-10-22 15:20:17 -07:00
Wenchen Fan 42d225f449 [SPARK-11216][SQL][FOLLOW-UP] add encoder/decoder for external row
address comments in https://github.com/apache/spark/pull/9184

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9212 from cloud-fan/encoder.
2015-10-22 10:53:59 -07:00
Davies Liu 1d97332715 [SPARK-11243][SQL] output UnsafeRow from columnar cache
This PR change InMemoryTableScan to output UnsafeRow, and optimize the unrolling and scanning by coping the bytes for var-length types between UnsafeRow and ByteBuffer directly without creating the wrapper objects. When scanning the decimals in TPC-DS store_sales table, it's 80% faster (copy it as long without create Decimal objects).

Author: Davies Liu <davies@databricks.com>

Closes #9203 from davies/unsafe_cache.
2015-10-21 19:20:31 -07:00
Dilip Biswal dce2f8c9d7 [SPARK-8654][SQL] Analysis exception when using NULL IN (...) : invalid cast
In the analysis phase , while processing the rules for IN predicate, we
compare the in-list types to the lhs expression type and generate
cast operation if necessary. In the case of NULL [NOT] IN expr1 , we end up
generating cast between in list types to NULL like cast (1 as NULL) which
is not a valid cast.

The fix is to find a common type between LHS and RHS expressions and cast
all the expression to the common type.

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

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

Closes #9036 from dilipbiswal/spark_8654_new.
2015-10-21 14:29:03 -07:00
Shagun Sodhani 19ad18638e [SPARK-11233][SQL] register cosh in function registry
Author: Shagun Sodhani <sshagunsodhani@gmail.com>

Closes #9199 from shagunsodhani/proposed-fix-#11233.
2015-10-21 14:18:06 -07:00
Yin Huai 3afe448d39 [SPARK-9740][SPARK-9592][SPARK-9210][SQL] Change the default behavior of First/Last to RESPECT NULLS.
I am changing the default behavior of `First`/`Last` to respect null values (the SQL standard default behavior).

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

Author: Yin Huai <yhuai@databricks.com>

Closes #8113 from yhuai/firstLast.
2015-10-21 13:43:17 -07:00
Davies Liu f8c6bec657 [SPARK-11197][SQL] run SQL on files directly
This PR introduce a new feature to run SQL directly on files without create a table, for example:

```
select id from json.`path/to/json/files` as j
```

Author: Davies Liu <davies@databricks.com>

Closes #9173 from davies/source.
2015-10-21 13:38:30 -07:00
Wenchen Fan 7c74ebca05 [SPARK-10743][SQL] keep the name of expression if possible when do cast
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8859 from cloud-fan/cast.
2015-10-21 13:22:35 -07:00
Dilip Biswal 49ea0e9d7c [SPARK-10534] [SQL] ORDER BY clause allows only columns that are present in the select projection list
Find out the missing attributes by recursively looking
at the sort order expression and rest of the code
takes care of projecting them out.

Added description from cloud-fan

I wanna explain a bit more about this bug.

When we resolve sort ordering, we will use a special method, which only resolves UnresolvedAttributes and UnresolvedExtractValue. However, for something like Floor('a), even the 'a is resolved, the floor expression may still being unresolved as data type mismatch(for example, 'a is string type and Floor need double type), thus can't pass this filter, and we can't push down this missing attribute 'a

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

Closes #9123 from dilipbiswal/SPARK-10534.
2015-10-21 11:10:32 -07:00
Wenchen Fan ccf536f903 [SPARK-11216] [SQL] add encoder/decoder for external row
Implement encode/decode for external row based on `ClassEncoder`.

TODO:
* code cleanup
* ~~fix corner cases~~
* refactor the encoder interface
* improve test for product codegen, to cover more corner cases.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9184 from cloud-fan/encoder.
2015-10-21 11:06:34 -07:00
nitin goyal f62e326088 [SPARK-11179] [SQL] Push filters through aggregate
Push conjunctive predicates though Aggregate operators when their references are a subset of the groupingExpressions.

Query plan before optimisation :-
Filter ((c#138L = 2) && (a#0 = 3))
 Aggregate [a#0], [a#0,count(b#1) AS c#138L]
  Project [a#0,b#1]
   LocalRelation [a#0,b#1,c#2]

Query plan after optimisation :-
Filter (c#138L = 2)
 Aggregate [a#0], [a#0,count(b#1) AS c#138L]
  Filter (a#0 = 3)
   Project [a#0,b#1]
    LocalRelation [a#0,b#1,c#2]

Author: nitin goyal <nitin.goyal@guavus.com>
Author: nitin.goyal <nitin.goyal@guavus.com>

Closes #9167 from nitin2goyal/master.
2015-10-21 10:45:21 -07:00
Davies Liu 06e6b765d0 [SPARK-11149] [SQL] Improve cache performance for primitive types
This PR improve the performance by:

1) Generate an Iterator that take Iterator[CachedBatch] as input, and call accessors (unroll the loop for columns), avoid the expensive Iterator.flatMap.

2) Use Unsafe.getInt/getLong/getFloat/getDouble instead of ByteBuffer.getInt/getLong/getFloat/getDouble, the later one actually read byte by byte.

3) Remove the unnecessary copy() in Coalesce(), which is not related to memory cache, found during benchmark.

The following benchmark showed that we can speedup the columnar cache of int by 2x.

```
path = '/opt/tpcds/store_sales/'
int_cols = ['ss_sold_date_sk', 'ss_sold_time_sk', 'ss_item_sk','ss_customer_sk']
df = sqlContext.read.parquet(path).select(int_cols).cache()
df.count()

t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```

Author: Davies Liu <davies@databricks.com>

Closes #9145 from davies/byte_buffer.
2015-10-20 14:01:53 -07:00
Davies Liu 67d468f8d9 [SPARK-11111] [SQL] fast null-safe join
Currently, we use CartesianProduct for join with null-safe-equal condition.
```
scala> sqlContext.sql("select * from t a join t b on (a.i <=> b.i)").explain
== Physical Plan ==
TungstenProject [i#2,j#3,i#7,j#8]
 Filter (i#2 <=> i#7)
  CartesianProduct
   LocalTableScan [i#2,j#3], [[1,1]]
   LocalTableScan [i#7,j#8], [[1,1]]
```
Actually, we can have an equal-join condition as  `coalesce(i, default) = coalesce(b.i, default)`, then an partitioned join algorithm could be used.

After this PR, the plan will become:
```
>>> sqlContext.sql("select * from a join b ON a.id <=> b.id").explain()
TungstenProject [id#0L,id#1L]
 Filter (id#0L <=> id#1L)
  SortMergeJoin [coalesce(id#0L,0)], [coalesce(id#1L,0)]
   TungstenSort [coalesce(id#0L,0) ASC], false, 0
    TungstenExchange hashpartitioning(coalesce(id#0L,0),200)
     ConvertToUnsafe
      Scan PhysicalRDD[id#0L]
   TungstenSort [coalesce(id#1L,0) ASC], false, 0
    TungstenExchange hashpartitioning(coalesce(id#1L,0),200)
     ConvertToUnsafe
      Scan PhysicalRDD[id#1L]
```

Author: Davies Liu <davies@databricks.com>

Closes #9120 from davies/null_safe.
2015-10-20 13:40:24 -07:00
Wenchen Fan 478c7ce862 [SPARK-6740] [SQL] correctly parse NOT operator with comparison operations
We can't parse `NOT` operator with comparison operations like `SELECT NOT TRUE > TRUE`, this PR fixed it.

Takes over https://github.com/apache/spark/pull/6326.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8617 from cloud-fan/not.
2015-10-20 13:38:25 -07:00
Daoyuan Wang 94139557c5 [SPARK-10463] [SQL] remove PromotePrecision during optimization
PromotePrecision is not necessary after HiveTypeCoercion done.

Jira:
https://issues.apache.org/jira/browse/SPARK-10463

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

Closes #8621 from adrian-wang/promoterm.
2015-10-20 09:20:35 -07:00
Wenchen Fan 7893cd95db [SPARK-11119] [SQL] cleanup for unsafe array and map
The purpose of this PR is to keep the unsafe format detail only inside the unsafe class itself, so when we use them(like use unsafe array in unsafe map, use unsafe array and map in columnar cache), we don't need to understand the format before use them.

change list:
* unsafe array's 4-bytes numElements header is now required(was optional), and become a part of unsafe array format.
* w.r.t the previous changing, the `sizeInBytes` of unsafe array now counts the 4-bytes header.
* unsafe map's format was `[numElements] [key array numBytes] [key array content(without numElements header)] [value array content(without numElements header)]` before, which is a little hacky as it makes unsafe array's header optional. I think saving 4 bytes is not a big deal, so the format is now: `[key array numBytes] [unsafe key array] [unsafe value array]`.
* w.r.t the previous changing, the `sizeInBytes` of unsafe map now counts both map's header and array's header.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9131 from cloud-fan/unsafe.
2015-10-19 11:02:26 -07:00
navis.ryu b9c5e5d4ac [SPARK-11124] JsonParser/Generator should be closed for resource recycle
Some json parsers are not closed. parser in JacksonParser#parseJson, for example.

Author: navis.ryu <navis@apache.org>

Closes #9130 from navis/SPARK-11124.
2015-10-16 11:19:37 -07:00
Cheng Hao 9808052b5a [SPARK-11076] [SQL] Add decimal support for floor and ceil
Actually all of the `UnaryMathExpression` doens't support the Decimal, will create follow ups for supporing it. This is the first PR which will be good to review the approach I am taking.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #9086 from chenghao-intel/ceiling.
2015-10-14 20:56:08 -07:00
Josh Rosen 4ace4f8a9c [SPARK-11017] [SQL] Support ImperativeAggregates in TungstenAggregate
This patch extends TungstenAggregate to support ImperativeAggregate functions. The existing TungstenAggregate operator only supported DeclarativeAggregate functions, which are defined in terms of Catalyst expressions and can be evaluated via generated projections. ImperativeAggregate functions, on the other hand, are evaluated by calling their `initialize`, `update`, `merge`, and `eval` methods.

The basic strategy here is similar to how SortBasedAggregate evaluates both types of aggregate functions: use a generated projection to evaluate the expression-based declarative aggregates with dummy placeholder expressions inserted in place of the imperative aggregate function output, then invoke the imperative aggregate functions and target them against the aggregation buffer. The bulk of the diff here consists of code that was copied and adapted from SortBasedAggregate, with some key changes to handle TungstenAggregate's sort fallback path.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9038 from JoshRosen/support-interpreted-in-tungsten-agg-final.
2015-10-14 17:27:50 -07:00
Reynold Xin 2b5e31c7e9 [SPARK-11113] [SQL] Remove DeveloperApi annotation from private classes.
o.a.s.sql.catalyst and o.a.s.sql.execution are supposed to be private.

Author: Reynold Xin <rxin@databricks.com>

Closes #9121 from rxin/SPARK-11113.
2015-10-14 16:27:43 -07:00
Wenchen Fan 56d7da14ab [SPARK-10104] [SQL] Consolidate different forms of table identifiers
Right now, we have QualifiedTableName, TableIdentifier, and Seq[String] to represent table identifiers. We should only have one form and TableIdentifier is the best one because it provides methods to get table name, database name, return unquoted string, and return quoted string.

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8453 from cloud-fan/table-name.
2015-10-14 16:05:37 -07:00
Wenchen Fan e170c22160 [SPARK-11032] [SQL] correctly handle having
We should not stop resolving having when the having condtion is resolved, or something like `count(1)` will crash.

Author: Wenchen Fan <cloud0fan@163.com>

Closes #9105 from cloud-fan/having.
2015-10-13 17:11:22 -07:00
Michael Armbrust 328d1b3e4b [SPARK-11090] [SQL] Constructor for Product types from InternalRow
This is a first draft of the ability to construct expressions that will take a catalyst internal row and construct a Product (case class or tuple) that has fields with the correct names.  Support include:
 - Nested classes
 - Maps
 - Efficiently handling of arrays of primitive types

Not yet supported:
 - Case classes that require custom collection types (i.e. List instead of Seq).

Author: Michael Armbrust <michael@databricks.com>

Closes #9100 from marmbrus/productContructor.
2015-10-13 17:09:17 -07:00
Josh Rosen ef72673b23 [SPARK-11080] [SQL] Incorporate per-JVM id into ExprId to prevent unsafe cross-JVM comparisions
In the current implementation of named expressions' `ExprIds`, we rely on a per-JVM AtomicLong to ensure that expression ids are unique within a JVM. However, these expression ids will not be _globally_ unique. This opens the potential for id collisions if new expression ids happen to be created inside of tasks rather than on the driver.

There are currently a few cases where tasks allocate expression ids, which happen to be safe because those expressions are never compared to expressions created on the driver. In order to guard against the introduction of invalid comparisons between driver-created and executor-created expression ids, this patch extends `ExprId` to incorporate a UUID to identify the JVM that created the id, which prevents collisions.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9093 from JoshRosen/SPARK-11080.
2015-10-13 15:09:31 -07:00
Davies Liu c4da5345a0 [SPARK-10990] [SPARK-11018] [SQL] improve unrolling of complex types
This PR improve the unrolling and read of complex types in columnar cache:
1) Using UnsafeProjection to do serialization of complex types, so they will not be serialized three times (two for actualSize)
2) Copy the bytes from UnsafeRow/UnsafeArrayData to ByteBuffer directly, avoiding the immediate byte[]
3) Using the underlying array in ByteBuffer to create UTF8String/UnsafeRow/UnsafeArrayData without copy.

Combine these optimizations,  we can reduce the unrolling time from 25s to 21s (20% less), reduce the scanning time from 3.5s to 2.5s (28% less).

```
df = sqlContext.read.parquet(path)
t = time.time()
df.cache()
df.count()
print 'unrolling', time.time() - t

for i in range(10):
    t = time.time()
    print df.select("*")._jdf.queryExecution().toRdd().count()
    print time.time() - t
```

The schema is
```
root
 |-- a: struct (nullable = true)
 |    |-- b: long (nullable = true)
 |    |-- c: string (nullable = true)
 |-- d: array (nullable = true)
 |    |-- element: long (containsNull = true)
 |-- e: map (nullable = true)
 |    |-- key: long
 |    |-- value: string (valueContainsNull = true)
```

Now the columnar cache depends on that UnsafeProjection support all the data types (including UDT), this PR also fix that.

Author: Davies Liu <davies@databricks.com>

Closes #9016 from davies/complex2.
2015-10-12 21:12:59 -07:00
Liang-Chi Hsieh fcb37a0417 [SPARK-10960] [SQL] SQL with windowing function should be able to refer column in inner select
JIRA: https://issues.apache.org/jira/browse/SPARK-10960

When accessing a column in inner select from a select with window function, `AnalysisException` will be thrown. For example, an query like this:

     select area, rank() over (partition by area order by tmp.month) + tmp.tmp1 as c1 from (select month, area, product, 1 as tmp1 from windowData) tmp

Currently, the rule `ExtractWindowExpressions` in `Analyzer` only extracts regular expressions from `WindowFunction`, `WindowSpecDefinition` and `AggregateExpression`. We need to also extract other attributes as the one in `Alias` as shown in the above query.

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

Closes #9011 from viirya/fix-window-inner-column.
2015-10-12 09:16:14 -07:00
Davies Liu 3390b400d0 [SPARK-10810] [SPARK-10902] [SQL] Improve session management in SQL
This PR improve the sessions management by replacing the thread-local based to one SQLContext per session approach, introduce separated temporary tables and UDFs/UDAFs for each session.

A new session of SQLContext could be created by:

1) create an new SQLContext
2) call newSession() on existing SQLContext

For HiveContext, in order to reduce the cost for each session, the classloader and Hive client are shared across multiple sessions (created by newSession).

CacheManager is also shared by multiple sessions, so cache a table multiple times in different sessions will not cause multiple copies of in-memory cache.

Added jars are still shared by all the sessions, because SparkContext does not support sessions.

cc marmbrus yhuai rxin

Author: Davies Liu <davies@databricks.com>

Closes #8909 from davies/sessions.
2015-10-08 17:34:24 -07:00
Reynold Xin 84ea287178 [SPARK-10914] UnsafeRow serialization breaks when two machines have different Oops size.
UnsafeRow contains 3 pieces of information when pointing to some data in memory (an object, a base offset, and length). When the row is serialized with Java/Kryo serialization, the object layout in memory can change if two machines have different pointer width (Oops in JVM).

To reproduce, launch Spark using

MASTER=local-cluster[2,1,1024] bin/spark-shell --conf "spark.executor.extraJavaOptions=-XX:-UseCompressedOops"

And then run the following

scala> sql("select 1 xx").collect()

Author: Reynold Xin <rxin@databricks.com>

Closes #9030 from rxin/SPARK-10914.
2015-10-08 17:25:14 -07:00
Cheng Lian 02149ff08e [SPARK-8848] [SQL] Refactors Parquet write path to follow parquet-format
This PR refactors Parquet write path to follow parquet-format spec.  It's a successor of PR #7679, but with less non-essential changes.

Major changes include:

1.  Replaces `RowWriteSupport` and `MutableRowWriteSupport` with `CatalystWriteSupport`

    - Writes Parquet data using standard layout defined in parquet-format

      Specifically, we are now writing ...

      - ... arrays and maps in standard 3-level structure with proper annotations and field names
      - ... decimals as `INT32` and `INT64` whenever possible, and taking `FIXED_LEN_BYTE_ARRAY` as the final fallback

    - Supports legacy mode which is compatible with Spark 1.4 and prior versions

      The legacy mode is by default off, and can be turned on by flipping SQL option `spark.sql.parquet.writeLegacyFormat` to `true`.

    - Eliminates per value data type dispatching costs via prebuilt composed writer functions

1.  Cleans up the last pieces of old Parquet support code

As pointed out by rxin previously, we probably want to rename all those `Catalyst*` Parquet classes to `Parquet*` for clarity.  But I'd like to do this in a follow-up PR to minimize code review noises in this one.

Author: Cheng Lian <lian@databricks.com>

Closes #8988 from liancheng/spark-8848/standard-parquet-write-path.
2015-10-08 16:18:35 -07:00
Michael Armbrust 9e66a53c99 [SPARK-10993] [SQL] Inital code generated encoder for product types
This PR is a first cut at code generating an encoder that takes a Scala `Product` type and converts it directly into the tungsten binary format.  This is done through the addition of a new set of expression that can be used to invoke methods on raw JVM objects, extracting fields and converting the result into the required format.  These can then be used directly in an `UnsafeProjection` allowing us to leverage the existing encoding logic.

According to some simple benchmarks, this can significantly speed up conversion (~4x).  However, replacing CatalystConverters is deferred to a later PR to keep this PR at a reasonable size.

```scala
case class SomeInts(a: Int, b: Int, c: Int, d: Int, e: Int)

val data = SomeInts(1, 2, 3, 4, 5)
val encoder = ProductEncoder[SomeInts]
val converter = CatalystTypeConverters.createToCatalystConverter(ScalaReflection.schemaFor[SomeInts].dataType)

(1 to 5).foreach {iter =>
  benchmark(s"converter $iter") {
    var i = 100000000
    while (i > 0) {
      val res = converter(data).asInstanceOf[InternalRow]
      assert(res.getInt(0) == 1)
      assert(res.getInt(1) == 2)
      i -= 1
    }
  }

  benchmark(s"encoder $iter") {
    var i = 100000000
    while (i > 0) {
      val res = encoder.toRow(data)
      assert(res.getInt(0) == 1)
      assert(res.getInt(1) == 2)
      i -= 1
    }
  }
}
```

Results:
```
[info] converter 1: 7170ms
[info] encoder 1: 1888ms
[info] converter 2: 6763ms
[info] encoder 2: 1824ms
[info] converter 3: 6912ms
[info] encoder 3: 1802ms
[info] converter 4: 7131ms
[info] encoder 4: 1798ms
[info] converter 5: 7350ms
[info] encoder 5: 1912ms
```

Author: Michael Armbrust <michael@databricks.com>

Closes #9019 from marmbrus/productEncoder.
2015-10-08 14:28:14 -07:00
Michael Armbrust a8226a9f14 Revert [SPARK-8654] [SQL] Fix Analysis exception when using NULL IN
This reverts commit dcbd58a929 from #8983

Author: Michael Armbrust <michael@databricks.com>

Closes #9034 from marmbrus/revert8654.
2015-10-08 13:49:10 -07:00
Yin Huai 82d275f27c [SPARK-10887] [SQL] Build HashedRelation outside of HashJoinNode.
This PR refactors `HashJoinNode` to take a existing `HashedRelation`. So, we can reuse this node for both `ShuffledHashJoin` and `BroadcastHashJoin`.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #8953 from yhuai/SPARK-10887.
2015-10-08 11:56:44 -07:00
Dilip Biswal dcbd58a929 [SPARK-8654] [SQL] Fix Analysis exception when using NULL IN (...)
In the analysis phase , while processing the rules for IN predicate, we
compare the in-list types to the lhs expression type and generate
cast operation if necessary. In the case of NULL [NOT] IN expr1 , we end up
generating cast between in list types to NULL like cast (1 as NULL) which
is not a valid cast.

The fix is to not generate such a cast if the lhs type is a NullType instead
we translate the expression to Literal(Null).

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

Closes #8983 from dilipbiswal/spark_8654.
2015-10-08 10:41:45 -07:00
Michael Armbrust 5c9fdf74e3 [SPARK-10998] [SQL] Show non-children in default Expression.toString
Its pretty hard to debug problems with expressions when you can't see all the arguments.

Before: `invoke()`
After: `invoke(inputObject#1, intField, IntegerType)`

Author: Michael Armbrust <michael@databricks.com>

Closes #9022 from marmbrus/expressionToString.
2015-10-08 10:22:06 -07:00
Davies Liu 075a0b6582 [SPARK-10917] [SQL] improve performance of complex type in columnar cache
This PR improve the performance of complex types in columnar cache by using UnsafeProjection instead of KryoSerializer.

A simple benchmark show that this PR could improve the performance of scanning a cached table with complex columns by 15x (comparing to Spark 1.5).

Here is the code used to benchmark:

```
df = sc.range(1<<23).map(lambda i: Row(a=Row(b=i, c=str(i)), d=range(10), e=dict(zip(range(10), [str(i) for i in range(10)])))).toDF()
df.write.parquet("table")
```
```
df = sqlContext.read.parquet("table")
df.cache()
df.count()
t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```

Author: Davies Liu <davies@databricks.com>

Closes #8971 from davies/complex.
2015-10-07 15:58:07 -07:00
Josh Rosen 7e2e268289 [SPARK-9702] [SQL] Use Exchange to implement logical Repartition operator
This patch allows `Repartition` to support UnsafeRows. This is accomplished by implementing the logical `Repartition` operator in terms of `Exchange` and a new `RoundRobinPartitioning`.

Author: Josh Rosen <joshrosen@databricks.com>
Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #8083 from JoshRosen/SPARK-9702.
2015-10-07 15:53:37 -07:00
Davies Liu 37526aca24 [SPARK-10980] [SQL] fix bug in create Decimal
The created decimal is wrong if using `Decimal(unscaled, precision, scale)` with unscaled > 1e18 and and precision > 18 and scale > 0.

This bug exists since the beginning.

Author: Davies Liu <davies@databricks.com>

Closes #9014 from davies/fix_decimal.
2015-10-07 15:51:09 -07:00
Reynold Xin 6dbfd7ecf4 [SPARK-10982] [SQL] Rename ExpressionAggregate -> DeclarativeAggregate.
DeclarativeAggregate matches more closely with ImperativeAggregate we already have.

Author: Reynold Xin <rxin@databricks.com>

Closes #9013 from rxin/SPARK-10982.
2015-10-07 15:38:46 -07:00
Josh Rosen a9ecd06149 [SPARK-10941] [SQL] Refactor AggregateFunction2 and AlgebraicAggregate interfaces to improve code clarity
This patch refactors several of the Aggregate2 interfaces in order to improve code clarity.

The biggest change is a refactoring of the `AggregateFunction2` class hierarchy. In the old code, we had a class named `AlgebraicAggregate` that inherited from `AggregateFunction2`, added a new set of methods, then banned the use of the inherited methods. I found this to be fairly confusing because.

If you look carefully at the existing code, you'll see that subclasses of `AggregateFunction2` fall into two disjoint categories: imperative aggregation functions which directly extended `AggregateFunction2` and declarative, expression-based aggregate functions which extended `AlgebraicAggregate`. In order to make this more explicit, this patch refactors things so that `AggregateFunction2` is a sealed abstract class with two subclasses, `ImperativeAggregateFunction` and `ExpressionAggregateFunction`. The superclass, `AggregateFunction2`, now only contains methods and fields that are common to both subclasses.

After making this change, I updated the various AggregationIterator classes to comply with this new naming scheme. I also performed several small renamings in the aggregate interfaces themselves in order to improve clarity and rewrote or expanded a number of comments.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8973 from JoshRosen/tungsten-agg-comments.
2015-10-07 13:19:49 -07:00
Michael Armbrust f5d154bc73 [SPARK-10966] [SQL] Codegen framework cleanup
This PR is mostly cosmetic and cleans up some warts in codegen (nearly all of which were inherited from the original quasiquote version).
 - Add lines numbers to errors (in stacktraces when debug logging is on, and always for compile fails)
 - Use a variable for input row instead of hardcoding "i" everywhere
 - rename `primitive` -> `value` (since its often actually an object)

Author: Michael Armbrust <michael@databricks.com>

Closes #9006 from marmbrus/codegen-cleanup.
2015-10-07 10:17:29 -07:00
Wenchen Fan 4e0027feae [SPARK-10585] [SQL] [FOLLOW-UP] remove no-longer-necessary code for unsafe generation
These code was left there to produce clear diff for https://github.com/apache/spark/pull/8747

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8991 from cloud-fan/clean.
2015-10-05 23:24:12 -07:00
Wenchen Fan a609eb20d9 [SPARK-10934] [SQL] handle hashCode of unsafe array correctly
`Murmur3_x86_32.hashUnsafeWords` only accepts word-aligned bytes, but unsafe array is not.

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8987 from cloud-fan/hash.
2015-10-05 17:31:54 -07:00
Wenchen Fan c4871369db [SPARK-10585] [SQL] only copy data once when generate unsafe projection
This PR is a completely rewritten of GenerateUnsafeProjection, to accomplish the goal of copying data only once. The old code of GenerateUnsafeProjection is still there to reduce review difficulty.

Instead of creating unsafe conversion code for struct, array and map, we create code of writing the content to the global row buffer.

Author: Wenchen Fan <cloud0fan@163.com>
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8747 from cloud-fan/copy-once.
2015-10-05 13:00:58 -07:00
Takeshi YAMAMURO 2272962eb0 [SPARK-9867] [SQL] Move utilities for binary data into ByteArray
The utilities such as Substring#substringBinarySQL and BinaryPrefixComparator#computePrefix for binary data are put together in ByteArray for easy-to-read.

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

Closes #8122 from maropu/CleanUpForBinaryType.
2015-10-01 21:33:27 -04:00
Cheng Hao 4d8c7c6d1c [SPARK-10865] [SPARK-10866] [SQL] Fix bug of ceil/floor, which should returns long instead of the Double type
Floor & Ceiling function should returns Long type, rather than Double.

Verified with MySQL & Hive.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #8933 from chenghao-intel/ceiling.
2015-10-01 11:48:15 -07:00
Nathan Howell 89ea0041ae [SPARK-9617] [SQL] Implement json_tuple
This is an implementation of Hive's `json_tuple` function using Jackson Streaming.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #7946 from NathanHowell/SPARK-9617.
2015-09-30 15:33:12 -07:00
Herman van Hovell 16fd2a2f42 [SPARK-9741] [SQL] Approximate Count Distinct using the new UDAF interface.
This PR implements a HyperLogLog based Approximate Count Distinct function using the new UDAF interface.

The implementation is inspired by the ClearSpring HyperLogLog implementation and should produce the same results.

There is still some documentation and testing left to do.

cc yhuai

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

Closes #8362 from hvanhovell/SPARK-9741.
2015-09-30 10:12:52 -07:00
Cheng Lian 4d5a005b0d [SPARK-10811] [SQL] Eliminates unnecessary byte array copying
When reading Parquet string and binary-backed decimal values, Parquet `Binary.getBytes` always returns a copied byte array, which is unnecessary. Since the underlying implementation of `Binary` values there is guaranteed to be `ByteArraySliceBackedBinary`, and Parquet itself never reuses underlying byte arrays, we can use `Binary.toByteBuffer.array()` to steal the underlying byte arrays without copying them.

This brings performance benefits when scanning Parquet string and binary-backed decimal columns. Note that, this trick doesn't cover binary-backed decimals with precision greater than 18.

My micro-benchmark result is that, this brings a ~15% performance boost for scanning TPC-DS `store_sales` table (scale factor 15).

Another minor optimization done in this PR is that, now we directly construct a Java `BigDecimal` in `Decimal.toJavaBigDecimal` without constructing a Scala `BigDecimal` first. This brings another ~5% performance gain.

Author: Cheng Lian <lian@databricks.com>

Closes #8907 from liancheng/spark-10811/eliminate-array-copying.
2015-09-29 23:30:27 -07:00
Wenchen Fan 418e5e4cbd [SPARK-10741] [SQL] Hive Query Having/OrderBy against Parquet table is not working
https://issues.apache.org/jira/browse/SPARK-10741
I choose the second approach: do not change output exprIds when convert MetastoreRelation to LogicalRelation

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8889 from cloud-fan/hot-bug.
2015-09-27 09:08:38 -07:00
Wenchen Fan 341b13f8f5 [SPARK-10765] [SQL] use new aggregate interface for hive UDAF
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8874 from cloud-fan/hive-agg.
2015-09-24 09:54:07 -07:00
Yin Huai 5aea987c90 [SPARK-10737] [SQL] When using UnsafeRows, SortMergeJoin may return wrong results
https://issues.apache.org/jira/browse/SPARK-10737

Author: Yin Huai <yhuai@databricks.com>

Closes #8854 from yhuai/SMJBug.
2015-09-22 13:31:35 -07:00
Wenchen Fan 5017c685f4 [SPARK-10740] [SQL] handle nondeterministic expressions correctly for set operations
https://issues.apache.org/jira/browse/SPARK-10740

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8858 from cloud-fan/non-deter.
2015-09-22 12:14:59 -07:00
Davies Liu 22d40159e6 [SPARK-10593] [SQL] fix resolve output of Generate
The output of Generate should not be resolved as Reference.

Author: Davies Liu <davies@databricks.com>

Closes #8755 from davies/view.
2015-09-22 11:07:10 -07:00
zsxwing e789000b88 [SPARK-10155] [SQL] Change SqlParser to object to avoid memory leak
Since `scala.util.parsing.combinator.Parsers` is thread-safe since Scala 2.10 (See [SI-4929](https://issues.scala-lang.org/browse/SI-4929)), we can change SqlParser to object to avoid memory leak.

I didn't change other subclasses of `scala.util.parsing.combinator.Parsers` because there is only one instance in one SQLContext, which should not be an issue.

Author: zsxwing <zsxwing@gmail.com>

Closes #8357 from zsxwing/sql-memory-leak.
2015-09-19 18:22:43 -07:00
Holden Karau 3a22b1004f [SPARK-10449] [SQL] Don't merge decimal types with incompatable precision or scales
From JIRA: Schema merging should only handle struct fields. But currently we also reconcile decimal precision and scale information.

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8634 from holdenk/SPARK-10449-dont-merge-different-precision.
2015-09-18 13:47:14 -07:00
Yijie Shen c6f8135ee5 [SPARK-10539] [SQL] Project should not be pushed down through Intersect or Except #8742
Intersect and Except are both set operators and they use the all the columns to compare equality between rows. When pushing their Project parent down, the relations they based on would change, therefore not an equivalent transformation.

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

I added some comments based on the fix of https://github.com/apache/spark/pull/8742.

Author: Yijie Shen <henry.yijieshen@gmail.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #8823 from yhuai/fix_set_optimization.
2015-09-18 13:20:13 -07:00
navis.ryu e3b5d6cb29 [SPARK-10684] [SQL] StructType.interpretedOrdering need not to be serialized
Kryo fails with buffer overflow even with max value (2G).

{noformat}
org.apache.spark.SparkException: Kryo serialization failed: Buffer overflow. Available: 0, required: 1
Serialization trace:
containsChild (org.apache.spark.sql.catalyst.expressions.BoundReference)
child (org.apache.spark.sql.catalyst.expressions.SortOrder)
array (scala.collection.mutable.ArraySeq)
ordering (org.apache.spark.sql.catalyst.expressions.InterpretedOrdering)
interpretedOrdering (org.apache.spark.sql.types.StructType)
schema (org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema). To avoid this, increase spark.kryoserializer.buffer.max value.
        at org.apache.spark.serializer.KryoSerializerInstance.serialize(KryoSerializer.scala:263)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:240)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
{noformat}

Author: navis.ryu <navis@apache.org>

Closes #8808 from navis/SPARK-10684.
2015-09-18 00:43:02 -07:00
Yin Huai aad644fbe2 [SPARK-10639] [SQL] Need to convert UDAF's result from scala to sql type
https://issues.apache.org/jira/browse/SPARK-10639

Author: Yin Huai <yhuai@databricks.com>

Closes #8788 from yhuai/udafConversion.
2015-09-17 11:14:52 -07:00
Reynold Xin 49c649fa0b Tiny style fix for d39f15ea2b. 2015-09-16 15:32:01 -07:00
Kevin Cox d39f15ea2b [SPARK-9794] [SQL] Fix datetime parsing in SparkSQL.
This fixes https://issues.apache.org/jira/browse/SPARK-9794 by using a real ISO8601 parser. (courtesy of the xml component of the standard java library)

cc: angelini

Author: Kevin Cox <kevincox@kevincox.ca>

Closes #8396 from kevincox/kevincox-sql-time-parsing.
2015-09-16 15:30:17 -07:00
Wenchen Fan 31a229aa73 [SPARK-10475] [SQL] improve column prunning for Project on Sort
Sometimes we can't push down the whole `Project` though `Sort`, but we still have a chance to push down part of it.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8644 from cloud-fan/column-prune.
2015-09-15 13:36:52 -07:00
Liang-Chi Hsieh 841972e22c [SPARK-10437] [SQL] Support aggregation expressions in Order By
JIRA: https://issues.apache.org/jira/browse/SPARK-10437

If an expression in `SortOrder` is a resolved one, such as `count(1)`, the corresponding rule in `Analyzer` to make it work in order by will not be applied.

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

Closes #8599 from viirya/orderby-agg.
2015-09-15 13:33:32 -07:00
Sean Owen 4e2242bb41 [SPARK-10576] [BUILD] Move .java files out of src/main/scala
Move .java files in `src/main/scala` to `src/main/java` root, except for `package-info.java` (to stay next to package.scala)

Author: Sean Owen <sowen@cloudera.com>

Closes #8736 from srowen/SPARK-10576.
2015-09-14 15:03:51 -07:00
Davies Liu 7e32387ae6 [SPARK-10522] [SQL] Nanoseconds of Timestamp in Parquet should be positive
Or Hive can't read it back correctly.

Thanks vanzin for report this.

Author: Davies Liu <davies@databricks.com>

Closes #8674 from davies/positive_nano.
2015-09-14 14:20:49 -07:00
JihongMa f4a22808e0 [SPARK-6548] Adding stddev to DataFrame functions
Adding STDDEV support for DataFrame using 1-pass online /parallel algorithm to compute variance. Please review the code change.

Author: JihongMa <linlin200605@gmail.com>
Author: Jihong MA <linlin200605@gmail.com>
Author: Jihong MA <jihongma@jihongs-mbp.usca.ibm.com>
Author: Jihong MA <jihongma@Jihongs-MacBook-Pro.local>

Closes #6297 from JihongMA/SPARK-SQL.
2015-09-12 10:17:15 -07:00
Yash Datta 1eede3b254 [SPARK-7142] [SQL] Minor enhancement to BooleanSimplification Optimizer rule. Incorporate review comments
Adding changes suggested by cloud-fan  in #5700

cc marmbrus

Author: Yash Datta <Yash.Datta@guavus.com>

Closes #8716 from saucam/bool_simp.
2015-09-11 14:55:15 -07:00
Wenchen Fan d5d647380f [SPARK-10442] [SQL] fix string to boolean cast
When we cast string to boolean in hive, it returns `true` if the length of string is > 0, and spark SQL follows this behavior.

However, this behavior is very different from other SQL systems:

1. [presto](https://github.com/facebook/presto/blob/master/presto-main/src/main/java/com/facebook/presto/type/VarcharOperators.java#L89-L118) will return `true` for 't' 'true' '1', `false` for 'f' 'false' '0', throw exception for others.
2. [redshift](http://docs.aws.amazon.com/redshift/latest/dg/r_Boolean_type.html) will return `true` for 't' 'true' 'y' 'yes' '1', `false` for 'f' 'false' 'n' 'no' '0', null for others.
3. [postgresql](http://www.postgresql.org/docs/devel/static/datatype-boolean.html) will return `true` for 't' 'true' 'y' 'yes' 'on' '1', `false` for 'f' 'false' 'n' 'no' 'off' '0', throw exception for others.
4. [vertica](https://my.vertica.com/docs/5.0/HTML/Master/2983.htm) will return `true` for 't' 'true' 'y' 'yes' '1', `false` for 'f' 'false' 'n' 'no' '0', null for others.
5. [impala](http://www.cloudera.com/content/cloudera/en/documentation/cloudera-impala/latest/topics/impala_boolean.html) throw exception when try to cast string to boolean.
6. mysql, oracle, sqlserver don't have boolean type

Whether we should change the cast behavior according to other SQL system or not is not decided yet, this PR is a test to see if we changed, how many compatibility tests will fail.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8698 from cloud-fan/string2boolean.
2015-09-11 14:15:16 -07:00
Cheng Lian e1d7f64296 [SPARK-10472] [SQL] Fixes DataType.typeName for UDT
Before this fix, `MyDenseVectorUDT.typeName` gives `mydensevecto`, which is not desirable.

Author: Cheng Lian <lian@databricks.com>

Closes #8640 from liancheng/spark-10472/udt-type-name.
2015-09-11 18:26:56 +08:00
Yash Datta f892d927d7 [SPARK-7142] [SQL] Minor enhancement to BooleanSimplification Optimizer rule
Use these in the optimizer as well:

            A and (not(A) or B) => A and B
            not(A and B) => not(A) or not(B)
            not(A or B) => not(A) and not(B)

Author: Yash Datta <Yash.Datta@guavus.com>

Closes #5700 from saucam/bool_simp.
2015-09-10 10:34:00 -07:00
Wenchen Fan 4f1daa1ef6 [SPARK-10065] [SQL] avoid the extra copy when generate unsafe array
The reason for this extra copy is that we iterate the array twice: calculate elements data size and copy elements to array buffer.

A simple solution is to follow `createCodeForStruct`, we can dynamically grow the buffer when needed and thus don't need to know the data size ahead.

This PR also include some typo and style fixes, and did some minor refactor to make sure `input.primitive` is always variable name not code when generate unsafe code.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8496 from cloud-fan/avoid-copy.
2015-09-10 10:04:38 -07:00
Wenchen Fan 71da1633c4 [SPARK-10461] [SQL] make sure input.primitive is always variable name not code at GenerateUnsafeProjection
When we generate unsafe code inside `createCodeForXXX`, we always assign the `input.primitive` to a temp variable in case `input.primitive` is expression code.

This PR did some refactor to make sure `input.primitive` is always variable name, and some other typo and style fixes.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8613 from cloud-fan/minor.
2015-09-09 10:57:29 -07:00
Luc Bourlier c1bc4f439f [SPARK-10227] fatal warnings with sbt on Scala 2.11
The bulk of the changes are on `transient` annotation on class parameter. Often the compiler doesn't generate a field for this parameters, so the the transient annotation would be unnecessary.
But if the class parameter are used in methods, then fields are created. So it is safer to keep the annotations.

The remainder are some potential bugs, and deprecated syntax.

Author: Luc Bourlier <luc.bourlier@typesafe.com>

Closes #8433 from skyluc/issue/sbt-2.11.
2015-09-09 09:57:58 +01:00
Cheng Hao d637a666d5 [SPARK-10327] [SQL] Cache Table is not working while subquery has alias in its project list
```scala
    import org.apache.spark.sql.hive.execution.HiveTableScan
    sql("select key, value, key + 1 from src").registerTempTable("abc")
    cacheTable("abc")

    val sparkPlan = sql(
      """select a.key, b.key, c.key from
        |abc a join abc b on a.key=b.key
        |join abc c on a.key=c.key""".stripMargin).queryExecution.sparkPlan

    assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e }.size === 3) // failed
    assert(sparkPlan.collect { case e: HiveTableScan => e }.size === 0) // failed
```

The actual plan is:

```
== Parsed Logical Plan ==
'Project [unresolvedalias('a.key),unresolvedalias('b.key),unresolvedalias('c.key)]
 'Join Inner, Some(('a.key = 'c.key))
  'Join Inner, Some(('a.key = 'b.key))
   'UnresolvedRelation [abc], Some(a)
   'UnresolvedRelation [abc], Some(b)
  'UnresolvedRelation [abc], Some(c)

== Analyzed Logical Plan ==
key: int, key: int, key: int
Project [key#14,key#61,key#66]
 Join Inner, Some((key#14 = key#66))
  Join Inner, Some((key#14 = key#61))
   Subquery a
    Subquery abc
     Project [key#14,value#15,(key#14 + 1) AS _c2#16]
      MetastoreRelation default, src, None
   Subquery b
    Subquery abc
     Project [key#61,value#62,(key#61 + 1) AS _c2#58]
      MetastoreRelation default, src, None
  Subquery c
   Subquery abc
    Project [key#66,value#67,(key#66 + 1) AS _c2#63]
     MetastoreRelation default, src, None

== Optimized Logical Plan ==
Project [key#14,key#61,key#66]
 Join Inner, Some((key#14 = key#66))
  Project [key#14,key#61]
   Join Inner, Some((key#14 = key#61))
    Project [key#14]
     InMemoryRelation [key#14,value#15,_c2#16], true, 10000, StorageLevel(true, true, false, true, 1), (Project [key#14,value#15,(key#14 + 1) AS _c2#16]), Some(abc)
    Project [key#61]
     MetastoreRelation default, src, None
  Project [key#66]
   MetastoreRelation default, src, None

== Physical Plan ==
TungstenProject [key#14,key#61,key#66]
 BroadcastHashJoin [key#14], [key#66], BuildRight
  TungstenProject [key#14,key#61]
   BroadcastHashJoin [key#14], [key#61], BuildRight
    ConvertToUnsafe
     InMemoryColumnarTableScan [key#14], (InMemoryRelation [key#14,value#15,_c2#16], true, 10000, StorageLevel(true, true, false, true, 1), (Project [key#14,value#15,(key#14 + 1) AS _c2#16]), Some(abc))
    ConvertToUnsafe
     HiveTableScan [key#61], (MetastoreRelation default, src, None)
  ConvertToUnsafe
   HiveTableScan [key#66], (MetastoreRelation default, src, None)
```

Author: Cheng Hao <hao.cheng@intel.com>

Closes #8494 from chenghao-intel/weird_cache.
2015-09-08 16:16:50 -07:00
Yin Huai 7a9dcbc91d [SPARK-10441] [SQL] Save data correctly to json.
https://issues.apache.org/jira/browse/SPARK-10441

Author: Yin Huai <yhuai@databricks.com>

Closes #8597 from yhuai/timestampJson.
2015-09-08 14:10:12 -07:00
Wenchen Fan 5fd57955ef [SPARK-10316] [SQL] respect nondeterministic expressions in PhysicalOperation
We did a lot of special handling for non-deterministic expressions in `Optimizer`. However, `PhysicalOperation` just collects all Projects and Filters and mess it up. We should respect the operators order caused by non-deterministic expressions in `PhysicalOperation`.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8486 from cloud-fan/fix.
2015-09-08 12:05:41 -07:00
Andrew Or 3339e6f674 [SPARK-10450] [SQL] Minor improvements to readability / style / typos etc.
Author: Andrew Or <andrew@databricks.com>

Closes #8603 from andrewor14/minor-sql-changes.
2015-09-04 15:20:20 -07:00
Wenchen Fan c3c0e431a6 [SPARK-10176] [SQL] Show partially analyzed plans when checkAnswer fails to analyze
This PR takes over https://github.com/apache/spark/pull/8389.

This PR improves `checkAnswer` to print the partially analyzed plan in addition to the user friendly error message, in order to aid debugging failing tests.

In doing so, I ran into a conflict with the various ways that we bring a SQLContext into the tests. Depending on the trait we refer to the current context as `sqlContext`, `_sqlContext`, `ctx` or `hiveContext` with access modifiers `public`, `protected` and `private` depending on the defining class.

I propose we refactor as follows:

1. All tests should only refer to a `protected sqlContext` when testing general features, and `protected hiveContext` when it is a method that only exists on a `HiveContext`.
2. All tests should only import `testImplicits._` (i.e., don't import `TestHive.implicits._`)

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8584 from cloud-fan/cleanupTests.
2015-09-04 15:17:37 -07:00
Wenchen Fan fc48307797 [SPARK-10389] [SQL] support order by non-attribute grouping expression on Aggregate
For example, we can write `SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1` in PostgreSQL, and we should support this in Spark SQL.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8548 from cloud-fan/support-order-by-non-attribute.
2015-09-02 11:32:27 -07:00
Davies Liu bb7f352393 [SPARK-10323] [SQL] fix nullability of In/InSet/ArrayContain
After this PR, In/InSet/ArrayContain will return null if value is null, instead of false. They also will return null even if there is a null in the set/array.

Author: Davies Liu <davies@databricks.com>

Closes #8492 from davies/fix_in.
2015-08-28 14:38:20 -07:00
Josh Rosen d3f87dc394 [SPARK-10325] Override hashCode() for public Row
This commit fixes an issue where the public SQL `Row` class did not override `hashCode`, causing it to violate the hashCode() + equals() contract. To fix this, I simply ported the `hashCode` implementation from the 1.4.x version of `Row`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8500 from JoshRosen/SPARK-10325 and squashes the following commits:

51ffea1 [Josh Rosen] Override hashCode() for public Row.
2015-08-28 11:51:42 -07:00
Davies Liu 7467b52ed0 [SPARK-10215] [SQL] Fix precision of division (follow the rule in Hive)
Follow the rule in Hive for decimal division. see ac755ebe26/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPDivide.java (L113)

cc chenghao-intel

Author: Davies Liu <davies@databricks.com>

Closes #8415 from davies/decimal_div2.
2015-08-25 15:20:24 -07:00
Davies Liu ec89bd840a [SPARK-10245] [SQL] Fix decimal literals with precision < scale
In BigDecimal or java.math.BigDecimal, the precision could be smaller than scale, for example, BigDecimal("0.001") has precision = 1 and scale = 3. But DecimalType require that the precision should be larger than scale, so we should use the maximum of precision and scale when inferring the schema from decimal literal.

Author: Davies Liu <davies@databricks.com>

Closes #8428 from davies/smaller_decimal.
2015-08-25 14:55:34 -07:00
Sean Owen 69c9c17716 [SPARK-9613] [CORE] Ban use of JavaConversions and migrate all existing uses to JavaConverters
Replace `JavaConversions` implicits with `JavaConverters`

Most occurrences I've seen so far are necessary conversions; a few have been avoidable. None are in critical code as far as I see, yet.

Author: Sean Owen <sowen@cloudera.com>

Closes #8033 from srowen/SPARK-9613.
2015-08-25 12:33:13 +01:00
Davies Liu 2f493f7e39 [SPARK-10177] [SQL] fix reading Timestamp in parquet from Hive
We misunderstood the Julian days and nanoseconds of the day in parquet (as TimestampType) from Hive/Impala, they are overlapped, so can't be added together directly.

In order to avoid the confusing rounding when do the converting, we use `2440588` as the Julian Day of epoch of unix timestamp (which should be 2440587.5).

Author: Davies Liu <davies@databricks.com>
Author: Cheng Lian <lian@databricks.com>

Closes #8400 from davies/timestamp_parquet.
2015-08-25 16:00:44 +08:00
Josh Rosen 82268f07ab [SPARK-9293] [SPARK-9813] Analysis should check that set operations are only performed on tables with equal numbers of columns
This patch adds an analyzer rule to ensure that set operations (union, intersect, and except) are only applied to tables with the same number of columns. Without this rule, there are scenarios where invalid queries can return incorrect results instead of failing with error messages; SPARK-9813 provides one example of this problem. In other cases, the invalid query can crash at runtime with extremely confusing exceptions.

I also performed a bit of cleanup to refactor some of those logical operators' code into a common `SetOperation` base class.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7631 from JoshRosen/SPARK-9293.
2015-08-25 00:04:10 -07:00
Michael Armbrust 2bf338c626 [SPARK-10165] [SQL] Await child resolution in ResolveFunctions
Currently, we eagerly attempt to resolve functions, even before their children are resolved.  However, this is not valid in cases where we need to know the types of the input arguments (i.e. when resolving Hive UDFs).

As a fix, this PR delays function resolution until the functions children are resolved.  This change also necessitates a change to the way we resolve aggregate expressions that are not in aggregate operators (e.g., in `HAVING` or `ORDER BY` clauses).  Specifically, we can't assume that these misplaced functions will be resolved, allowing us to differentiate aggregate functions from normal functions.  To compensate for this change we now attempt to resolve these unresolved expressions in the context of the aggregate operator, before checking to see if any aggregate expressions are present.

Author: Michael Armbrust <michael@databricks.com>

Closes #8371 from marmbrus/hiveUDFResolution.
2015-08-24 18:10:51 -07:00
Josh Rosen d7b4c09527 [SPARK-10190] Fix NPE in CatalystTypeConverters Decimal toScala converter
This adds a missing null check to the Decimal `toScala` converter in `CatalystTypeConverters`, fixing an NPE.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8401 from JoshRosen/SPARK-10190.
2015-08-24 16:17:45 -07:00
Yijie Shen 90cb9f0565 [SPARK-9401] [SQL] Fully implement code generation for ConcatWs
This PR adds full codegen support for ConcatWs, is a substitute of #7782

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

cc davies

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #8353 from yjshen/concatws.
2015-08-22 10:16:35 -07:00
Daoyuan Wang 3c462f5d87 [SPARK-10130] [SQL] type coercion for IF should have children resolved first
Type coercion for IF should have children resolved first, or we could meet unresolved exception.

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

Closes #8331 from adrian-wang/spark10130.
2015-08-21 12:21:51 -07:00
Tarek Auel afe9f03fd9 [SPARK-9400] [SQL] codegen for StringLocate
This is based on #7779 , thanks to tarekauel . Fix the conflict and nullability.

Closes #7779 and #8274 .

Author: Tarek Auel <tarek.auel@googlemail.com>
Author: Davies Liu <davies@databricks.com>

Closes #8330 from davies/stringLocate.
2015-08-20 15:10:13 -07:00
Yin Huai 43e0135421 [SPARK-10092] [SQL] Multi-DB support follow up.
https://issues.apache.org/jira/browse/SPARK-10092

This pr is a follow-up one for Multi-DB support. It has the following changes:

* `HiveContext.refreshTable` now accepts `dbName.tableName`.
* `HiveContext.analyze` now accepts `dbName.tableName`.
* `CreateTableUsing`, `CreateTableUsingAsSelect`, `CreateTempTableUsing`, `CreateTempTableUsingAsSelect`, `CreateMetastoreDataSource`, and `CreateMetastoreDataSourceAsSelect` all take `TableIdentifier` instead of the string representation of table name.
* When you call `saveAsTable` with a specified database, the data will be saved to the correct location.
* Explicitly do not allow users to create a temporary with a specified database name (users cannot do it before).
* When we save table to metastore, we also check if db name and table name can be accepted by hive (using `MetaStoreUtils.validateName`).

Author: Yin Huai <yhuai@databricks.com>

Closes #8324 from yhuai/saveAsTableDB.
2015-08-20 15:30:31 +08:00
Reynold Xin 2f2686a73f [SPARK-9242] [SQL] Audit UDAF interface.
A few minor changes:

1. Improved documentation
2. Rename apply(distinct....) to distinct.
3. Changed MutableAggregationBuffer from a trait to an abstract class.
4. Renamed returnDataType to dataType to be more consistent with other expressions.

And unrelated to UDAFs:

1. Renamed file names in expressions to use suffix "Expressions" to be more consistent.
2. Moved regexp related expressions out to its own file.
3. Renamed StringComparison => StringPredicate.

Author: Reynold Xin <rxin@databricks.com>

Closes #8321 from rxin/SPARK-9242.
2015-08-19 17:35:41 -07:00
Wenchen Fan b0dbaec4f9 [SPARK-6489] [SQL] add column pruning for Generate
This PR takes over https://github.com/apache/spark/pull/5358

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8268 from cloud-fan/6489.
2015-08-19 15:05:06 -07:00
Daoyuan Wang 373a376c04 [SPARK-10083] [SQL] CaseWhen should support type coercion of DecimalType and FractionalType
create t1 (a decimal(7, 2), b long);
select case when 1=1 then a else 1.0 end from t1;
select case when 1=1 then a else b end from t1;

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

Closes #8270 from adrian-wang/casewhenfractional.
2015-08-19 14:31:51 -07:00
Davies Liu 1f4c4fe6df [SPARK-10090] [SQL] fix decimal scale of division
We should rounding the result of multiply/division of decimal to expected precision/scale, also check overflow.

Author: Davies Liu <davies@databricks.com>

Closes #8287 from davies/decimal_division.
2015-08-19 14:03:47 -07:00
Davies Liu e05da5cb5e [SPARK-10107] [SQL] fix NPE in format_number
Author: Davies Liu <davies@databricks.com>

Closes #8305 from davies/format_number.
2015-08-19 13:43:04 -07:00
Reynold Xin 1ff0580eda [SPARK-10093] [SPARK-10096] [SQL] Avoid transformation on executors & fix UDFs on complex types
This is kind of a weird case, but given a sufficiently complex query plan (in this case a TungstenProject with an Exchange underneath), we could have NPEs on the executors due to the time when we were calling transformAllExpressions

In general we should ensure that all transformations occur on the driver and not on the executors. Some reasons for avoid executor side transformations include:

* (this case) Some operator constructors require state such as access to the Spark/SQL conf so doing a makeCopy on the executor can fail.
* (unrelated reason for avoid executor transformations) ExprIds are calculated using an atomic integer, so you can violate their uniqueness constraint by constructing them anywhere other than the driver.

This subsumes #8285.

Author: Reynold Xin <rxin@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #8295 from rxin/SPARK-10096.
2015-08-18 22:08:15 -07:00
Davies Liu 270ee67775 [SPARK-10095] [SQL] use public API of BigInteger
In UnsafeRow, we use the private field of BigInteger for better performance, but it actually didn't contribute much (3% in one benchmark) to end-to-end runtime, and make it not portable (may fail on other JVM implementations).

So we should use the public API instead.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #8286 from davies/portable_decimal.
2015-08-18 20:39:59 -07:00
Davies Liu 5af3838d2e [SPARK-10038] [SQL] fix bug in generated unsafe projection when there is binary in ArrayData
The type for array of array in Java is slightly different than array of others.

cc cloud-fan

Author: Davies Liu <davies@databricks.com>

Closes #8250 from davies/array_binary.
2015-08-17 23:27:55 -07:00
Yin Huai 772e7c18fb [SPARK-9592] [SQL] Fix Last function implemented based on AggregateExpression1.
https://issues.apache.org/jira/browse/SPARK-9592

#8113 has the fundamental fix. But, if we want to minimize the number of changed lines, we can go with this one. Then, in 1.6, we merge #8113.

Author: Yin Huai <yhuai@databricks.com>

Closes #8172 from yhuai/lastFix and squashes the following commits:

b28c42a [Yin Huai] Regression test.
af87086 [Yin Huai] Fix last.
2015-08-17 15:30:50 -07:00
Yijie Shen b265e282b6 [SPARK-9526] [SQL] Utilize randomized tests to reveal potential bugs in sql expressions
JIRA: https://issues.apache.org/jira/browse/SPARK-9526

This PR is a follow up of #7830, aiming at utilizing randomized tests to reveal more potential bugs in sql expression.

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7855 from yjshen/property_check.
2015-08-17 14:10:19 -07:00
Wenchen Fan 570567258b [SPARK-9955] [SQL] correct error message for aggregate
We should skip unresolved `LogicalPlan`s for `PullOutNondeterministic`, as calling `output` on unresolved `LogicalPlan` will produce confusing error message.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8203 from cloud-fan/error-msg and squashes the following commits:

1c67ca7 [Wenchen Fan] move test
7593080 [Wenchen Fan] correct error message for aggregate
2015-08-15 14:13:12 -07:00
Wenchen Fan ec29f2034a [SPARK-9634] [SPARK-9323] [SQL] cleanup unnecessary Aliases in LogicalPlan at the end of analysis
Also alias the ExtractValue instead of wrapping it with UnresolvedAlias when resolve attribute in LogicalPlan, as this alias will be trimmed if it's unnecessary.

Based on #7957 without the changes to mllib, but instead maintaining earlier behavior when using `withColumn` on expressions that already have metadata.

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

Closes #8215 from marmbrus/pr/7957.
2015-08-14 20:59:54 -07:00
Davies Liu 37586e5449 [HOTFIX] fix duplicated braces
Author: Davies Liu <davies@databricks.com>

Closes #8219 from davies/fix_typo.
2015-08-14 20:56:55 -07:00
Liang-Chi Hsieh 7c7c7529a1 [MINOR] [SQL] Remove canEqual in Row
As `InternalRow` does not extend `Row` now, I think we can remove it.

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

Closes #8170 from viirya/remove_canequal.
2015-08-13 22:06:09 -07:00
Andrew Or 8187b3ae47 [SPARK-9580] [SQL] Replace singletons in SQL tests
A fundamental limitation of the existing SQL tests is that *there is simply no way to create your own `SparkContext`*. This is a serious limitation because the user may wish to use a different master or config. As a case in point, `BroadcastJoinSuite` is entirely commented out because there is no way to make it pass with the existing infrastructure.

This patch removes the singletons `TestSQLContext` and `TestData`, and instead introduces a `SharedSQLContext` that starts a context per suite. Unfortunately the singletons were so ingrained in the SQL tests that this patch necessarily needed to touch *all* the SQL test files.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/8111)
<!-- Reviewable:end -->

Author: Andrew Or <andrew@databricks.com>

Closes #8111 from andrewor14/sql-tests-refactor.
2015-08-13 17:42:01 -07:00
Cheng Lian 6993031011 [SPARK-9757] [SQL] Fixes persistence of Parquet relation with decimal column
PR #7967 enables us to save data source relations to metastore in Hive compatible format when possible. But it fails to persist Parquet relations with decimal column(s) to Hive metastore of versions lower than 1.2.0. This is because `ParquetHiveSerDe` in Hive versions prior to 1.2.0 doesn't support decimal. This PR checks for this case and falls back to Spark SQL specific metastore table format.

Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Lian <lian@databricks.com>

Closes #8130 from liancheng/spark-9757/old-hive-parquet-decimal.
2015-08-13 16:16:50 +08:00
Josh Rosen b1581ac288 [SPARK-9854] [SQL] RuleExecutor.timeMap should be thread-safe
`RuleExecutor.timeMap` is currently a non-thread-safe mutable HashMap; this can lead to infinite loops if multiple threads are concurrently modifying the map.  I believe that this is responsible for some hangs that I've observed in HiveQuerySuite.

This patch addresses this by using a Guava `AtomicLongMap`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8120 from JoshRosen/rule-executor-time-map-fix.
2015-08-11 22:46:59 -07:00
Josh Rosen dfe347d2ca [SPARK-9785] [SQL] HashPartitioning compatibility should consider expression ordering
HashPartitioning compatibility is currently defined w.r.t the _set_ of expressions, but the ordering of those expressions matters when computing hash codes; this could lead to incorrect answers if we mistakenly avoided a shuffle based on the assumption that HashPartitionings with the same expressions in different orders will produce equivalent row hashcodes. The first commit adds a regression test which illustrates this problem.

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

Author: Josh Rosen <joshrosen@databricks.com>

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

b61412f [Josh Rosen] Demonstrate that I haven't cheated in my fix
0b4d7d9 [Josh Rosen] Update so that clusteringSet is only used in satisfies().
dc9c9d7 [Josh Rosen] Add failing regression test for SPARK-9785
2015-08-11 08:52:15 -07:00
Reynold Xin d378396f86 [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
PlatformDependent.UNSAFE is way too verbose.

Author: Reynold Xin <rxin@databricks.com>

Closes #8094 from rxin/SPARK-9815 and squashes the following commits:

229b603 [Reynold Xin] [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
2015-08-11 08:41:06 -07:00
Josh Rosen 91e9389f39 [SPARK-9729] [SPARK-9363] [SQL] Use sort merge join for left and right outer join
This patch adds a new `SortMergeOuterJoin` operator that performs left and right outer joins using sort merge join.  It also refactors `SortMergeJoin` in order to improve performance and code clarity.

Along the way, I also performed a couple pieces of minor cleanup and optimization:

- Rename the `HashJoin` physical planner rule to `EquiJoinSelection`, since it's also used for non-hash joins.
- Rewrite the comment at the top of `HashJoin` to better explain the precedence for choosing join operators.
- Update `JoinSuite` to use `SqlTestUtils.withConf` for changing SQLConf settings.

This patch incorporates several ideas from adrian-wang's patch, #5717.

Closes #5717.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7904)
<!-- Reviewable:end -->

Author: Josh Rosen <joshrosen@databricks.com>
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #7904 from JoshRosen/outer-join-smj and squashes 1 commits.
2015-08-10 22:04:41 -07:00
Davies Liu c4fd2a2422 [SPARK-9759] [SQL] improve decimal.times() and cast(int, decimalType)
This patch optimize two things:

1. passing MathContext to JavaBigDecimal.multiply/divide/reminder to do right rounding, because java.math.BigDecimal.apply(MathContext) is expensive

2. Cast integer/short/byte to decimal directly (without double)

This two optimizations could speed up the end-to-end time of a aggregation (SUM(short * decimal(5, 2)) 75% (from 19s -> 10.8s)

Author: Davies Liu <davies@databricks.com>

Closes #8052 from davies/optimize_decimal and squashes the following commits:

225efad [Davies Liu] improve decimal.times() and cast(int, decimalType)
2015-08-10 13:55:11 -07:00
Davies Liu fe2fb7fb71 [SPARK-9620] [SQL] generated UnsafeProjection should support many columns or large exressions
Currently, generated UnsafeProjection can reach 64k byte code limit of Java. This patch will split the generated expressions into multiple functions, to avoid the limitation.

After this patch, we can work well with table that have up to 64k columns (hit max number of constants limit in Java), it should be enough in practice.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #8044 from davies/wider_table and squashes the following commits:

9192e6c [Davies Liu] fix generated safe projection
d1ef81a [Davies Liu] fix failed tests
737b3d3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
ffcd132 [Davies Liu] address comments
1b95be4 [Davies Liu] put the generated class into sql package
77ed72d [Davies Liu] address comments
4518e17 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
75ccd01 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
495e932 [Davies Liu] support wider table with more than 1k columns for generated projections
2015-08-10 13:52:18 -07:00
Josh Rosen 23cf5af08d [SPARK-9703] [SQL] Refactor EnsureRequirements to avoid certain unnecessary shuffles
This pull request refactors the `EnsureRequirements` planning rule in order to avoid the addition of certain unnecessary shuffles.

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

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

Author: Josh Rosen <joshrosen@databricks.com>

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

38006e7 [Josh Rosen] Rewrite EnsureRequirements _yet again_ to make things even simpler
0983f75 [Josh Rosen] More guarantees vs. compatibleWith cleanup; delete BroadcastPartitioning.
8784bd9 [Josh Rosen] Giant comment explaining compatibleWith vs. guarantees
1307c50 [Josh Rosen] Update conditions for requiring child compatibility.
18cddeb [Josh Rosen] Rename DummyPlan to DummySparkPlan.
2c7e126 [Josh Rosen] Merge remote-tracking branch 'origin/master' into exchange-fixes
fee65c4 [Josh Rosen] Further refinement to comments / reasoning
642b0bb [Josh Rosen] Further expand comment / reasoning
06aba0c [Josh Rosen] Add more comments
8dbc845 [Josh Rosen] Add even more tests.
4f08278 [Josh Rosen] Fix the test by adding the compatibility check to EnsureRequirements
a1c12b9 [Josh Rosen] Add failing test to demonstrate allCompatible bug
0725a34 [Josh Rosen] Small assertion cleanup.
5172ac5 [Josh Rosen] Add test for requiresChildrenToProduceSameNumberOfPartitions.
2e0f33a [Josh Rosen] Write a more generic test for EnsureRequirements.
752b8de [Josh Rosen] style fix
c628daf [Josh Rosen] Revert accidental ExchangeSuite change.
c9fb231 [Josh Rosen] Rewrite exchange to fix better handle this case.
adcc742 [Josh Rosen] Move test to PlannerSuite.
0675956 [Josh Rosen] Preserving ordering and partitioning in row format converters also does not help.
cc5669c [Josh Rosen] Adding outputPartitioning to Repartition does not fix the test.
2dfc648 [Josh Rosen] Add failing test illustrating bad exchange planning.
2015-08-09 14:26:01 -07:00
Yijie Shen 68ccc6e184 [SPARK-8930] [SQL] Throw a AnalysisException with meaningful messages if DataFrame#explode takes a star in expressions
Author: Yijie Shen <henry.yijieshen@gmail.com>

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

eae181d [Yijie Shen] change explaination message
54c9d11 [Yijie Shen] meaning message for * in explode
2015-08-09 11:44:51 -07:00
Wenchen Fan 106c0789d8 [SPARK-9738] [SQL] remove FromUnsafe and add its codegen version to GenerateSafe
In https://github.com/apache/spark/pull/7752 we added `FromUnsafe` to convert nexted unsafe data like array/map/struct to safe versions. It's a quick solution and we already have `GenerateSafe` to do the conversion which is codegened. So we should remove `FromUnsafe` and implement its codegen version in `GenerateSafe`.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8029 from cloud-fan/from-unsafe and squashes the following commits:

ed40d8f [Wenchen Fan] add the copy back
a93fd4b [Wenchen Fan] cogengen FromUnsafe
2015-08-08 08:33:14 -07:00
Reynold Xin 05d04e10a8 [SPARK-9733][SQL] Improve physical plan explain for data sources
All data sources show up as "PhysicalRDD" in physical plan explain. It'd be better if we can show the name of the data source.

Without this patch:
```
== Physical Plan ==
NewAggregate with UnsafeHybridAggregationIterator ArrayBuffer(date#0, cat#1) ArrayBuffer((sum(CAST((CAST(count#2, IntegerType) + 1), LongType))2,mode=Final,isDistinct=false))
 Exchange hashpartitioning(date#0,cat#1)
  NewAggregate with UnsafeHybridAggregationIterator ArrayBuffer(date#0, cat#1) ArrayBuffer((sum(CAST((CAST(count#2, IntegerType) + 1), LongType))2,mode=Partial,isDistinct=false))
   PhysicalRDD [date#0,cat#1,count#2], MapPartitionsRDD[3] at
```

With this patch:
```
== Physical Plan ==
TungstenAggregate(key=[date#0,cat#1], value=[(sum(CAST((CAST(count#2, IntegerType) + 1), LongType)),mode=Final,isDistinct=false)]
 Exchange hashpartitioning(date#0,cat#1)
  TungstenAggregate(key=[date#0,cat#1], value=[(sum(CAST((CAST(count#2, IntegerType) + 1), LongType)),mode=Partial,isDistinct=false)]
   ConvertToUnsafe
    Scan ParquetRelation[file:/scratch/rxin/spark/sales4][date#0,cat#1,count#2]
```

Author: Reynold Xin <rxin@databricks.com>

Closes #8024 from rxin/SPARK-9733 and squashes the following commits:

811b90e [Reynold Xin] Fixed Python test case.
52cab77 [Reynold Xin] Cast.
eea9ccc [Reynold Xin] Fix test case.
fcecb22 [Reynold Xin] [SPARK-9733][SQL] Improve explain message for data source scan node.
2015-08-07 13:41:45 -07:00
Reynold Xin 9897cc5e3d [SPARK-9736] [SQL] JoinedRow.anyNull should delegate to the underlying rows.
JoinedRow.anyNull currently loops through every field to check for null, which is inefficient if the underlying rows are UnsafeRows. It should just delegate to the underlying implementation.

Author: Reynold Xin <rxin@databricks.com>

Closes #8027 from rxin/SPARK-9736 and squashes the following commits:

03a2e92 [Reynold Xin] Include all files.
90f1add [Reynold Xin] [SPARK-9736][SQL] JoinedRow.anyNull should delegate to the underlying rows.
2015-08-07 11:29:13 -07:00
Wenchen Fan 2432c2e239 [SPARK-8382] [SQL] Improve Analysis Unit test framework
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8025 from cloud-fan/analysis and squashes the following commits:

51461b1 [Wenchen Fan] move test file to test folder
ec88ace [Wenchen Fan] Improve Analysis Unit test framework
2015-08-07 11:28:43 -07:00
Wenchen Fan e57d6b5613 [SPARK-9683] [SQL] copy UTF8String when convert unsafe array/map to safe
When we convert unsafe row to safe row, we will do copy if the column is struct or string type. However, the string inside unsafe array/map are not copied, which may cause problems.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7990 from cloud-fan/copy and squashes the following commits:

c13d1e3 [Wenchen Fan] change test name
fe36294 [Wenchen Fan] we should deep copy UTF8String when convert unsafe row to safe row
2015-08-07 00:00:43 -07:00
Michael Armbrust 0867b23c74 [SPARK-9650][SQL] Fix quoting behavior on interpolated column names
Make sure that `$"column"` is consistent with other methods with respect to backticks.  Adds a bunch of tests for various ways of constructing columns.

Author: Michael Armbrust <michael@databricks.com>

Closes #7969 from marmbrus/namesWithDots and squashes the following commits:

53ef3d7 [Michael Armbrust] [SPARK-9650][SQL] Fix quoting behavior on interpolated column names
2bf7a92 [Michael Armbrust] WIP
2015-08-06 17:31:16 -07:00
Yin Huai 3504bf3aa9 [SPARK-9630] [SQL] Clean up new aggregate operators (SPARK-9240 follow up)
This is the followup of https://github.com/apache/spark/pull/7813. It renames `HybridUnsafeAggregationIterator` to `TungstenAggregationIterator` and makes it only work with `UnsafeRow`. Also, I add a `TungstenAggregate` that uses `TungstenAggregationIterator` and make `SortBasedAggregate` (renamed from `SortBasedAggregate`) only works with `SafeRow`.

Author: Yin Huai <yhuai@databricks.com>

Closes #7954 from yhuai/agg-followUp and squashes the following commits:

4d2f4fc [Yin Huai] Add comments and free map.
0d7ddb9 [Yin Huai] Add TungstenAggregationQueryWithControlledFallbackSuite to test fall back process.
91d69c2 [Yin Huai] Rename UnsafeHybridAggregationIterator to  TungstenAggregateIteraotr and make it only work with UnsafeRow.
2015-08-06 15:04:44 -07:00
Wenchen Fan 1f62f104c7 [SPARK-9632][SQL] update InternalRow.toSeq to make it accept data type info
This re-applies #7955, which was reverted due to a race condition to fix build breaking.

Author: Wenchen Fan <cloud0fan@outlook.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #8002 from rxin/InternalRow-toSeq and squashes the following commits:

332416a [Reynold Xin] Merge pull request #7955 from cloud-fan/toSeq
21665e2 [Wenchen Fan] fix hive again...
4addf29 [Wenchen Fan] fix hive
bc16c59 [Wenchen Fan] minor fix
33d802c [Wenchen Fan] pass data type info to InternalRow.toSeq
3dd033e [Wenchen Fan] move the default special getters implementation from InternalRow to BaseGenericInternalRow
2015-08-06 13:11:59 -07:00
Yin Huai cdd53b762b [SPARK-9632] [SQL] [HOT-FIX] Fix build.
seems https://github.com/apache/spark/pull/7955 breaks the build.

Author: Yin Huai <yhuai@databricks.com>

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

6c257dd [Yin Huai] Fix build.
2015-08-06 11:15:54 -07:00
Davies Liu 2eca46a17a Revert "[SPARK-9632][SQL] update InternalRow.toSeq to make it accept data type info"
This reverts commit 6e009cb9c4.
2015-08-06 11:15:37 -07:00
Wenchen Fan 6e009cb9c4 [SPARK-9632][SQL] update InternalRow.toSeq to make it accept data type info
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7955 from cloud-fan/toSeq and squashes the following commits:

21665e2 [Wenchen Fan] fix hive again...
4addf29 [Wenchen Fan] fix hive
bc16c59 [Wenchen Fan] minor fix
33d802c [Wenchen Fan] pass data type info to InternalRow.toSeq
3dd033e [Wenchen Fan] move the default special getters implementation from InternalRow to BaseGenericInternalRow
2015-08-06 10:40:54 -07:00
Davies Liu 5b965d64ee [SPARK-9644] [SQL] Support update DecimalType with precision > 18 in UnsafeRow
In order to support update a varlength (actually fixed length) object, the space should be preserved even  it's null. And, we can't call setNullAt(i) for it anymore, we because setNullAt(i) will remove the offset of the preserved space, should call setDecimal(i, null, precision) instead.

After this, we can do hash based aggregation on DecimalType with precision > 18. In a tests, this could decrease the end-to-end run time of aggregation query from 37 seconds (sort based) to 24 seconds (hash based).

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7978 from davies/update_decimal and squashes the following commits:

bed8100 [Davies Liu] isSettable -> isMutable
923c9eb [Davies Liu] address comments and fix bug
385891d [Davies Liu] Merge branch 'master' of github.com:apache/spark into update_decimal
36a1872 [Davies Liu] fix tests
cd6c524 [Davies Liu] support set decimal with precision > 18
2015-08-06 09:10:57 -07:00
zhichao.li aead18ffca [SPARK-8266] [SQL] add function translate
![translate](http://www.w3resource.com/PostgreSQL/postgresql-translate-function.png)

Author: zhichao.li <zhichao.li@intel.com>

Closes #7709 from zhichao-li/translate and squashes the following commits:

9418088 [zhichao.li] refine checking condition
f2ab77a [zhichao.li] clone string
9d88f2d [zhichao.li] fix indent
6aa2962 [zhichao.li] style
e575ead [zhichao.li] add python api
9d4bab0 [zhichao.li] add special case for fodable and refactor unittest
eda7ad6 [zhichao.li] update to use TernaryExpression
cdfd4be [zhichao.li] add function translate
2015-08-06 09:02:30 -07:00
Josh Rosen 9c878923db [SPARK-9054] [SQL] Rename RowOrdering to InterpretedOrdering; use newOrdering in SMJ
This patches renames `RowOrdering` to `InterpretedOrdering` and updates SortMergeJoin to use the `SparkPlan` methods for constructing its ordering so that it may benefit from codegen.

This is an updated version of #7408.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7973 from JoshRosen/SPARK-9054 and squashes the following commits:

e610655 [Josh Rosen] Add comment RE: Ascending ordering
34b8e0c [Josh Rosen] Import ordering
be19a0f [Josh Rosen] [SPARK-9054] [SQL] Rename RowOrdering to InterpretedOrdering; use newOrdering in more places.
2015-08-05 16:33:42 -07:00
Liang-Chi Hsieh e1e05873fc [SPARK-9403] [SQL] Add codegen support in In and InSet
This continues tarekauel's work in #7778.

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Tarek Auel <tarek.auel@googlemail.com>

Closes #7893 from viirya/codegen_in and squashes the following commits:

81ff97b [Liang-Chi Hsieh] For comments.
47761c6 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
cf4bf41 [Liang-Chi Hsieh] For comments.
f532b3c [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
446bbcd [Liang-Chi Hsieh] Fix bug.
b3d0ab4 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
4610eff [Liang-Chi Hsieh] Relax the types of references and update optimizer test.
224f18e [Liang-Chi Hsieh] Beef up the test cases for In and InSet to include all primitive data types.
86dc8aa [Liang-Chi Hsieh] Only convert In to InSet when the number of items in set is more than the threshold.
b7ded7e [Tarek Auel] [SPARK-9403][SQL] codeGen in / inSet
2015-08-05 11:38:56 -07:00
Yin Huai 1f8c364b9c [SPARK-9141] [SQL] [MINOR] Fix comments of PR #7920
This is a follow-up of https://github.com/apache/spark/pull/7920 to fix comments.

Author: Yin Huai <yhuai@databricks.com>

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

4d0ee80 [Yin Huai] Fix comments.
2015-08-05 11:03:02 -07:00
Michael Armbrust 23d982204b [SPARK-9141] [SQL] Remove project collapsing from DataFrame API
Currently we collapse successive projections that are added by `withColumn`.  However, this optimization violates the constraint that adding nodes to a plan will never change its analyzed form and thus breaks caching.  Instead of doing early optimization, in this PR I just fix some low-hanging slowness in the analyzer.  In particular, I add a mechanism for skipping already analyzed subplans, `resolveOperators` and `resolveExpression`.  Since trees are generally immutable after construction, it's safe to annotate a plan as already analyzed as any transformation will create a new tree with this bit no longer set.

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

2145031 [Michael Armbrust] fix hive udfs tests
5a5a525 [Michael Armbrust] remove wrong comment
7a507d5 [Michael Armbrust] style
b59d710 [Michael Armbrust] revert small change
1fa5949 [Michael Armbrust] move logic into LogicalPlan, add tests
0e2cb43 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into withColumnCache
c926e24 [Michael Armbrust] naming
e593a2d [Michael Armbrust] style
f5a929e [Michael Armbrust] [SPARK-9141][SQL] Remove project collapsing from DataFrame API
38b1c83 [Michael Armbrust] WIP
2015-08-05 09:01:45 -07:00
Yijie Shen 84ca3183b6 [SPARK-9628][SQL]Rename int to SQLDate, long to SQLTimestamp for better readability
JIRA: https://issues.apache.org/jira/browse/SPARK-9628

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7953 from yjshen/datetime_alias and squashes the following commits:

3cac3cc [Yijie Shen] rename int to SQLDate, long to SQLTimestamp for better readability
2015-08-05 02:04:28 -07:00
Takeshi YAMAMURO 6d8a6e4161 [SPARK-9360] [SQL] Support BinaryType in PrefixComparators for UnsafeExternalSort
The current implementation of UnsafeExternalSort uses NoOpPrefixComparator for binary-typed data.
So, we need to add BinaryPrefixComparator in PrefixComparators.

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

Closes #7676 from maropu/BinaryTypePrefixComparator and squashes the following commits:

fe6f31b [Takeshi YAMAMURO] Apply comments
d943c04 [Takeshi YAMAMURO] Add a codegen'd entry for BinaryType in SortPrefix
ecf3ac5 [Takeshi YAMAMURO] Support BinaryType in PrefixComparator
2015-08-05 00:56:35 -07:00
Davies Liu 781c8d71a0 [SPARK-9119] [SPARK-8359] [SQL] match Decimal.precision/scale with DecimalType
Let Decimal carry the correct precision and scale with DecimalType.

cc rxin yhuai

Author: Davies Liu <davies@databricks.com>

Closes #7925 from davies/decimal_scale and squashes the following commits:

e19701a [Davies Liu] some tweaks
57d78d2 [Davies Liu] fix tests
5d5bc69 [Davies Liu] match precision and scale with DecimalType
2015-08-04 23:12:49 -07:00
Pedro Rodriguez d34548587a [SPARK-8231] [SQL] Add array_contains
This PR is based on #7580 , thanks to EntilZha

PR for work on https://issues.apache.org/jira/browse/SPARK-8231

Currently, I have an initial implementation for contains. Based on discussion on JIRA, it should behave same as Hive: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArrayContains.java#L102-L128

Main points are:
1. If the array is empty, null, or the value is null, return false
2. If there is a type mismatch, throw error
3. If comparison is not supported, throw error

Closes #7580

Author: Pedro Rodriguez <prodriguez@trulia.com>
Author: Pedro Rodriguez <ski.rodriguez@gmail.com>
Author: Davies Liu <davies@databricks.com>

Closes #7949 from davies/array_contains and squashes the following commits:

d3c08bc [Davies Liu] use foreach() to avoid copy
bc3d1fe [Davies Liu] fix array_contains
719e37d [Davies Liu] Merge branch 'master' of github.com:apache/spark into array_contains
e352cf9 [Pedro Rodriguez] fixed diff from master
4d5b0ff [Pedro Rodriguez] added docs and another type check
ffc0591 [Pedro Rodriguez] fixed unit test
7a22deb [Pedro Rodriguez] Changed test to use strings instead of long/ints which are different between python 2 an 3
b5ffae8 [Pedro Rodriguez] fixed pyspark test
4e7dce3 [Pedro Rodriguez] added more docs
3082399 [Pedro Rodriguez] fixed unit test
46f9789 [Pedro Rodriguez] reverted change
d3ca013 [Pedro Rodriguez] Fixed type checking to match hive behavior, then added tests to insure this
8528027 [Pedro Rodriguez] added more tests
686e029 [Pedro Rodriguez] fix scala style
d262e9d [Pedro Rodriguez] reworked type checking code and added more tests
2517a58 [Pedro Rodriguez] removed unused import
28b4f71 [Pedro Rodriguez] fixed bug with type conversions and re-added tests
12f8795 [Pedro Rodriguez] fix scala style checks
e8a20a9 [Pedro Rodriguez] added python df (broken atm)
65b562c [Pedro Rodriguez] made array_contains nullable false
33b45aa [Pedro Rodriguez] reordered test
9623c64 [Pedro Rodriguez] fixed test
4b4425b [Pedro Rodriguez] changed Arrays in tests to Seqs
72cb4b1 [Pedro Rodriguez] added checkInputTypes and docs
69c46fb [Pedro Rodriguez] added tests and codegen
9e0bfc4 [Pedro Rodriguez] initial attempt at implementation
2015-08-04 22:34:02 -07:00
Yijie Shen a7fe48f687 [SPARK-9432][SQL] Audit expression unit tests to make sure we pass the proper numeric ranges
JIRA: https://issues.apache.org/jira/browse/SPARK-9432

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7933 from yjshen/numeric_ranges and squashes the following commits:

e719f78 [Yijie Shen] proper integral range check
2015-08-04 18:19:26 -07:00
Wenchen Fan 7c8fc1f7cb [SPARK-9598][SQL] do not expose generic getter in internal row
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7932 from cloud-fan/generic-getter and squashes the following commits:

c60de4c [Wenchen Fan] do not expose generic getter in internal row
2015-08-04 17:05:19 -07:00
Josh Rosen ab8ee1a3b9 [SPARK-9452] [SQL] Support records larger than page size in UnsafeExternalSorter
This patch extends UnsafeExternalSorter to support records larger than the page size. The basic strategy is the same as in #7762: store large records in their own overflow pages.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7891 from JoshRosen/large-records-in-sql-sorter and squashes the following commits:

967580b [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-records-in-sql-sorter
948c344 [Josh Rosen] Add large records tests for KV sorter.
3c17288 [Josh Rosen] Combine memory and disk cleanup into general cleanupResources() method
380f217 [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-records-in-sql-sorter
27eafa0 [Josh Rosen] Fix page size in PackedRecordPointerSuite
a49baef [Josh Rosen] Address initial round of review comments
3edb931 [Josh Rosen] Remove accidentally-committed debug statements.
2b164e2 [Josh Rosen] Support large records in UnsafeExternalSorter.
2015-08-04 14:42:11 -07:00
Wenchen Fan f4b1ac08a1 [SPARK-9553][SQL] remove the no-longer-necessary createCode and createStructCode, and replace the usage
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7890 from cloud-fan/minor and squashes the following commits:

c3b1be3 [Wenchen Fan] fix style
b0cbe2e [Wenchen Fan] remove the createCode and createStructCode, and replace the usage of them by createStructCode
2015-08-04 14:40:46 -07:00
Michael Armbrust 34a0eb2e89 [SPARK-9512][SQL] Revert SPARK-9251, Allow evaluation while sorting
The analysis rule has a bug and we ended up making the sorter still capable of doing evaluation, so lets revert this for now.

Author: Michael Armbrust <michael@databricks.com>

Closes #7906 from marmbrus/revertSortProjection and squashes the following commits:

2da6972 [Michael Armbrust] unrevert unrelated changes
4f2b00c [Michael Armbrust] Revert "[SPARK-9251][SQL] do not order by expressions which still need evaluation"
2015-08-04 10:07:53 -07:00
Yijie Shen b5034c9c59 [SPARK-9541] [SQL] DataTimeUtils cleanup
JIRA: https://issues.apache.org/jira/browse/SPARK-9541

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7870 from yjshen/datetime_cleanup and squashes the following commits:

9203e33 [Yijie Shen] revert getMonth & getDayOfMonth
5cad119 [Yijie Shen] rebase code
7d62a74 [Yijie Shen] remove tmp tuple inside split date
e98aaac [Yijie Shen] DataTimeUtils cleanup
2015-08-04 09:09:52 -07:00
Davies Liu 73dedb589d [SPARK-8246] [SQL] Implement get_json_object
This is based on #7485 , thanks to NathanHowell

Tests were copied from Hive, but do not seem to be super comprehensive. I've generally replicated Hive's unusual behavior rather than following a JSONPath reference, except for one case (as noted in the comments). I don't know if there is a way of fully replicating Hive's behavior without a slower TreeNode implementation, so I've erred on the side of performance instead.

Author: Davies Liu <davies@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Author: Nathan Howell <nhowell@godaddy.com>

Closes #7901 from davies/get_json_object and squashes the following commits:

3ace9b9 [Davies Liu] Merge branch 'get_json_object' of github.com:davies/spark into get_json_object
98766fc [Davies Liu] Merge branch 'master' of github.com:apache/spark into get_json_object
a7dc6d0 [Davies Liu] Update JsonExpressionsSuite.scala
c818519 [Yin Huai] new results.
18ce26b [Davies Liu] fix tests
6ac29fb [Yin Huai] Golden files.
25eebef [Davies Liu] use HiveQuerySuite
e0ac6ec [Yin Huai] Golden answer files.
940c060 [Davies Liu] tweat code style
44084c5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into get_json_object
9192d09 [Nathan Howell] Match Hive’s behavior for unwrapping arrays of one element
8dab647 [Nathan Howell] [SPARK-8246] [SQL] Implement get_json_object
2015-08-04 09:07:09 -07:00
Tarek Auel b1f88a38d5 [SPARK-8244] [SQL] string function: find in set
This PR is based on #7186 (just fix the conflict), thanks to tarekauel .

find_in_set(string str, string strList): int

Returns the first occurance of str in strList where strList is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument contains any commas. For example, find_in_set('ab', 'abc,b,ab,c,def') returns 3.

Only add this to SQL, not DataFrame.

Closes #7186

Author: Tarek Auel <tarek.auel@googlemail.com>
Author: Davies Liu <davies@databricks.com>

Closes #7900 from davies/find_in_set and squashes the following commits:

4334209 [Davies Liu] Merge branch 'master' of github.com:apache/spark into find_in_set
8f00572 [Davies Liu] Merge branch 'master' of github.com:apache/spark into find_in_set
243ede4 [Tarek Auel] [SPARK-8244][SQL] hive compatibility
1aaf64e [Tarek Auel] [SPARK-8244][SQL] unit test fix
e4093a4 [Tarek Auel] [SPARK-8244][SQL] final modifier for COMMA_UTF8
0d05df5 [Tarek Auel] Merge branch 'master' into SPARK-8244
208d710 [Tarek Auel] [SPARK-8244] address comments & bug fix
71b2e69 [Tarek Auel] [SPARK-8244] find_in_set
66c7fda [Tarek Auel] Merge branch 'master' into SPARK-8244
61b8ca2 [Tarek Auel] [SPARK-8224] removed loop and split; use unsafe String comparison
4f75a65 [Tarek Auel] Merge branch 'master' into SPARK-8244
e3b20c8 [Tarek Auel] [SPARK-8244] added type check
1c2bbb7 [Tarek Auel] [SPARK-8244] findInSet
2015-08-04 08:59:42 -07:00
Reynold Xin b2e4b85d2d Revert "[SPARK-9372] [SQL] Filter nulls in join keys"
This reverts commit 687c8c3715.
2015-08-03 14:51:15 -07:00
Andrew Or 702aa9d7fb [SPARK-8735] [SQL] Expose memory usage for shuffles, joins and aggregations
This patch exposes the memory used by internal data structures on the SparkUI. This tracks memory used by all spilling operations and SQL operators backed by Tungsten, e.g. `BroadcastHashJoin`, `ExternalSort`, `GeneratedAggregate` etc. The metric exposed is "peak execution memory", which broadly refers to the peak in-memory sizes of each of these data structure.

A separate patch will extend this by linking the new information to the SQL operators themselves.

<img width="950" alt="screen shot 2015-07-29 at 7 43 17 pm" src="https://cloud.githubusercontent.com/assets/2133137/8974776/b90fc980-362a-11e5-9e2b-842da75b1641.png">
<img width="802" alt="screen shot 2015-07-29 at 7 43 05 pm" src="https://cloud.githubusercontent.com/assets/2133137/8974777/baa76492-362a-11e5-9b77-e364a6a6b64e.png">

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7770)
<!-- Reviewable:end -->

Author: Andrew Or <andrew@databricks.com>

Closes #7770 from andrewor14/expose-memory-metrics and squashes the following commits:

9abecb9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
f5b0d68 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
d7df332 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
8eefbc5 [Andrew Or] Fix non-failing tests
9de2a12 [Andrew Or] Fix tests due to another logical merge conflict
876bfa4 [Andrew Or] Fix failing test after logical merge conflict
361a359 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
40b4802 [Andrew Or] Fix style?
d0fef87 [Andrew Or] Fix tests?
b3b92f6 [Andrew Or] Address comments
0625d73 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
c00a197 [Andrew Or] Fix potential NPEs
10da1cd [Andrew Or] Fix compile
17f4c2d [Andrew Or] Fix compile?
a87b4d0 [Andrew Or] Fix compile?
d70874d [Andrew Or] Fix test compile + address comments
2840b7d [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
6aa2f7a [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
b889a68 [Andrew Or] Minor changes: comments, spacing, style
663a303 [Andrew Or] UnsafeShuffleWriter: update peak memory before close
d090a94 [Andrew Or] Fix style
2480d84 [Andrew Or] Expand test coverage
5f1235b [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
1ecf678 [Andrew Or] Minor changes: comments, style, unused imports
0b6926c [Andrew Or] Oops
111a05e [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
a7a39a5 [Andrew Or] Strengthen presence check for accumulator
a919eb7 [Andrew Or] Add tests for unsafe shuffle writer
23c845d [Andrew Or] Add tests for SQL operators
a757550 [Andrew Or] Address comments
b5c51c1 [Andrew Or] Re-enable test in JavaAPISuite
5107691 [Andrew Or] Add tests for internal accumulators
59231e4 [Andrew Or] Fix tests
9528d09 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
5b5e6f3 [Andrew Or] Add peak execution memory to summary table + tooltip
92b4b6b [Andrew Or] Display peak execution memory on the UI
eee5437 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
d9b9015 [Andrew Or] Track execution memory in unsafe shuffles
770ee54 [Andrew Or] Track execution memory in broadcast joins
9c605a4 [Andrew Or] Track execution memory in GeneratedAggregate
9e824f2 [Andrew Or] Add back execution memory tracking for *ExternalSort
4ef4cb1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
e6c3e2f [Andrew Or] Move internal accumulators creation to Stage
a417592 [Andrew Or] Expose memory metrics in UnsafeExternalSorter
3c4f042 [Andrew Or] Track memory usage in ExternalAppendOnlyMap / ExternalSorter
bd7ab3f [Andrew Or] Add internal accumulators to TaskContext
2015-08-03 14:22:07 -07:00
Reynold Xin 7a9d09f0bb [SQL][minor] Simplify UnsafeRow.calculateBitSetWidthInBytes.
Author: Reynold Xin <rxin@databricks.com>

Closes #7897 from rxin/calculateBitSetWidthInBytes and squashes the following commits:

2e73b3a [Reynold Xin] [SQL][minor] Simplify UnsafeRow.calculateBitSetWidthInBytes.
2015-08-03 11:22:02 -07:00
Joseph Batchik dfe7bd168d [SPARK-9511] [SQL] Fixed Table Name Parsing
The issue was that the tokenizer was parsing "1one" into the numeric 1 using the code on line 110. I added another case to accept strings that start with a number and then have a letter somewhere else in it as well.

Author: Joseph Batchik <joseph.batchik@cloudera.com>

Closes #7844 from JDrit/parse_error and squashes the following commits:

b8ca12f [Joseph Batchik] fixed parsing issue by adding another case
2015-08-03 11:17:38 -07:00
Reynold Xin 8be198c869 Two minor comments from code review on 191bf2689. 2015-08-03 04:26:18 -07:00
Davies Liu 191bf2689d [SPARK-9518] [SQL] cleanup generated UnsafeRowJoiner and fix bug
Currently, when copy the bitsets, we didn't consider that the row1 may not sit in the beginning of byte array.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7892 from davies/clean_join and squashes the following commits:

14cce9e [Davies Liu] cleanup generated UnsafeRowJoiner and fix bug
2015-08-03 04:23:26 -07:00
Wenchen Fan 137f47865d [SPARK-9551][SQL] add a cheap version of copy for UnsafeRow to reuse a copy buffer
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7885 from cloud-fan/cheap-copy and squashes the following commits:

0900ca1 [Wenchen Fan] replace == with ===
73f4ada [Wenchen Fan] add tests
07b865a [Wenchen Fan] add a cheap version of copy
2015-08-03 04:21:15 -07:00
Yin Huai 1ebd41b141 [SPARK-9240] [SQL] Hybrid aggregate operator using unsafe row
This PR adds a base aggregation iterator `AggregationIterator`, which is used to create `SortBasedAggregationIterator` (for sort-based aggregation) and `UnsafeHybridAggregationIterator` (first it tries hash-based aggregation and falls back to the sort-based aggregation (using external sorter) if we cannot allocate memory for the map). With these two iterators, we will not need existing iterators and I am removing those. Also, we can use a single physical `Aggregate` operator and it internally determines what iterators to used.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #7813 from yhuai/AggregateOperator and squashes the following commits:

e317e2b [Yin Huai] Remove unnecessary change.
74d93c5 [Yin Huai] Merge remote-tracking branch 'upstream/master' into AggregateOperator
ba6afbc [Yin Huai] Add a little bit more comments.
c9cf3b6 [Yin Huai] update
0f1b06f [Yin Huai] Remove unnecessary code.
21fd15f [Yin Huai] Remove unnecessary change.
964f88b [Yin Huai] Implement fallback strategy.
b1ea5cf [Yin Huai] wip
7fcbd87 [Yin Huai] Add a flag to control what iterator to use.
533d5b2 [Yin Huai] Prepare for fallback!
33b7022 [Yin Huai] wip
bd9282b [Yin Huai] UDAFs now supports UnsafeRow.
f52ee53 [Yin Huai] wip
3171f44 [Yin Huai] wip
d2c45a0 [Yin Huai] wip
f60cc83 [Yin Huai] Also check input schema.
af32210 [Yin Huai] Check iter.hasNext before we create an iterator because the constructor of the iterato will read at least one row from a non-empty input iter.
299008c [Yin Huai] First round cleanup.
3915bac [Yin Huai] Create a base iterator class for aggregation iterators and add the initial version of the hybrid iterator.
2015-08-03 00:23:08 -07:00
Yijie Shen 98d6d9c7a9 [SPARK-9549][SQL] fix bugs in expressions
JIRA: https://issues.apache.org/jira/browse/SPARK-9549

This PR fix the following bugs:
1.  `UnaryMinus`'s codegen version would fail to compile when the input is `Long.MinValue`
2.  `BinaryComparison` would fail to compile in codegen mode when comparing Boolean types.
3.  `AddMonth` would fail if passed a huge negative month, which would lead accessing negative index of `monthDays` array.
4.  `Nanvl` with different type operands.

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7882 from yjshen/minor_bug_fix and squashes the following commits:

41bbd2c [Yijie Shen] fix bug in Nanvl type coercion
3dee204 [Yijie Shen] address comments
4fa5de0 [Yijie Shen] fix bugs in expressions
2015-08-03 00:15:24 -07:00
Wenchen Fan 608353c8e8 [SPARK-9404][SPARK-9542][SQL] unsafe array data and map data
This PR adds a UnsafeArrayData, current we encode it in this way:

first 4 bytes is the # elements
then each 4 byte is the start offset of the element, unless it is negative, in which case the element is null.
followed by the elements themselves

an example:  [10, 11, 12, 13, null, 14] will be encoded as:
5, 28, 32, 36, 40, -44, 44, 10, 11, 12, 13, 14

Note that, when we read a UnsafeArrayData from bytes, we can read the first 4 bytes as numElements and take the rest(first 4 bytes skipped) as value region.

unsafe map data just use 2 unsafe array data, first 4 bytes is # of elements, second 4 bytes is numBytes of key array, the follows key array data and value array data.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7752 from cloud-fan/unsafe-array and squashes the following commits:

3269bd7 [Wenchen Fan] fix a bug
6445289 [Wenchen Fan] add unit tests
49adf26 [Wenchen Fan] add unsafe map
20d1039 [Wenchen Fan] add comments and unsafe converter
821b8db [Wenchen Fan] add unsafe array
2015-08-02 23:41:16 -07:00
Yin Huai 687c8c3715 [SPARK-9372] [SQL] Filter nulls in join keys
This PR adds an optimization rule, `FilterNullsInJoinKey`, to add `Filter` before join operators to filter out rows having null values for join keys.

This optimization is guarded by a new SQL conf, `spark.sql.advancedOptimization`.

The code in this PR was authored by yhuai; I'm opening this PR to factor out this change from #7685, a larger pull request which contains two other optimizations.

Author: Yin Huai <yhuai@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #7768 from JoshRosen/filter-nulls-in-join-key and squashes the following commits:

c02fc3f [Yin Huai] Address Josh's comments.
0a8e096 [Yin Huai] Update comments.
ea7d5a6 [Yin Huai] Make sure we do not keep adding filters.
be88760 [Yin Huai] Make it clear that FilterNullsInJoinKeySuite.scala is used to test FilterNullsInJoinKey.
8bb39ad [Yin Huai] Fix non-deterministic tests.
303236b [Josh Rosen] Revert changes that are unrelated to null join key filtering
40eeece [Josh Rosen] Merge remote-tracking branch 'origin/master' into filter-nulls-in-join-key
c57a954 [Yin Huai] Bug fix.
d3d2e64 [Yin Huai] First round of cleanup.
f9516b0 [Yin Huai] Style
c6667e7 [Yin Huai] Add PartitioningCollection.
e616d3b [Yin Huai] wip
7c2d2d8 [Yin Huai] Bug fix and refactoring.
69bb072 [Yin Huai] Introduce NullSafeHashPartitioning and NullUnsafePartitioning.
d5b84c3 [Yin Huai] Do not add unnessary filters.
2201129 [Yin Huai] Filter out rows that will not be joined in equal joins early.
2015-08-02 23:32:09 -07:00
Yin Huai 114ff926fc [SPARK-2205] [SQL] Avoid unnecessary exchange operators in multi-way joins
This PR adds `PartitioningCollection`, which is used to represent the `outputPartitioning` for SparkPlans with multiple children (e.g. `ShuffledHashJoin`). So, a `SparkPlan` can have multiple descriptions of its partitioning schemes. Taking `ShuffledHashJoin` as an example, it has two descriptions of its partitioning schemes, i.e. `left.outputPartitioning` and `right.outputPartitioning`. So when we have a query like `select * from t1 join t2 on (t1.x = t2.x) join t3 on (t2.x = t3.x)` will only have three Exchange operators (when shuffled joins are needed) instead of four.

The code in this PR was authored by yhuai; I'm opening this PR to factor out this change from #7685, a larger pull request which contains two other optimizations.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7773)
<!-- Reviewable:end -->

Author: Yin Huai <yhuai@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #7773 from JoshRosen/multi-way-join-planning-improvements and squashes the following commits:

5c45924 [Josh Rosen] Merge remote-tracking branch 'origin/master' into multi-way-join-planning-improvements
cd8269b [Josh Rosen] Refactor test to use SQLTestUtils
2963857 [Yin Huai] Revert unnecessary SqlConf change.
73913f7 [Yin Huai] Add comments and test. Also, revert the change in ShuffledHashOuterJoin for now.
4a99204 [Josh Rosen] Delete unrelated expression change
884ab95 [Josh Rosen] Carve out only SPARK-2205 changes.
247e5fa [Josh Rosen] Merge remote-tracking branch 'origin/master' into multi-way-join-planning-improvements
c57a954 [Yin Huai] Bug fix.
d3d2e64 [Yin Huai] First round of cleanup.
f9516b0 [Yin Huai] Style
c6667e7 [Yin Huai] Add PartitioningCollection.
e616d3b [Yin Huai] wip
7c2d2d8 [Yin Huai] Bug fix and refactoring.
69bb072 [Yin Huai] Introduce NullSafeHashPartitioning and NullUnsafePartitioning.
d5b84c3 [Yin Huai] Do not add unnessary filters.
2201129 [Yin Huai] Filter out rows that will not be joined in equal joins early.
2015-08-02 20:44:23 -07:00
Reynold Xin 30e89111d6 [SPARK-9546][SQL] Centralize orderable data type checking.
This pull request creates two isOrderable functions in RowOrdering that can be used to check whether a data type or a sequence of expressions can be used in sorting.

Author: Reynold Xin <rxin@databricks.com>

Closes #7880 from rxin/SPARK-9546 and squashes the following commits:

f9e322d [Reynold Xin] Fixed tests.
0439b43 [Reynold Xin] [SPARK-9546][SQL] Centralize orderable data type checking.
2015-08-02 20:12:03 -07:00
Reynold Xin 9d03ad910b [SPARK-9543][SQL] Add randomized testing for UnsafeKVExternalSorter.
The detailed approach is documented in UnsafeKVExternalSorterSuite.testKVSorter(), working as follows:

1. Create input by generating data randomly based on the given key/value schema (which is also randomly drawn from a list of candidate types)
2. Run UnsafeKVExternalSorter on the generated data
3. Collect the output from the sorter, and make sure the keys are sorted in ascending order
4. Sort the input by both key and value, and sort the sorter output also by both key and value. Compare the sorted input and sorted output together to make sure all the key/values match.
5. Check memory allocation to make sure there is no memory leak.

There is also a spill flag. When set to true, the sorter will spill probabilistically roughly every 100 records.

Author: Reynold Xin <rxin@databricks.com>

Closes #7873 from rxin/kvsorter-randomized-test and squashes the following commits:

a08c251 [Reynold Xin] Resource cleanup.
0488b5c [Reynold Xin] [SPARK-9543][SQL] Add randomized testing for UnsafeKVExternalSorter.
2015-08-02 17:54:30 -07:00
Liang-Chi Hsieh 0722f43316 [SPARK-7937][SQL] Support comparison on StructType
This brings #6519 up-to-date with master branch.

Closes #6519.

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #7877 from rxin/sort-struct and squashes the following commits:

4968231 [Reynold Xin] Minor fixes.
2537813 [Reynold Xin] Merge branch 'compare_named_struct' of github.com:viirya/spark-1 into sort-struct
d2ba8ad [Liang-Chi Hsieh] Remove unused import.
3a3f40e [Liang-Chi Hsieh] Don't need to add compare to InternalRow because we can use RowOrdering.
dae6aad [Liang-Chi Hsieh] Fix nested struct.
d5349c7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
43d4354 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
1f66196 [Liang-Chi Hsieh] Reuse RowOrdering and GenerateOrdering.
f8b2e9c [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
1187a65 [Liang-Chi Hsieh] Fix scala style.
9d67f68 [Liang-Chi Hsieh] Fix wrongly merging.
8f4d775 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
94b27d5 [Liang-Chi Hsieh] Remove test for error on complex type comparison.
2071693 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
3c142e4 [Liang-Chi Hsieh] Fix scala style.
cf58dc3 [Liang-Chi Hsieh] Use checkAnswer.
f651b8d [Liang-Chi Hsieh] Remove Either and move orderings to BinaryComparison to reuse it.
b6e1009 [Liang-Chi Hsieh] Fix scala style.
3922b54 [Liang-Chi Hsieh] Support ordering on named_struct.
2015-08-02 17:53:44 -07:00
Reynold Xin 2e981b7bfa [SPARK-9531] [SQL] UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter
This pull request adds a destructAndCreateExternalSorter method to UnsafeFixedWidthAggregationMap. The new method does the following:

1. Creates a new external sorter UnsafeKVExternalSorter
2. Adds all the data into an in-memory sorter, sorts them
3. Spills the sorted in-memory data to disk

This method can be used to fallback to sort-based aggregation when under memory pressure.

The pull request also includes accounting fixes from JoshRosen.

TODOs (that can be done in follow-up PRs)
- [x] Address Josh's feedbacks from #7849
- [x] More documentation and test cases
- [x] Make sure we are doing memory accounting correctly with test cases (e.g. did we release the memory in BytesToBytesMap twice?)
- [ ] Look harder at possible memory leaks and exception handling
- [ ] Randomized tester for the KV sorter as well as the aggregation map

Author: Reynold Xin <rxin@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #7860 from rxin/kvsorter and squashes the following commits:

986a58c [Reynold Xin] Bug fix.
599317c [Reynold Xin] Style fix and slightly more compact code.
fe7bd4e [Reynold Xin] Bug fixes.
fd71bef [Reynold Xin] Merge remote-tracking branch 'josh/large-records-in-sql-sorter' into kvsorter-with-josh-fix
3efae38 [Reynold Xin] More fixes and documentation.
45f1b09 [Josh Rosen] Ensure that spill files are cleaned up
f6a9bd3 [Reynold Xin] Josh feedback.
9be8139 [Reynold Xin] Remove testSpillFrequency.
7cbe759 [Reynold Xin] [SPARK-9531][SQL] UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter.
ae4a8af [Josh Rosen] Detect leaked unsafe memory in UnsafeExternalSorterSuite.
52f9b06 [Josh Rosen] Detect ShuffleMemoryManager leaks in UnsafeExternalSorter.
2015-08-02 12:32:14 -07:00
Davies Liu 16b928c543 [SPARK-9529] [SQL] improve TungstenSort on DecimalType
Generate prefix for DecimalType, fix the random generator of decimal

cc JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #7857 from davies/sort_decimal and squashes the following commits:

2433959 [Davies Liu] Merge branch 'master' of github.com:apache/spark into sort_decimal
de24253 [Davies Liu] fix style
0a54c1a [Davies Liu] sort decimal
2015-08-01 23:36:06 -07:00
Davies Liu 57084e0c7c [SPARK-9459] [SQL] use generated FromUnsafeProjection to do deep copy for UTF8String and struct
When accessing a column in UnsafeRow, it's good to avoid the copy, then we should do deep copy when turn the UnsafeRow into generic Row, this PR brings generated FromUnsafeProjection to do that.

This PR also fix the expressions that cache the UTF8String, which should also copy it.

Author: Davies Liu <davies@databricks.com>

Closes #7840 from davies/avoid_copy and squashes the following commits:

230c8a1 [Davies Liu] address comment
fd797c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into avoid_copy
e095dd0 [Davies Liu] rollback rename
8ef5b0b [Davies Liu] copy String in Columnar
81360b8 [Davies Liu] fix class name
9aecb88 [Davies Liu] use FromUnsafeProjection to do deep copy for UTF8String and struct
2015-08-01 21:50:42 -07:00
Davies Liu c1b0cbd762 [SPARK-8185] [SPARK-8188] [SPARK-8191] [SQL] function datediff, to_utc_timestamp, from_utc_timestamp
This PR is based on #7643 , thanks to adrian-wang

Author: Davies Liu <davies@databricks.com>
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #7847 from davies/datediff and squashes the following commits:

74333d7 [Davies Liu] fix bug
22d8a8c [Davies Liu] optimize
85cdd21 [Davies Liu] remove unnecessary tests
241d90c [Davies Liu] Merge branch 'master' of github.com:apache/spark into datediff
e9dc0f5 [Davies Liu] fix datediff/to_utc_timestamp/from_utc_timestamp
c360447 [Daoyuan Wang] function datediff, to_utc_timestamp, from_utc_timestamp (commits merged)
2015-08-01 21:46:46 -07:00
HuJiayin 00cd92f32f [SPARK-8269] [SQL] string function: initcap
This PR is based on #7208 , thanks to HuJiayin

Closes #7208

Author: HuJiayin <jiayin.hu@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7850 from davies/initcap and squashes the following commits:

54472e9 [Davies Liu] fix python test
17ffe51 [Davies Liu] Merge branch 'master' of github.com:apache/spark into initcap
ca46390 [Davies Liu] Merge branch 'master' of github.com:apache/spark into initcap
3a906e4 [Davies Liu] implement title case in UTF8String
8b2506a [HuJiayin] Update functions.py
2cd43e5 [HuJiayin] fix python style check
b616c0e [HuJiayin] add python api
1f5a0ef [HuJiayin] add codegen
7e0c604 [HuJiayin] Merge branch 'master' of https://github.com/apache/spark into initcap
6a0b958 [HuJiayin] add column
c79482d [HuJiayin] support soundex
7ce416b [HuJiayin] support initcap rebase code
2015-08-01 21:44:57 -07:00
Davies Liu 5d9e33d9a2 [SPARK-9495] prefix of DateType/TimestampType
cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7856 from davies/sort_improve and squashes the following commits:

5fc81bd [Davies Liu] support DateType/TimestampType
2015-08-01 18:22:46 -07:00
Reynold Xin 3d1535d488 [SPARK-9520] [SQL] Support in-place sort in UnsafeFixedWidthAggregationMap
This pull request adds a sortedIterator method to UnsafeFixedWidthAggregationMap that sorts its data in-place by the grouping key.

This is needed so we can fallback to external sorting for aggregation.

Author: Reynold Xin <rxin@databricks.com>

Closes #7849 from rxin/bytes2bytes-sorting and squashes the following commits:

75018c6 [Reynold Xin] Updated documentation.
81a8694 [Reynold Xin] [SPARK-9520][SQL] Support in-place sort in UnsafeFixedWidthAggregationMap.
2015-08-01 13:20:26 -07:00
zhichao.li c5166f7a69 [SPARK-8263] [SQL] substr/substring should also support binary type
This is based on #7641, thanks to zhichao-li

Closes #7641

Author: zhichao.li <zhichao.li@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7848 from davies/substr and squashes the following commits:

461b709 [Davies Liu] remove bytearry from tests
b45377a [Davies Liu] Merge branch 'master' of github.com:apache/spark into substr
01d795e [zhichao.li] scala style
99aa130 [zhichao.li] add substring to dataframe
4f68bfe [zhichao.li] add binary type support for substring
2015-08-01 08:48:46 -07:00
Cheng Hao cf6c9ca32a [SPARK-8232] [SQL] Add sort_array support
This PR is based on #7581 , just fix the conflict.

Author: Cheng Hao <hao.cheng@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7851 from davies/sort_array and squashes the following commits:

a80ef66 [Davies Liu] fix conflict
7cfda65 [Davies Liu] Merge branch 'master' of github.com:apache/spark into sort_array
664c960 [Cheng Hao] update the sort_array by using the ArrayData
276d2d5 [Cheng Hao] add empty line
0edab9c [Cheng Hao] Add asending/descending support for sort_array
80fc0f8 [Cheng Hao] Add type checking
a42b678 [Cheng Hao] Add sort_array support
2015-08-01 08:32:29 -07:00
Davies Liu 60ea7ab4bb Revert "[SPARK-8232] [SQL] Add sort_array support"
This reverts commit 67ad4e21fc.
2015-08-01 00:41:15 -07:00
Wenchen Fan 1d59a4162b [SPARK-9480][SQL] add MapData and cleanup internal row stuff
This PR adds a `MapData` as internal representation of map type in Spark SQL, and provides a default implementation with just 2 `ArrayData`.

After that, we have specialized getters for all internal type, so I removed generic getter in `ArrayData` and added specialized `toArray` for it.
Also did some refactor and cleanup for `InternalRow` and its subclasses.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7799 from cloud-fan/map-data and squashes the following commits:

77d482f [Wenchen Fan] fix python
e8f6682 [Wenchen Fan] skip MapData equality check in HiveInspectorSuite
40cc9db [Wenchen Fan] add toString
6e06ec9 [Wenchen Fan] some more cleanup
a90aca1 [Wenchen Fan] add MapData
2015-08-01 00:17:15 -07:00
Reynold Xin d90f2cf7a2 [SPARK-9517][SQL] BytesToBytesMap should encode data the same way as UnsafeExternalSorter
BytesToBytesMap current encodes key/value data in the following format:
```
8B key length, key data, 8B value length, value data
```

UnsafeExternalSorter, on the other hand, encodes data this way:
```
4B record length, data
```

As a result, we cannot pass records encoded by BytesToBytesMap directly into UnsafeExternalSorter for sorting. However, if we rearrange data slightly, we can then pass the key/value records directly into UnsafeExternalSorter:
```
4B key+value length, 4B key length, key data, value data
```

Author: Reynold Xin <rxin@databricks.com>

Closes #7845 from rxin/kvsort-rebase and squashes the following commits:

5716b59 [Reynold Xin] Fixed test.
2e62ccb [Reynold Xin] Updated BytesToBytesMap's data encoding to put the key first.
a51b641 [Reynold Xin] Added a KV sorter interface.
2015-07-31 23:55:16 -07:00