Commit graph

1875 commits

Author SHA1 Message Date
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
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
Hossein 14844118b5 [SPARK-13261][SQL] Expose maxCharactersPerColumn as a user configurable option
This patch expose `maxCharactersPerColumn` and `maxColumns` to user in CSV data source.

Author: Hossein <hossein@databricks.com>

Closes #11147 from falaki/SPARK-13261.
2016-02-19 14:46:56 -08:00
Brandon Bradley dbb08cdd5a [SPARK-12966][SQL] ArrayType(DecimalType) support in Postgres JDBC
Fixes error `org.postgresql.util.PSQLException: Unable to find server array type for provided name decimal(38,18)`.

* Passes scale metadata to JDBC dialect for usage in type conversions.
* Removes unused length/scale/precision parameters from `createArrayOf` parameter `typeName` (for writing).
* Adds configurable precision and scale to Postgres `DecimalType` (for reading).
* Adds a new kind of test that verifies the schema written by `DataFrame.write.jdbc`.

Author: Brandon Bradley <bradleytastic@gmail.com>

Closes #10928 from blbradley/spark-12966.
2016-02-19 14:43:21 -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 95e1ab223e [SPARK-13237] [SQL] generated broadcast outer join
This PR support codegen for broadcast outer join.

In order to reduce the duplicated codes, this PR merge HashJoin and HashOuterJoin together (also BroadcastHashJoin and BroadcastHashOuterJoin).

Author: Davies Liu <davies@databricks.com>

Closes #11130 from davies/gen_out.
2016-02-18 15:15:06 -08:00
jerryshao 1eac380008 [SPARK-13109][BUILD] Fix SBT publishLocal issue
Add local ivy repo to the SBT build file to fix this.

Scaladoc compile error is fixed.

Author: jerryshao <sshao@hortonworks.com>

Closes #11001 from jerryshao/SPARK-13109.
2016-02-17 15:05:40 -08:00
Takuya UESHIN 04e8afe362 [SPARK-13357][SQL] Use generated projection and ordering for TakeOrderedAndProjectNode
`TakeOrderedAndProjectNode` should use generated projection and ordering like other `LocalNode`s.

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

Closes #11230 from ueshin/issues/SPARK-13357.
2016-02-17 00:21:15 -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
Reynold Xin 354d4c24be [SPARK-13296][SQL] Move UserDefinedFunction into sql.expressions.
This pull request has the following changes:

1. Moved UserDefinedFunction into expressions package. This is more consistent with how we structure the packages for window functions and UDAFs.

2. Moved UserDefinedPythonFunction into execution.python package, so we don't have a random private class in the top level sql package.

3. Move everything in execution/python.scala into the newly created execution.python package.

Most of the diffs are just straight copy-paste.

Author: Reynold Xin <rxin@databricks.com>

Closes #11181 from rxin/SPARK-13296.
2016-02-13 21:06:31 -08:00
Davies Liu 2228f074e1 [SPARK-13293][SQL] generate Expand
Expand suffer from create the UnsafeRow from same input multiple times, with codegen, it only need to copy some of the columns.

After this, we can see 3X improvements (from 43 seconds to 13 seconds) on a TPCDS query (Q67) that have eight columns in Rollup.

Ideally, we could mask some of the columns based on bitmask, I'd leave that in the future, because currently Aggregation (50 ns) is much slower than that just copy the variables (1-2 ns).

Author: Davies Liu <davies@databricks.com>

Closes #11177 from davies/gen_expand.
2016-02-12 17:32:15 -08:00
hyukjinkwon ac7d6af1ca [SPARK-13260][SQL] count(*) does not work with CSV data source
https://issues.apache.org/jira/browse/SPARK-13260
This is a quicky fix for `count(*)`.

When the `requiredColumns` is empty, currently it returns `sqlContext.sparkContext.emptyRDD[Row]` which does not have the count.

Just like JSON datasource, this PR lets the CSV datasource count the rows but do not parse each set of tokens.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11169 from HyukjinKwon/SPARK-13260.
2016-02-12 11:54:58 -08:00
Davies Liu b10af5e238 [SPARK-12915][SQL] add SQL metrics of numOutputRows for whole stage codegen
This PR add SQL metrics (numOutputRows) for generated operators (same as non-generated), the cost is about 0.2 nano seconds per row.

<img width="806" alt="gen metrics" src="https://cloud.githubusercontent.com/assets/40902/12994694/47f5881e-d0d7-11e5-9d47-78229f559ab0.png">

Author: Davies Liu <davies@databricks.com>

Closes #11170 from davies/gen_metric.
2016-02-11 18:00:03 -08:00
jayadevanmurali 0d50a22084 [SPARK-12982][SQL] Add table name validation in temp table registration
Add the table name validation at the temp table creation

Author: jayadevanmurali <jayadevan.m@tcs.com>

Closes #11051 from jayadevanmurali/branch-0.2-SPARK-12982.
2016-02-11 21:21:03 +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
Davies Liu 8f744fe3d9 [SPARK-13234] [SQL] remove duplicated SQL metrics
For lots of SQL operators, we have metrics for both of input and output, the number of input rows should be exactly the number of output rows of child, we could only have metrics for output rows.

After we improved the performance using whole stage codegen, the overhead of SQL metrics are not trivial anymore, we should avoid that if it's not necessary.

This PR remove all the SQL metrics for number of input rows, add SQL metric of number of output rows for all LeafNode. All remove the SQL metrics from those operators that have the same number of rows from input and output (for example, Projection, we may don't need that).

The new SQL UI will looks like:

![metrics](https://cloud.githubusercontent.com/assets/40902/12965227/63614e5e-d009-11e5-88b3-84fea04f9c20.png)

Author: Davies Liu <davies@databricks.com>

Closes #11163 from davies/remove_metrics.
2016-02-10 23:23:01 -08: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 0f09f02269 [SPARK-13205][SQL] SQL Generation Support for Self Join
This PR addresses two issues:
  - Self join does not work in SQL Generation
  - When creating new instances for `LogicalRelation`, `metastoreTableIdentifier` is lost.

liancheng Could you please review the code changes? Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11084 from gatorsmile/selfJoinInSQLGen.
2016-02-11 11:08:21 +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
raela 719973b05e [SPARK-13274] Fix Aggregator Links on GroupedDataset Scala API
Update Aggregator links to point to #org.apache.spark.sql.expressions.Aggregator

Author: raela <raela@databricks.com>

Closes #11158 from raelawang/master.
2016-02-10 17:00:54 -08:00
Tathagata Das 0902e20288 [SPARK-13146][SQL] Management API for continuous queries
### Management API for Continuous Queries

**API for getting status of each query**
- Whether active or not
- Unique name of each query
- Status of the sources and sinks
- Exceptions

**API for managing each query**
- Immediately stop an active query
- Waiting for a query to be terminated, correctly or with error

**API for managing multiple queries**
- Listing all active queries
- Getting an active query by name
- Waiting for any one of the active queries to be terminated

**API for listening to query life cycle events**
- ContinuousQueryListener API for query start, progress and termination events.

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

Closes #11030 from tdas/streaming-df-management-api.
2016-02-10 16:45:06 -08:00
Takeshi YAMAMURO 5947fa8fa1 [SPARK-13057][SQL] Add benchmark codes and the performance results for implemented compression schemes for InMemoryRelation
This pr adds benchmark codes for in-memory cache compression to make future developments and discussions more smooth.

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

Closes #10965 from maropu/ImproveColumnarCache.
2016-02-10 13:34:02 -08:00
Josh Rosen ce3bdaeeff [HOTFIX] Fix Scala 2.10 build break in TakeOrderedAndProjectSuite. 2016-02-10 12:44:40 -08:00
Josh Rosen 5cf20598ce [SPARK-13254][SQL] Fix planning of TakeOrderedAndProject operator
The patch for SPARK-8964 ("use Exchange to perform shuffle in Limit" / #7334) inadvertently broke the planning of the TakeOrderedAndProject operator: because ReturnAnswer was the new root of the query plan, the TakeOrderedAndProject rule was unable to match before BasicOperators.

This patch fixes this by moving the `TakeOrderedAndCollect` and `CollectLimit` rules into the same strategy.

In addition, I made changes to the TakeOrderedAndProject operator in order to make its `doExecute()` method lazy and added a new TakeOrderedAndProjectSuite which tests the new code path.

/cc davies and marmbrus for review.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11145 from JoshRosen/take-ordered-and-project-fix.
2016-02-10 11:00:38 -08:00
Shixiong Zhu b385ce3882 [SPARK-13149][SQL] Add FileStreamSource
`FileStreamSource` is an implementation of `org.apache.spark.sql.execution.streaming.Source`. It takes advantage of the existing `HadoopFsRelationProvider` to support various file formats. It remembers files in each batch and stores it into the metadata files so as to recover them when restarting. The metadata files are stored in the file system. There will be a further PR to clean up the metadata files periodically.

This is based on the initial work from marmbrus.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11034 from zsxwing/stream-df-file-source.
2016-02-09 18:50:06 -08:00
Takeshi YAMAMURO 6f710f9fd4 [SPARK-12476][SQL] Implement JdbcRelation#unhandledFilters for removing unnecessary Spark Filter
Input: SELECT * FROM jdbcTable WHERE col0 = 'xxx'

Current plan:
```
== Optimized Logical Plan ==
Project [col0#0,col1#1]
+- Filter (col0#0 = xxx)
   +- Relation[col0#0,col1#1] JDBCRelation(jdbc:postgresql:postgres,testRel,[Lorg.apache.spark.Partition;2ac7c683,{user=maropu, password=, driver=org.postgresql.Driver})

== Physical Plan ==
+- Filter (col0#0 = xxx)
   +- Scan JDBCRelation(jdbc:postgresql:postgres,testRel,[Lorg.apache.spark.Partition;2ac7c683,{user=maropu, password=, driver=org.postgresql.Driver})[col0#0,col1#1] PushedFilters: [EqualTo(col0,xxx)]
```

This patch enables a plan below;
```
== Optimized Logical Plan ==
Project [col0#0,col1#1]
+- Filter (col0#0 = xxx)
   +- Relation[col0#0,col1#1] JDBCRelation(jdbc:postgresql:postgres,testRel,[Lorg.apache.spark.Partition;2ac7c683,{user=maropu, password=, driver=org.postgresql.Driver})

== Physical Plan ==
Scan JDBCRelation(jdbc:postgresql:postgres,testRel,[Lorg.apache.spark.Partition;2ac7c683,{user=maropu, password=, driver=org.postgresql.Driver})[col0#0,col1#1] PushedFilters: [EqualTo(col0,xxx)]
```

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

Closes #10427 from maropu/RemoveFilterInJdbcScan.
2016-02-10 09:45:13 +08: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
Nong Li 3708d13f1a [SPARK-12992] [SQL] Support vectorized decoding in UnsafeRowParquetRecordReader.
WIP: running tests. Code needs a bit of clean up.

This patch completes the vectorized decoding with the goal of passing the existing
tests. There is still more patches to support the rest of the format spec, even
just for flat schemas.

This patch adds a new flag to enable the vectorized decoding. Tests were updated
to try with both modes where applicable.

Once this is working well, we can remove the previous code path.

Author: Nong Li <nong@databricks.com>

Closes #11055 from nongli/spark-12992-2.
2016-02-08 22:21:26 -08:00
Davies Liu ff0af0ddfa [SPARK-13095] [SQL] improve performance for broadcast join with dimension table
This PR improve the performance for Broadcast join with dimension tables, which is common in data warehouse.

If the join key can fit in a long, we will use a special api `get(Long)` to get the rows from HashedRelation.

If the HashedRelation only have unique keys, we will use a special api `getValue(Long)` or `getValue(InternalRow)`.

If the keys can fit within a long, also the keys are dense, we will use a array of UnsafeRow, instead a hash map.

TODO: will do cleanup

Author: Davies Liu <davies@databricks.com>

Closes #11065 from davies/gen_dim.
2016-02-08 14:09:14 -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
Tommy YU 81da3bee66 [SPARK-5865][API DOC] Add doc warnings for methods that return local data structures
rxin srowen
I work out note message for rdd.take function, please help to review.

If it's fine, I can apply to all other function later.

Author: Tommy YU <tummyyu@163.com>

Closes #10874 from Wenpei/spark-5865-add-warning-for-localdatastructure.
2016-02-06 17:29:09 +00: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
Davies Liu 875f507929 [SPARK-13215] [SQL] remove fallback in codegen
Since we remove the configuration for codegen, we are heavily reply on codegen (also TungstenAggregate require the generated MutableProjection to update UnsafeRow), should remove the fallback, which could make user confusing, see the discussion in SPARK-13116.

Author: Davies Liu <davies@databricks.com>

Closes #11097 from davies/remove_fallback.
2016-02-05 15:07:43 -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
Shixiong Zhu 7b73f1719c [SPARK-13166][SQL] Rename DataStreamReaderWriterSuite to DataFrameReaderWriterSuite
A follow up PR for #11062 because it didn't rename the test suite.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11096 from zsxwing/rename.
2016-02-05 13:44:34 -08:00
Reynold Xin 82d84ff2dd [SPARK-13187][SQL] Add boolean/long/double options in DataFrameReader/Writer
This patch adds option function for boolean, long, and double types. This makes it slightly easier for Spark users to specify options without turning them into strings. Using the JSON data source as an example.

Before this patch:
```scala
sqlContext.read.option("primitivesAsString", "true").json("/path/to/json")
```

After this patch:
Before this patch:
```scala
sqlContext.read.option("primitivesAsString", true).json("/path/to/json")
```

Author: Reynold Xin <rxin@databricks.com>

Closes #11072 from rxin/SPARK-13187.
2016-02-04 22:43:44 -08:00
Jakob Odersky 352102ed0b [SPARK-13208][CORE] Replace use of Pairs with Tuple2s
Another trivial deprecation fix for Scala 2.11

Author: Jakob Odersky <jakob@odersky.com>

Closes #11089 from jodersky/SPARK-13208.
2016-02-04 22:22:41 -08:00
gatorsmile e3c75c6398 [SPARK-12850][SQL] Support Bucket Pruning (Predicate Pushdown for Bucketed Tables)
JIRA: https://issues.apache.org/jira/browse/SPARK-12850

This PR is to support bucket pruning when the predicates are `EqualTo`, `EqualNullSafe`, `IsNull`, `In`, and `InSet`.

Like HIVE, in this PR, the bucket pruning works when the bucketing key has one and only one column.

So far, I do not find a way to verify how many buckets are actually scanned. However, I did verify it when doing the debug. Could you provide a suggestion how to do it properly? Thank you! cloud-fan yhuai rxin marmbrus

BTW, we can add more cases to support complex predicate including `Or` and `And`. Please let me know if I should do it in this PR.

Maybe we also need to add test cases to verify if bucket pruning works well for each data type.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10942 from gatorsmile/pruningBuckets.
2016-02-04 18:37:58 -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
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
Holden Karau a8e2ba776b [SPARK-13152][CORE] Fix task metrics deprecation warning
Make an internal non-deprecated version of incBytesRead and incRecordsRead so we don't have unecessary deprecation warnings in our build.

Right now incBytesRead and incRecordsRead are marked as deprecated and for internal use only. We should make private[spark] versions which are not deprecated and switch to those internally so as to not clutter up the warning messages when building.

cc andrewor14 who did the initial deprecation

Author: Holden Karau <holden@us.ibm.com>

Closes #11056 from holdenk/SPARK-13152-fix-task-metrics-deprecation-warnings.
2016-02-03 17:43:14 -08:00
Davies Liu de0914522f [SPARK-13131] [SQL] Use best and average time in benchmark
Best time is stabler than average time, also added a column for nano seconds per row (which could be used to estimate contributions of each components in a query).

Having best time and average time together for more information (we can see kind of variance).

rate, time per row and relative are all calculated using best time.

The result looks like this:
```
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
rang/filter/sum:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
rang/filter/sum codegen=false          14332 / 16646         36.0          27.8       1.0X
rang/filter/sum codegen=true              845 /  940        620.0           1.6      17.0X
```

Author: Davies Liu <davies@databricks.com>

Closes #11018 from davies/gen_bench.
2016-02-03 17:07:27 -08:00
Reynold Xin 915a75398e [SPARK-13166][SQL] Remove DataStreamReader/Writer
They seem redundant and we can simply use DataFrameReader/Writer. The new usage looks like:

```scala
val df = sqlContext.read.stream("...")
val handle = df.write.stream("...")
handle.stop()
```

Author: Reynold Xin <rxin@databricks.com>

Closes #11062 from rxin/SPARK-13166.
2016-02-03 16:10:11 -08:00
Davies Liu c4feec26eb [SPARK-12798] [SQL] generated BroadcastHashJoin
A row from stream side could match multiple rows on build side, the loop for these matched rows should not be interrupted when emitting a row, so we buffer the output rows in a linked list, check the termination condition on producer loop (for example, Range or Aggregate).

Author: Davies Liu <davies@databricks.com>

Closes #10989 from davies/gen_join.
2016-02-03 10:38:53 -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
Davies Liu 99a6e3c1e8 [SPARK-12951] [SQL] support spilling in generated aggregate
This PR add spilling support for generated TungstenAggregate.

If spilling happened, it's not that bad to do the iterator based sort-merge-aggregate (not generated).

The changes will be covered by TungstenAggregationQueryWithControlledFallbackSuite

Author: Davies Liu <davies@databricks.com>

Closes #10998 from davies/gen_spilling.
2016-02-02 19:47:44 -08:00
Nong Li 21112e8a14 [SPARK-12992] [SQL] Update parquet reader to support more types when decoding to ColumnarBatch.
This patch implements support for more types when doing the vectorized decode. There are
a few more types remaining but they should be very straightforward after this. This code
has a few copy and paste pieces but they are difficult to eliminate due to performance
considerations.

Specifically, this patch adds support for:
  - String, Long, Byte types
  - Dictionary encoding for those types.

Author: Nong Li <nong@databricks.com>

Closes #10908 from nongli/spark-12992.
2016-02-02 16:33: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
Reynold Xin 7f6e3ec79b [SPARK-13138][SQL] Add "logical" package prefix for ddl.scala
ddl.scala is defined in the execution package, and yet its reference of "UnaryNode" and "Command" are logical. This was fairly confusing when I was trying to understand the ddl code.

Author: Reynold Xin <rxin@databricks.com>

Closes #11021 from rxin/SPARK-13138.
2016-02-02 11:29:20 -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
hyukjinkwon b93830126c [SPARK-13114][SQL] Add a test for tokens more than the fields in schema
https://issues.apache.org/jira/browse/SPARK-13114

This PR adds a test for tokens more than the fields in schema.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11020 from HyukjinKwon/SPARK-13114.
2016-02-02 10:41:06 -08:00
Michael Armbrust 29d92181d0 [SPARK-13094][SQL] Add encoders for seq/array of primitives
Author: Michael Armbrust <michael@databricks.com>

Closes #11014 from marmbrus/seqEncoders.
2016-02-02 10:15:40 -08:00
Michael Armbrust 12a20c144f [SPARK-10820][SQL] Support for the continuous execution of structured queries
This is a follow up to 9aadcffabd that extends Spark SQL to allow users to _repeatedly_ optimize and execute structured queries.  A `ContinuousQuery` can be expressed using SQL, DataFrames or Datasets.  The purpose of this PR is only to add some initial infrastructure which will be extended in subsequent PRs.

## User-facing API

- `sqlContext.streamFrom` and `df.streamTo` return builder objects that are analogous to the `read/write` interfaces already available to executing queries in a batch-oriented fashion.
- `ContinuousQuery` provides an interface for interacting with a query that is currently executing in the background.

## Internal Interfaces
 - `StreamExecution` - executes streaming queries in micro-batches

The following are currently internal, but public APIs will be provided in a future release.
 - `Source` - an interface for providers of continually arriving data.  A source must have a notion of an `Offset` that monotonically tracks what data has arrived.  For fault tolerance, a source must be able to replay data given a start offset.
 - `Sink` - an interface that accepts the results of a continuously executing query.  Also responsible for tracking the offset that should be resumed from in the case of a failure.

## Testing
 - `MemoryStream` and `MemorySink` - simple implementations of source and sink that keep all data in memory and have methods for simulating durability failures
 - `StreamTest` - a framework for performing actions and checking invariants on a continuous query

Author: Michael Armbrust <michael@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Josh Rosen <rosenville@gmail.com>

Closes #11006 from marmbrus/structured-streaming.
2016-02-02 10:13:54 -08:00
Michael Armbrust 22ba21348b [SPARK-13087][SQL] Fix group by function for sort based aggregation
It is not valid to call `toAttribute` on a `NamedExpression` unless we know for sure that the child produced that `NamedExpression`.  The current code worked fine when the grouping expressions were simple, but when they were a derived value this blew up at execution time.

Author: Michael Armbrust <michael@databricks.com>

Closes #11013 from marmbrus/groupByFunction-master.
2016-02-02 16:48:59 +08:00
Reynold Xin 0fff5c6e63 [SPARK-13130][SQL] Make codegen variable names easier to read
1. Use lower case
2. Change long prefixes to something shorter (in this case I am changing only one: TungstenAggregate -> agg).

Author: Reynold Xin <rxin@databricks.com>

Closes #11017 from rxin/SPARK-13130.
2016-02-01 23:08:11 -08:00
Jacek Laskowski a2973fed30 Fix for [SPARK-12854][SQL] Implement complex types support in Columna…
…rBatch

Fixes build for Scala 2.11.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #10946 from jaceklaskowski/SPARK-12854-fix.
2016-02-01 13:57:48 -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
Herman van Hovell 5a8b978fab [SPARK-13049] Add First/last with ignore nulls to functions.scala
This PR adds the ability to specify the ```ignoreNulls``` option to the functions dsl, e.g:
```df.select($"id", last($"value", ignoreNulls = true).over(Window.partitionBy($"id").orderBy($"other"))```

This PR is some where between a bug fix (see the JIRA) and a new feature. I am not sure if we should backport to 1.6.

cc yhuai

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

Closes #10957 from hvanhovell/SPARK-13049.
2016-01-31 13:56:13 -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
Josh Rosen 289373b28c [SPARK-6363][BUILD] Make Scala 2.11 the default Scala version
This patch changes Spark's build to make Scala 2.11 the default Scala version. To be clear, this does not mean that Spark will stop supporting Scala 2.10: users will still be able to compile Spark for Scala 2.10 by following the instructions on the "Building Spark" page; however, it does mean that Scala 2.11 will be the default Scala version used by our CI builds (including pull request builds).

The Scala 2.11 compiler is faster than 2.10, so I think we'll be able to look forward to a slight speedup in our CI builds (it looks like it's about 2X faster for the Maven compile-only builds, for instance).

After this patch is merged, I'll update Jenkins to add new compile-only jobs to ensure that Scala 2.10 compilation doesn't break.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10608 from JoshRosen/SPARK-6363.
2016-01-30 00:20:28 -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
Andrew Or 12252d1da9 [SPARK-13071] Coalescing HadoopRDD overwrites existing input metrics
This issue is causing tests to fail consistently in master with Hadoop 2.6 / 2.7. This is because for Hadoop 2.5+ we overwrite existing values of `InputMetrics#bytesRead` in each call to `HadoopRDD#compute`. In the case of coalesce, e.g.
```
sc.textFile(..., 4).coalesce(2).count()
```
we will call `compute` multiple times in the same task, overwriting `bytesRead` values from previous calls to `compute`.

For a regression test, see `InputOutputMetricsSuite.input metrics for old hadoop with coalesce`. I did not add a new regression test because it's impossible without significant refactoring; there's a lot of existing duplicate code in this corner of Spark.

This was caused by #10835.

Author: Andrew Or <andrew@databricks.com>

Closes #10973 from andrewor14/fix-input-metrics-coalesce.
2016-01-29 18:03:08 -08:00
Reynold Xin 2cbc412821 [SPARK-13076][SQL] Rename ClientInterface -> HiveClient
And ClientWrapper -> HiveClientImpl.

I have some followup pull requests to introduce a new internal catalog, and I think this new naming reflects better the functionality of the two classes.

Author: Reynold Xin <rxin@databricks.com>

Closes #10981 from rxin/SPARK-13076.
2016-01-29 16:57:34 -08:00
Andrew Or e38b0baa38 [SPARK-13055] SQLHistoryListener throws ClassCastException
This is an existing issue uncovered recently by #10835. The reason for the exception was because the `SQLHistoryListener` gets all sorts of accumulators, not just the ones that represent SQL metrics. For example, the listener gets the `internal.metrics.shuffleRead.remoteBlocksFetched`, which is an Int, then it proceeds to cast the Int to a Long, which fails.

The fix is to mark accumulators representing SQL metrics using some internal metadata. Then we can identify which ones are SQL metrics and only process those in the `SQLHistoryListener`.

Author: Andrew Or <andrew@databricks.com>

Closes #10971 from andrewor14/fix-sql-history.
2016-01-29 13:45:03 -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
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
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
Brandon Bradley 3a40c0e575 [SPARK-12749][SQL] add json option to parse floating-point types as DecimalType
I tried to add this via `USE_BIG_DECIMAL_FOR_FLOATS` option from Jackson with no success.

Added test for non-complex types. Should I add a test for complex types?

Author: Brandon Bradley <bradleytastic@gmail.com>

Closes #10936 from blbradley/spark-12749.
2016-01-28 15:25:57 -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
Tejas Patil 676803963f [SPARK-12926][SQL] SQLContext to display warning message when non-sql configs are being set
Users unknowingly try to set core Spark configs in SQLContext but later realise that it didn't work. eg. sqlContext.sql("SET spark.shuffle.memoryFraction=0.4"). This PR adds a warning message when such operations are done.

Author: Tejas Patil <tejasp@fb.com>

Closes #10849 from tejasapatil/SPARK-12926.
2016-01-28 13:45:28 -08:00
Cheng Lian 415d0a859b [SPARK-12818][SQL] Specialized integral and string types for Count-min Sketch
This PR is a follow-up of #10911. It adds specialized update methods for `CountMinSketch` so that we can avoid doing internal/external row format conversion in `DataFrame.countMinSketch()`.

Author: Cheng Lian <lian@databricks.com>

Closes #10968 from liancheng/cms-specialized.
2016-01-28 12:26:03 -08:00
Nong Li 4a09123212 [SPARK-13045] [SQL] Remove ColumnVector.Struct in favor of ColumnarBatch.Row
These two classes became identical as the implementation progressed.

Author: Nong Li <nong@databricks.com>

Closes #10952 from nongli/spark-13045.
2016-01-27 15:35:31 -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
Wenchen Fan 680afabe78 [SPARK-12938][SQL] DataFrame API for Bloom filter
This PR integrates Bloom filter from spark-sketch into DataFrame. This version resorts to RDD.aggregate for building the filter. A more performant UDAF version can be built in future follow-up PRs.

This PR also add 2 specify `put` version(`putBinary` and `putLong`) into `BloomFilter`, which makes it easier to build a Bloom filter over a `DataFrame`.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10937 from cloud-fan/bloom-filter.
2016-01-27 13:29:09 -08:00
Andrew Or 87abcf7df9 [SPARK-12895][SPARK-12896] Migrate TaskMetrics to accumulators
The high level idea is that instead of having the executors send both accumulator updates and TaskMetrics, we should have them send only accumulator updates. This eliminates the need to maintain both code paths since one can be implemented in terms of the other. This effort is split into two parts:

**SPARK-12895: Implement TaskMetrics using accumulators.** TaskMetrics is basically just a bunch of accumulable fields. This patch makes TaskMetrics a syntactic wrapper around a collection of accumulators so we don't need to send TaskMetrics from the executors to the driver.

**SPARK-12896: Send only accumulator updates to the driver.** Now that TaskMetrics are expressed in terms of accumulators, we can capture all TaskMetrics values if we just send accumulator updates from the executors to the driver. This completes the parent issue SPARK-10620.

While an effort has been made to preserve as much of the public API as possible, there were a few known breaking DeveloperApi changes that would be very awkward to maintain. I will gather the full list shortly and post it here.

Note: This was once part of #10717. This patch is split out into its own patch from there to make it easier for others to review. Other smaller pieces of already been merged into master.

Author: Andrew Or <andrew@databricks.com>

Closes #10835 from andrewor14/task-metrics-use-accums.
2016-01-27 11:15:48 -08:00
Cheng Lian 58f5d8c1da [SPARK-12728][SQL] Integrates SQL generation with native view
This PR is a follow-up of PR #10541. It integrates the newly introduced SQL generation feature with native view to make native view canonical.

In this PR, a new SQL option `spark.sql.nativeView.canonical` is added.  When this option and `spark.sql.nativeView` are both `true`, Spark SQL tries to handle `CREATE VIEW` DDL statements using SQL query strings generated from view definition logical plans. If we failed to map the plan to SQL, we fallback to the original native view approach.

One important issue this PR fixes is that, now we can use CTE when defining a view.  Originally, when native view is turned on, we wrap the view definition text with an extra `SELECT`.  However, HiveQL parser doesn't allow CTE appearing as a subquery.  Namely, something like this is disallowed:

```sql
SELECT n
FROM (
  WITH w AS (SELECT 1 AS n)
  SELECT * FROM w
) v
```

This PR fixes this issue because the extra `SELECT` is no longer needed (also, CTE expressions are inlined as subqueries during analysis phase, thus there won't be CTE expressions in the generated SQL query string).

Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #10733 from liancheng/spark-12728.integrate-sql-gen-with-native-view.
2016-01-26 20:30:13 -08:00
Cheng Lian ce38a35b76 [SPARK-12935][SQL] DataFrame API for Count-Min Sketch
This PR integrates Count-Min Sketch from spark-sketch into DataFrame. This version resorts to `RDD.aggregate` for building the sketch. A more performant UDAF version can be built in future follow-up PRs.

Author: Cheng Lian <lian@databricks.com>

Closes #10911 from liancheng/cms-df-api.
2016-01-26 20:12:34 -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
Sean Owen 649e9d0f5b [SPARK-3369][CORE][STREAMING] Java mapPartitions Iterator->Iterable is inconsistent with Scala's Iterator->Iterator
Fix Java function API methods for flatMap and mapPartitions to require producing only an Iterator, not Iterable. Also fix DStream.flatMap to require a function producing TraversableOnce only, not Traversable.

CC rxin pwendell for API change; tdas since it also touches streaming.

Author: Sean Owen <sowen@cloudera.com>

Closes #10413 from srowen/SPARK-3369.
2016-01-26 11:55:28 +00:00
Reynold Xin d54cfed5a6 [SQL][MINOR] A few minor tweaks to CSV reader.
This pull request simply fixes a few minor coding style issues in csv, as I was reviewing the change post-hoc.

Author: Reynold Xin <rxin@databricks.com>

Closes #10919 from rxin/csv-minor.
2016-01-26 00:51:08 -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
gatorsmile 9348431da2 [SPARK-12975][SQL] Throwing Exception when Bucketing Columns are part of Partitioning Columns
When users are using `partitionBy` and `bucketBy` at the same time, some bucketing columns might be part of partitioning columns. For example,
```
        df.write
          .format(source)
          .partitionBy("i")
          .bucketBy(8, "i", "k")
          .saveAsTable("bucketed_table")
```
However, in the above case, adding column `i` into `bucketBy` is useless. It is just wasting extra CPU when reading or writing bucket tables. Thus, like Hive, we can issue an exception and let users do the change.

Also added a test case for checking if the information of `sortBy` and `bucketBy` columns are correctly saved in the metastore table.

Could you check if my understanding is correct? cloud-fan rxin marmbrus Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10891 from gatorsmile/commonKeysInPartitionByBucketBy.
2016-01-25 13:38:09 -08:00
Yin Huai 00026fa991 [SPARK-12901][SQL][HOT-FIX] Fix scala 2.11 compilation. 2016-01-25 12:59:11 -08:00
Davies Liu 7d877c3439 [SPARK-12902] [SQL] visualization for generated operators
This PR brings back visualization for generated operators, they looks like:

![sql](https://cloud.githubusercontent.com/assets/40902/12460920/0dc7956a-bf6b-11e5-9c3f-8389f452526e.png)

![stage](https://cloud.githubusercontent.com/assets/40902/12460923/11806ac4-bf6b-11e5-9c72-e84a62c5ea93.png)

Note: SQL metrics are not supported right now, because they are very slow, will be supported once we have batch mode.

Author: Davies Liu <davies@databricks.com>

Closes #10828 from davies/viz_codegen.
2016-01-25 12:44:20 -08:00
hyukjinkwon 3adebfc9a3 [SPARK-12901][SQL] Refactor options for JSON and CSV datasource (not case class and same format).
https://issues.apache.org/jira/browse/SPARK-12901
This PR refactors the options in JSON and CSV datasources.

In more details,

1. `JSONOptions` uses the same format as `CSVOptions`.
2. Not case classes.
3. `CSVRelation` that does not have to be serializable (it was `with Serializable` but I removed)

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #10895 from HyukjinKwon/SPARK-12901.
2016-01-25 00:57:56 -08:00
Cheng Lian 3327fd2817 [SPARK-12624][PYSPARK] Checks row length when converting Java arrays to Python rows
When actual row length doesn't conform to specified schema field length, we should give a better error message instead of throwing an unintuitive `ArrayOutOfBoundsException`.

Author: Cheng Lian <lian@databricks.com>

Closes #10886 from liancheng/spark-12624.
2016-01-24 19:40:34 -08:00