Commit graph

1643 commits

Author SHA1 Message Date
Reynold Xin d7d0cad0ad [SPARK-14855][SQL] Add "Exec" suffix to physical operators
## What changes were proposed in this pull request?
This patch adds "Exec" suffix to all physical operators. Before this patch, Spark's physical operators and logical operators are named the same (e.g. Project could be logical.Project or execution.Project), which caused small issues in code review and bigger issues in code refactoring.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #12617 from rxin/exec-node.
2016-04-22 17:43:56 -07:00
Tathagata Das c431a76d06 [SPARK-14832][SQL][STREAMING] Refactor DataSource to ensure schema is inferred only once when creating a file stream
## What changes were proposed in this pull request?

When creating a file stream using sqlContext.write.stream(), existing files are scanned twice for finding the schema
- Once, when creating a DataSource + StreamingRelation in the DataFrameReader.stream()
- Again, when creating streaming Source from the DataSource, in DataSource.createSource()

Instead, the schema should be generated only once, at the time of creating the dataframe, and when the streaming source is created, it should just reuse that schema

The solution proposed in this PR is to add a lazy field in DataSource that caches the schema. Then streaming Source created by the DataSource can just reuse the schema.

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

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

Closes #12591 from tdas/SPARK-14832.
2016-04-22 17:17:37 -07:00
Davies Liu c25b97fcce [SPARK-14582][SQL] increase parallelism for small tables
## What changes were proposed in this pull request?

This PR try to increase the parallelism for small table (a few of big files) to reduce the query time, by decrease the maxSplitBytes, the goal is to have at least one task per CPU in the cluster, if the total size of all files is bigger than openCostInBytes * 2 * nCPU.

For example, a small/medium table could be used as dimension table in huge query, this will be useful to reduce the time waiting for broadcast.

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #12344 from davies/more_partition.
2016-04-22 17:09:16 -07:00
Davies Liu 0419d63169 [SPARK-14791] [SQL] fix risk condition between broadcast and subquery
## What changes were proposed in this pull request?

SparkPlan.prepare() could be called in different threads (BroadcastExchange will call it in a thread pool), it only make sure that doPrepare() will only be called once, the second call to prepare() may return earlier before all the children had finished prepare(). Then some operator may call doProduce() before prepareSubqueries(), `null` will be used as the result of subquery, which is wrong. This cause TPCDS Q23B returns wrong answer sometimes.

This PR added synchronization for prepare(), make sure all the children had finished prepare() before return. Also call prepare() in produce() (similar to execute()).

Added checking for ScalarSubquery to make sure that the subquery has finished before using the result.

## How was this patch tested?

Manually tested with Q23B, no wrong answer anymore.

Author: Davies Liu <davies@databricks.com>

Closes #12600 from davies/fix_risk.
2016-04-22 12:29:53 -07:00
Davies Liu c417cec067 [SPARK-14763][SQL] fix subquery resolution
## What changes were proposed in this pull request?

Currently, a column could be resolved wrongly if there are columns from both outer table and subquery have the same name, we should only resolve the attributes that can't be resolved within subquery. They may have same exprId than other attributes in subquery, so we should create alias for them.

Also, the column in IN subquery could have same exprId, we should create alias for them.

## How was this patch tested?

Added regression tests. Manually tests TPCDS Q70 and Q95, work well after this patch.

Author: Davies Liu <davies@databricks.com>

Closes #12539 from davies/fix_subquery.
2016-04-22 20:55:41 +02:00
Reynold Xin aeb52bea56 [SPARK-14841][SQL] Move SQLBuilder into sql/core
## What changes were proposed in this pull request?
This patch moves SQLBuilder into sql/core so we can in the future move view generation also into sql/core.

## How was this patch tested?
Also moved unit tests.

Author: Reynold Xin <rxin@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #12602 from rxin/SPARK-14841.
2016-04-22 11:10:31 -07:00
Pete Robbins 5bed13a872 [SPARK-14848][SQL] Compare as Set in DatasetSuite - Java encoder
## What changes were proposed in this pull request?
Change test to compare sets rather than sequence

## How was this patch tested?
Full test runs on little endian and big endian platforms

Author: Pete Robbins <robbinspg@gmail.com>

Closes #12610 from robbinspg/DatasetSuiteFix.
2016-04-22 23:07:12 +08:00
Joan bf95b8da27 [SPARK-6429] Implement hashCode and equals together
## What changes were proposed in this pull request?

Implement some `hashCode` and `equals` together in order to enable the scalastyle.
This is a first batch, I will continue to implement them but I wanted to know your thoughts.

Author: Joan <joan@goyeau.com>

Closes #12157 from joan38/SPARK-6429-HashCode-Equals.
2016-04-22 12:24:12 +01:00
Reynold Xin 284b15d2fb [SPARK-14826][SQL] Remove HiveQueryExecution
## What changes were proposed in this pull request?
This patch removes HiveQueryExecution. As part of this, I consolidated all the describe commands into DescribeTableCommand.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #12588 from rxin/SPARK-14826.
2016-04-22 01:31:13 -07:00
Cheng Lian 145433f1aa [SPARK-14369] [SQL] Locality support for FileScanRDD
(This PR is a rebased version of PR #12153.)

## What changes were proposed in this pull request?

This PR adds preliminary locality support for `FileFormat` data sources by overriding `FileScanRDD.preferredLocations()`. The strategy can be divided into two parts:

1.  Block location lookup

    Unlike `HadoopRDD` or `NewHadoopRDD`, `FileScanRDD` doesn't have access to the underlying `InputFormat` or `InputSplit`, and thus can't rely on `InputSplit.getLocations()` to gather locality information. Instead, this PR queries block locations using `FileSystem.getBlockLocations()` after listing all `FileStatus`es in `HDFSFileCatalog` and convert all `FileStatus`es into `LocatedFileStatus`es.

    Note that although S3/S3A/S3N file systems don't provide valid locality information, their `getLocatedStatus()` implementations don't actually issue remote calls either. So there's no need to special case these file systems.

2.  Selecting preferred locations

    For each `FilePartition`, we pick up top 3 locations that containing the most data to be retrieved. This isn't necessarily the best algorithm out there. Further improvements may be brought up in follow-up PRs.

## How was this patch tested?

Tested by overriding default `FileSystem` implementation for `file:///` with a mocked one, which returns mocked block locations.

Author: Cheng Lian <lian@databricks.com>

Closes #12527 from liancheng/spark-14369-locality-rebased.
2016-04-21 21:48:09 -07:00
Sameer Agarwal b29bc3f515 [SPARK-14680] [SQL] Support all datatypes to use VectorizedHashmap in TungstenAggregate
## What changes were proposed in this pull request?

This PR adds support for all primitive datatypes, decimal types and stringtypes in the VectorizedHashmap during aggregation.

## How was this patch tested?

Existing tests for group-by aggregates should already test for all these datatypes. Additionally, manually inspected the generated code for all supported datatypes (details below).

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12440 from sameeragarwal/all-datatypes.
2016-04-21 21:31:01 -07:00
Sameer Agarwal f82aa82480 [SPARK-14774][SQL] Write unscaled values in ColumnVector.putDecimal
## What changes were proposed in this pull request?

We recently made `ColumnarBatch.row` mutable and added a new `ColumnVector.putDecimal` method to support putting `Decimal` values in the `ColumnarBatch`. This unfortunately introduced a bug wherein we were not updating the vector with the proper unscaled values.

## How was this patch tested?

This codepath is hit only when the vectorized aggregate hashmap is enabled. https://github.com/apache/spark/pull/12440 makes sure that a number of regression tests/benchmarks test this bugfix.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12541 from sameeragarwal/fix-bigdecimal.
2016-04-21 16:00:59 -07:00
Reynold Xin 1a95397bb6 [SPARK-14798][SQL] Move native command and script transformation parsing into SparkSqlAstBuilder
## What changes were proposed in this pull request?
This patch moves native command and script transformation into SparkSqlAstBuilder. This builds on #12561. See the last commit for diff.

## How was this patch tested?
Updated test cases to reflect this.

Author: Reynold Xin <rxin@databricks.com>

Closes #12564 from rxin/SPARK-14798.
2016-04-21 15:59:37 -07:00
Andrew Or a2e8d4fddd [SPARK-13643][SQL] Implement SparkSession
## What changes were proposed in this pull request?

After removing most of `HiveContext` in 8fc267ab33 we can now move existing functionality in `SQLContext` to `SparkSession`. As of this PR `SQLContext` becomes a simple wrapper that has a `SparkSession` and delegates all functionality to it.

## How was this patch tested?

Jenkins.

Author: Andrew Or <andrew@databricks.com>

Closes #12553 from andrewor14/implement-spark-session.
2016-04-21 14:18:18 -07:00
Liang-Chi Hsieh 4ac6e75cd6 [HOTFIX] Remove wrong DDL tests
## What changes were proposed in this pull request?

As we moved most parsing rules to `SparkSqlParser`, some tests expected to throw exception are not correct anymore.

## How was this patch tested?
`DDLCommandSuite`

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

Closes #12572 from viirya/hotfix-ddl.
2016-04-21 13:18:39 +02:00
Reynold Xin 77d847ddb2 [SPARK-14792][SQL] Move as many parsing rules as possible into SQL parser
## What changes were proposed in this pull request?
This patch moves as many parsing rules as possible into SQL parser. There are only three more left after this patch: (1) run native command, (2) analyze, and (3) script IO. These 3 will be dealt with in a follow-up PR.

## How was this patch tested?
No test change. This simply moves code around.

Author: Reynold Xin <rxin@databricks.com>

Closes #12556 from rxin/SPARK-14792.
2016-04-21 00:24:24 -07:00
Reynold Xin 334c293ec0 [SPARK-14769][SQL] Create built-in functionality for variable substitution
## What changes were proposed in this pull request?
In order to fully merge the Hive parser and the SQL parser, we'd need to support variable substitution in Spark. The implementation of the substitute algorithm is mostly copied from Hive, but I simplified the overall structure quite a bit and added more comprehensive test coverage.

Note that this pull request does not yet use this functionality anywhere.

## How was this patch tested?
Added VariableSubstitutionSuite for unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12538 from rxin/SPARK-14769.
2016-04-20 16:32:38 -07:00
Shixiong Zhu 7bc948557b [SPARK-14678][SQL] Add a file sink log to support versioning and compaction
## What changes were proposed in this pull request?

This PR adds a special log for FileStreamSink for two purposes:

- Versioning. A future Spark version should be able to read the metadata of an old FileStreamSink.
- Compaction. As reading from many small files is usually pretty slow, we should compact small metadata files into big files.

FileStreamSinkLog has a new log format instead of Java serialization format. It will write one log file for each batch. The first line of the log file is the version number, and there are multiple JSON lines following. Each JSON line is a JSON format of FileLog.

FileStreamSinkLog will compact log files every "spark.sql.sink.file.log.compactLen" batches into a big file. When doing a compact, it will read all history logs and merge them with the new batch. During the compaction, it will also delete the files that are deleted (marked by FileLog.action). When the reader uses allLogs to list all files, this method only returns the visible files (drops the deleted files).

## How was this patch tested?

FileStreamSinkLogSuite

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12435 from zsxwing/sink-log.
2016-04-20 13:33:04 -07:00
Tathagata Das cb8ea9e1f3 [SPARK-14741][SQL] Fixed error in reading json file stream inside a partitioned directory
## What changes were proposed in this pull request?

Consider the following directory structure
dir/col=X/some-files
If we create a text format streaming dataframe on `dir/col=X/`  then it should not consider as partitioning in columns. Even though the streaming dataframe does not do so, the generated batch dataframes pick up col as a partitioning columns, causing mismatch streaming source schema and generated df schema. This leads to runtime failure:
```
18:55:11.262 ERROR org.apache.spark.sql.execution.streaming.StreamExecution: Query query-0 terminated with error
java.lang.AssertionError: assertion failed: Invalid batch: c#2 != c#7,type#8
```
The reason is that the partition inferring code has no idea of a base path, above which it should not search of partitions. This PR makes sure that the batch DF is generated with the basePath set as the original path on which the file stream source is defined.

## How was this patch tested?

New unit test

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

Closes #12517 from tdas/SPARK-14741.
2016-04-20 12:22:51 -07:00
Wenchen Fan 85d759ca3a [SPARK-14704][CORE] create accumulators in TaskMetrics
## What changes were proposed in this pull request?

Before this PR, we create accumulators at driver side(and register them) and send them to executor side, then we create `TaskMetrics` with these accumulators at executor side.
After this PR, we will create `TaskMetrics` at driver side and send it to executor side, so that we can create accumulators inside `TaskMetrics` directly, which is cleaner.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12472 from cloud-fan/acc.
2016-04-19 21:20:24 -07:00
Luciano Resende 78b38109ed [SPARK-13419] [SQL] Update SubquerySuite to use checkAnswer for validation
## What changes were proposed in this pull request?

Change SubquerySuite to validate test results utilizing checkAnswer helper method

## How was this patch tested?

Existing tests

Author: Luciano Resende <lresende@apache.org>

Closes #12269 from lresende/SPARK-13419.
2016-04-19 21:02:10 -07:00
Joan 3ae25f244b [SPARK-13929] Use Scala reflection for UDTs
## What changes were proposed in this pull request?

Enable ScalaReflection and User Defined Types for plain Scala classes.

This involves the move of `schemaFor` from `ScalaReflection` trait (which is Runtime and Compile time (macros) reflection) to the `ScalaReflection` object (runtime reflection only) as I believe this code wouldn't work at compile time anyway as it manipulates `Class`'s that are not compiled yet.

## How was this patch tested?

Unit test

Author: Joan <joan@goyeau.com>

Closes #12149 from joan38/SPARK-13929-Scala-reflection.
2016-04-19 17:36:31 -07:00
Cheng Lian 10f273d8db [SPARK-14407][SQL] Hides HadoopFsRelation related data source API into execution/datasources package #12178
## What changes were proposed in this pull request?

This PR moves `HadoopFsRelation` related data source API into `execution/datasources` package.

Note that to avoid conflicts, this PR is based on #12153. Effective changes for this PR only consist of the last three commits. Will rebase after merging #12153.

## How was this patch tested?

Existing tests.

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

Closes #12361 from liancheng/spark-14407-hide-hadoop-fs-relation.
2016-04-19 17:32:23 -07:00
Herman van Hovell da8859226e [SPARK-4226] [SQL] Support IN/EXISTS Subqueries
### What changes were proposed in this pull request?
This PR adds support for in/exists predicate subqueries to Spark. Predicate sub-queries are used as a filtering condition in a query (this is the only supported use case). A predicate sub-query comes in two forms:

- `[NOT] EXISTS(subquery)`
- `[NOT] IN (subquery)`

This PR is (loosely) based on the work of davies (https://github.com/apache/spark/pull/10706) and chenghao-intel (https://github.com/apache/spark/pull/9055). They should be credited for the work they did.

### How was this patch tested?
Modified parsing unit tests.
Added tests to `org.apache.spark.sql.SQLQuerySuite`

cc rxin, davies & chenghao-intel

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

Closes #12306 from hvanhovell/SPARK-4226.
2016-04-19 15:16:02 -07:00
Wenchen Fan 5cb2e33609 [SPARK-14675][SQL] ClassFormatError when use Seq as Aggregator buffer type
## What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/12067, we now use expressions to do the aggregation in `TypedAggregateExpression`. To implement buffer merge, we produce a new buffer deserializer expression by replacing `AttributeReference` with right-side buffer attribute, like other `DeclarativeAggregate`s do, and finally combine the left and right buffer deserializer with `Invoke`.

However, after https://github.com/apache/spark/pull/12338, we will add loop variable to class members when codegen `MapObjects`. If the `Aggregator` buffer type is `Seq`, which is implemented by `MapObjects` expression, we will add the same loop variable to class members twice(by left and right buffer deserializer), which cause the `ClassFormatError`.

This PR fixes this issue by calling `distinct` before declare the class menbers.

## How was this patch tested?

new regression test in `DatasetAggregatorSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12468 from cloud-fan/bug.
2016-04-19 10:51:58 -07:00
Wenchen Fan 9ee95b6ecc [SPARK-14491] [SQL] refactor object operator framework to make it easy to eliminate serializations
## What changes were proposed in this pull request?

This PR tries to separate the serialization and deserialization logic from object operators, so that it's easier to eliminate unnecessary serializations in optimizer.

Typed aggregate related operators are special, they will deserialize the input row to multiple objects and it's difficult to simply use a deserializer operator to abstract it, so we still mix the deserialization logic there.

## How was this patch tested?

existing tests and new test in `EliminateSerializationSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12260 from cloud-fan/encoder.
2016-04-19 10:00:44 -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
Reynold Xin 1a3966472c [SPARK-14696][SQL] Add implicit encoders for boxed primitive types
## What changes were proposed in this pull request?
We currently only have implicit encoders for scala primitive types. We should also add implicit encoders for boxed primitives. Otherwise, the following code would not have an encoder:

```scala
sqlContext.range(1000).map { i => i }
```

## How was this patch tested?
Added a unit test case for this.

Author: Reynold Xin <rxin@databricks.com>

Closes #12466 from rxin/SPARK-14696.
2016-04-18 17:03:15 +08:00
Wenchen Fan 2f1d0320c9 [SPARK-13363][SQL] support Aggregator in RelationalGroupedDataset
## What changes were proposed in this pull request?

set the input encoder for `TypedColumn` in `RelationalGroupedDataset.agg`.

## How was this patch tested?

new tests in `DatasetAggregatorSuite`

close https://github.com/apache/spark/pull/11269

This PR brings https://github.com/apache/spark/pull/12359 up to date and fix the compile.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12451 from cloud-fan/agg.
2016-04-18 14:27:26 +08:00
Reynold Xin 527c780bb0 Revert "[SPARK-13363][SQL] support Aggregator in RelationalGroupedDataset"
This reverts commit 12854464c4.
2016-04-16 01:05:26 -07:00
Wenchen Fan 12854464c4 [SPARK-13363][SQL] support Aggregator in RelationalGroupedDataset
## What changes were proposed in this pull request?

set the input encoder for `TypedColumn` in `RelationalGroupedDataset.agg`.

## How was this patch tested?

new tests in `DatasetAggregatorSuite`

close https://github.com/apache/spark/pull/11269

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12359 from cloud-fan/agg.
2016-04-16 00:31:51 -07:00
Sameer Agarwal 4df65184b6 [SPARK-14620][SQL] Use/benchmark a better hash in VectorizedHashMap
## What changes were proposed in this pull request?

This PR uses a better hashing algorithm while probing the AggregateHashMap:

```java
long h = 0
h = (h ^ (0x9e3779b9)) + key_1 + (h << 6) + (h >>> 2);
h = (h ^ (0x9e3779b9)) + key_2 + (h << 6) + (h >>> 2);
h = (h ^ (0x9e3779b9)) + key_3 + (h << 6) + (h >>> 2);
...
h = (h ^ (0x9e3779b9)) + key_n + (h << 6) + (h >>> 2);
return h
```

Depends on: https://github.com/apache/spark/pull/12345
## How was this patch tested?

    Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4
    Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
    Aggregate w keys:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    codegen = F                              2417 / 2457          8.7         115.2       1.0X
    codegen = T hashmap = F                  1554 / 1581         13.5          74.1       1.6X
    codegen = T hashmap = T                   877 /  929         23.9          41.8       2.8X

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12379 from sameeragarwal/hash.
2016-04-15 15:55:31 -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
Sameer Agarwal b5c60bcdca [SPARK-14447][SQL] Speed up TungstenAggregate w/ keys using VectorizedHashMap
## What changes were proposed in this pull request?

This patch speeds up group-by aggregates by around 3-5x by leveraging an in-memory `AggregateHashMap` (please see https://github.com/apache/spark/pull/12161), an append-only aggregate hash map that can act as a 'cache' for extremely fast key-value lookups while evaluating aggregates (and fall back to the `BytesToBytesMap` if a given key isn't found).

Architecturally, it is backed by a power-of-2-sized array for index lookups and a columnar batch that stores the key-value pairs. The index lookups in the array rely on linear probing (with a small number of maximum tries) and use an inexpensive hash function which makes it really efficient for a majority of lookups. However, using linear probing and an inexpensive hash function also makes it less robust as compared to the `BytesToBytesMap` (especially for a large number of keys or even for certain distribution of keys) and requires us to fall back on the latter for correctness.

## How was this patch tested?

    Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4
    Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
    Aggregate w keys:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    codegen = F                              2124 / 2204          9.9         101.3       1.0X
    codegen = T hashmap = F                  1198 / 1364         17.5          57.1       1.8X
    codegen = T hashmap = T                   369 /  600         56.8          17.6       5.8X

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12345 from sameeragarwal/tungsten-aggregate-integration.
2016-04-14 20:57:03 -07:00
Reynold Xin dac40b68dc [SPARK-14619] Track internal accumulators (metrics) by stage attempt
## What changes were proposed in this pull request?
When there are multiple attempts for a stage, we currently only reset internal accumulator values if all the tasks are resubmitted. It would make more sense to reset the accumulator values for each stage attempt. This will allow us to eventually get rid of the internal flag in the Accumulator class. This is part of my bigger effort to simplify accumulators and task metrics.

## How was this patch tested?
Covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12378 from rxin/SPARK-14619.
2016-04-14 10:54:57 -07:00
gatorsmile 0d22092cd9 [SPARK-14125][SQL] Native DDL Support: Alter View
#### What changes were proposed in this pull request?
This PR is to provide a native DDL support for the following three Alter View commands:

Based on the Hive DDL document:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
##### 1. ALTER VIEW RENAME
**Syntax:**
```SQL
ALTER VIEW view_name RENAME TO new_view_name
```
- to change the name of a view to a different name
- not allowed to rename a view's name by ALTER TABLE

##### 2. ALTER VIEW SET TBLPROPERTIES
**Syntax:**
```SQL
ALTER VIEW view_name SET TBLPROPERTIES ('comment' = new_comment);
```
- to add metadata to a view
- not allowed to set views' properties by ALTER TABLE
- ignore it if trying to set a view's existing property key when the value is the same
- overwrite the value if trying to set a view's existing key to a different value

##### 3. ALTER VIEW UNSET TBLPROPERTIES
**Syntax:**
```SQL
ALTER VIEW view_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key')
```
- to remove metadata from a view
- not allowed to unset views' properties by ALTER TABLE
- issue an exception if trying to unset a view's non-existent key

#### How was this patch tested?
Added test cases to verify if it works properly.

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

Closes #12324 from gatorsmile/alterView.
2016-04-14 08:34:11 -07: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 62b7f306fb [SPARK-14607] [SPARK-14484] [SQL] fix case-insensitive predicates in FileSourceStrategy
## What changes were proposed in this pull request?

When prune the partitions or push down predicates, case-sensitivity is not respected. In order to make it work with case-insensitive, this PR update the AttributeReference inside predicate to use the name from schema.

## How was this patch tested?

Add regression tests for case-insensitive.

Author: Davies Liu <davies@databricks.com>

Closes #12371 from davies/case_insensi.
2016-04-13 17:17:19 -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
Wenchen Fan a5f8c9b15b [SPARK-14554][SQL][FOLLOW-UP] use checkDataset to check the result
## What changes were proposed in this pull request?

address this comment: https://github.com/apache/spark/pull/12322#discussion_r59417359

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12346 from cloud-fan/tmp.
2016-04-13 11:41:09 +08: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
Shixiong Zhu 6bf692147c [SPARK-14474][SQL] Move FileSource offset log into checkpointLocation
## What changes were proposed in this pull request?

Now that we have a single location for storing checkpointed state. This PR just propagates the checkpoint location into FileStreamSource so that we don't have one random log off on its own.

## How was this patch tested?

test("metadataPath should be in checkpointLocation")

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12247 from zsxwing/file-source-log-location.
2016-04-12 10:46:28 -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
Wenchen Fan 678b96e77b [SPARK-14535][SQL] Remove buildInternalScan from FileFormat
## What changes were proposed in this pull request?

Now `HadoopFsRelation` with all kinds of file formats can be handled in `FileSourceStrategy`, we can remove the branches for  `HadoopFsRelation` in `FileSourceStrategy` and the `buildInternalScan` API from `FileFormat`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12300 from cloud-fan/remove.
2016-04-11 22:59:42 -07:00
Wenchen Fan 52a801124f [SPARK-14554][SQL] disable whole stage codegen if there are too many input columns
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/12047/files#diff-94a1f59bcc9b6758c4ca874652437634R529, we may split field expressions codes in `CreateExternalRow` to support wide table. However, the whole stage codegen framework doesn't support it, because the input for expressions is not always the input row, but can be `CodeGenContext.currentVars`, which doesn't work well with `CodeGenContext.splitExpressions`.

Actually we do have a check to guard against this cases, but it's incomplete, it only checks output fields.

This PR improves the whole stage codegen support check, to disable it if there are too many input fields, so that we can avoid splitting field expressions codes in `CreateExternalRow` for whole stage codegen.

TODO: Is it a better solution if we can make `CodeGenContext.currentVars` work well with `CodeGenContext.splitExpressions`?

## How was this patch tested?

new test in DatasetSuite.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12322 from cloud-fan/codegen.
2016-04-11 22:58:35 -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
Eric Liang 6f27027d96 [SPARK-14475] Propagate user-defined context from driver to executors
## What changes were proposed in this pull request?

This adds a new API call `TaskContext.getLocalProperty` for getting properties set in the driver from executors. These local properties are automatically propagated from the driver to executors. For streaming, the context for streaming tasks will be the initial driver context when ssc.start() is called.

## How was this patch tested?

Unit tests.

cc JoshRosen

Author: Eric Liang <ekl@databricks.com>

Closes #12248 from ericl/sc-2813.
2016-04-11 18:33:54 -07:00
Rekha Joshi e82d95bf63 [SPARK-14372][SQL] Dataset.randomSplit() needs a Java version
## What changes were proposed in this pull request?

1.Added method randomSplitAsList() in Dataset for java
for https://issues.apache.org/jira/browse/SPARK-14372

## How was this patch tested?

TestSuite

Author: Rekha Joshi <rekhajoshm@gmail.com>
Author: Joshi <rekhajoshm@gmail.com>

Closes #12184 from rekhajoshm/SPARK-14372.
2016-04-11 17:13:30 +08: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
Davies Liu 5cb5edaf9c [SPARK-14419] [SQL] Improve HashedRelation for key fit within Long
## What changes were proposed in this pull request?

Currently, we use java HashMap for HashedRelation if the key could fit within a Long. The java HashMap and CompactBuffer are not memory efficient, the memory used by them is also accounted accurately.

This PR introduce a LongToUnsafeRowMap (similar to BytesToBytesMap) for better memory efficiency and performance.

This PR reopen #12190 to fix bugs.

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #12278 from davies/long_map3.
2016-04-09 17:44:38 -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
gatorsmile 9be5558e00 [SPARK-14481][SQL] Issue Exceptions for All Unsupported Options during Parsing
#### What changes were proposed in this pull request?
"Not good to slightly ignore all the un-supported options/clauses. We should either support it or throw an exception." A comment from yhuai in another PR https://github.com/apache/spark/pull/12146

- Can `Explain` be an exception? The `Formatted` clause is used in `HiveCompatibilitySuite`.
- Two unsupported clauses in `Drop Table` are handled in a separate PR: https://github.com/apache/spark/pull/12146

#### How was this patch tested?
Test cases are added to verify all the cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12255 from gatorsmile/warningToException.
2016-04-09 14:10:44 -07:00
Davies Liu f7ec854f1b Revert "[SPARK-14419] [SQL] Improve HashedRelation for key fit within Long"
This reverts commit 90c0a04506.
2016-04-09 13:51:28 -07:00
Davies Liu 90c0a04506 [SPARK-14419] [SQL] Improve HashedRelation for key fit within Long
## What changes were proposed in this pull request?

Currently, we use java HashMap for HashedRelation if the key could fit within a Long. The java HashMap and CompactBuffer are not memory efficient, the memory used by them is also accounted accurately.

This PR introduce a LongToUnsafeRowMap (similar to BytesToBytesMap) for better memory efficiency and performance.

## How was this patch tested?

Updated existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #12190 from davies/long_map2.
2016-04-09 00:37:55 -07:00
Reynold Xin 520dde48d0 [SPARK-14451][SQL] Move encoder definition into Aggregator interface
## What changes were proposed in this pull request?
When we first introduced Aggregators, we required the user of Aggregators to (implicitly) specify the encoders. It would actually make more sense to have the encoders be specified by the implementation of Aggregators, since each implementation should have the most state about how to encode its own data type.

Note that this simplifies the Java API because Java users no longer need to explicitly specify encoders for aggregators.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #12231 from rxin/SPARK-14451.
2016-04-09 00:00:39 -07:00
hyukjinkwon 73b56a3c6c [SPARK-14189][SQL] JSON data sources find compatible types even if inferred decimal type is not capable of the others
## What changes were proposed in this pull request?

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

When inferred types in the same field during finding compatible `DataType`, are `IntegralType` and `DecimalType` but `DecimalType` is not capable of the given `IntegralType`, JSON data source simply fails to find a compatible type resulting in `StringType`.

This can be observed when `prefersDecimal` is enabled.

```scala
def mixedIntegerAndDoubleRecords: RDD[String] =
  sqlContext.sparkContext.parallelize(
    """{"a": 3, "b": 1.1}""" ::
    """{"a": 3.1, "b": 1}""" :: Nil)

val jsonDF = sqlContext.read
  .option("prefersDecimal", "true")
  .json(mixedIntegerAndDoubleRecords)
  .printSchema()
```

- **Before**

```
root
 |-- a: string (nullable = true)
 |-- b: string (nullable = true)
```

- **After**

```
root
 |-- a: decimal(21, 1) (nullable = true)
 |-- b: decimal(21, 1) (nullable = true)
```
(Note that integer is inferred as `LongType` which becomes `DecimalType(20, 0)`)

## How was this patch tested?

unit tests were used and style tests by `dev/run_tests`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11993 from HyukjinKwon/SPARK-14189.
2016-04-08 00:30:26 -07:00
hyukjinkwon 725b860e2b [SPARK-14103][SQL] Parse unescaped quotes in CSV data source.
## What changes were proposed in this pull request?

This PR resolves the problem during parsing unescaped quotes in input data. For example, currently the data below:

```
"a"b,ccc,ddd
e,f,g
```

produces a data below:

- **Before**

```bash
["a"b,ccc,ddd[\n]e,f,g]  <- as a value.
```

- **After**

```bash
["a"b], [ccc], [ddd]
[e], [f], [g]
```

This PR bumps up the Univocity parser's version. This was fixed in `2.0.2`, https://github.com/uniVocity/univocity-parsers/issues/60.

## How was this patch tested?

Unit tests in `CSVSuite` and `sbt/sbt scalastyle`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12226 from HyukjinKwon/SPARK-14103-quote.
2016-04-08 00:28:59 -07: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
Tathagata Das 3aa7d76395 [SQL][TESTS] Fix for flaky test in ContinuousQueryManagerSuite
## What changes were proposed in this pull request?

The timeouts were lower the other timeouts in the test. Other tests were stable over the last month.

## How was this patch tested?

Jenkins tests.

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

Closes #12219 from tdas/flaky-test-fix.
2016-04-07 10:51:49 -07:00
Reynold Xin 9ca0760d67 [SPARK-10063][SQL] Remove DirectParquetOutputCommitter
## What changes were proposed in this pull request?
This patch removes DirectParquetOutputCommitter. This was initially created by Databricks as a faster way to write Parquet data to S3. However, given how the underlying S3 Hadoop implementation works, this committer only works when there are no failures. If there are multiple attempts of the same task (e.g. speculation or task failures or node failures), the output data can be corrupted. I don't think this performance optimization outweighs the correctness issue.

## How was this patch tested?
Removed the related tests also.

Author: Reynold Xin <rxin@databricks.com>

Closes #12229 from rxin/SPARK-10063.
2016-04-07 00:51:45 -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
Luciano Resende 611dbce4bd [SPARK-12555][SQL] Result should not be corrupted after input columns are reordered
This PR add test case described in SPARK-12555 to validate that correct data is returned when input data is reordered and to avoid future regressions.

Author: Luciano Resende <lresende@apache.org>

Closes #11623 from lresende/SPARK-12555.
2016-04-07 08:35:00 +08:00
Marcelo Vanzin 864d1b4d66 [SPARK-14436][SQL] Make JavaDatasetAggregatorSuiteBase public.
Without this, unit tests that extend that class fail for me locally
on maven, because JUnit tries to run methods in that class and gets
an IllegalAccessError.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #12212 from vanzin/SPARK-14436.
2016-04-06 16:50:59 -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
Shixiong Zhu a4ead6d388 [SPARK-14382][SQL] QueryProgress should be post after committedOffsets is updated
## What changes were proposed in this pull request?

Make sure QueryProgress is post after committedOffsets is updated. If QueryProgress is post before committedOffsets is updated, the listener may see a wrong sinkStatus (created from committedOffsets).

See https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.2/644/testReport/junit/org.apache.spark.sql.util/ContinuousQueryListenerSuite/single_listener/ for an example of the failure.

## How was this patch tested?

Existing unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12155 from zsxwing/SPARK-14382.
2016-04-06 12:28:04 -07:00
Sameer Agarwal bb1fa5b218 [SPARK-14320][SQL] Make ColumnarBatch.Row mutable
## What changes were proposed in this pull request?

In order to leverage a data structure like `AggregateHashMap` (https://github.com/apache/spark/pull/12055) to speed up aggregates with keys, we need to make `ColumnarBatch.Row` mutable.

## How was this patch tested?

Unit test in `ColumnarBatchSuite`. Also, tested via `BenchmarkWholeStageCodegen`.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12103 from sameeragarwal/mutable-row.
2016-04-06 11:59:42 -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
Michael Armbrust 59236e5c5b [SPARK-14288][SQL] Memory Sink for streaming
This PR exposes the internal testing `MemorySink` though the data source API.  This will allow users to easily test streaming applications in the Spark shell or other local tests.

Usage:
```scala
inputStream.write
  .format("memory")
  .queryName("memStream")
  .startStream()

// Now you can query the result of the stream here.
sqlContext.table("memStream")
```

The most complicated part of the logic is choosing the checkpoint directory.  There are a few requirements we are attempting to satisfy here:
 - when working in the shell locally, it should just work with no extra configuration.
 - when working on a cluster you should be able to make it easily create the checkpoint on a distributed file system so you can test aggregation (state checkpoints are also stored in this directory and must be accessible from workers).
 - it should be clear that you can't resume since the data is just in memory.

The chosen algorithm proceeds as follows:
 - the user gives a checkpoint directory, use it
 - if the conf has a checkpoint location, use `$location/$queryName`
 - if neither, create a local directory
 - always check to make sure there are no offsets written to the directory

Author: Michael Armbrust <michael@databricks.com>

Closes #12119 from marmbrus/memorySink.
2016-04-06 10:05:02 -07:00
gatorsmile 68be5b9e8a [SPARK-14396][SQL] Throw Exceptions for DDLs of Partitioned Views
#### What changes were proposed in this pull request?

Because the concept of partitioning is associated with physical tables, we disable all the supports of partitioned views, which are defined in the following three commands in [Hive DDL Manual](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Create/Drop/AlterView):
```
ALTER VIEW view DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...];

ALTER VIEW view ADD [IF NOT EXISTS] PARTITION spec;

CREATE VIEW [IF NOT EXISTS] [db_name.]view_name [(column_name [COMMENT column_comment], ...) ]
  [COMMENT view_comment]
  [TBLPROPERTIES (property_name = property_value, ...)]
  AS SELECT ...;
```

An exception is thrown when users issue any of these three DDL commands.

#### How was this patch tested?
Added test cases for parsing create view and changed the existing test cases to verify if the exceptions are thrown.

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

Closes #12169 from gatorsmile/viewPartition.
2016-04-05 22:33:44 -07:00
Andrew Or adbfdb878d [SPARK-14128][SQL] Alter table DDL followup
## What changes were proposed in this pull request?

This is just a followup to #12121, which implemented the alter table DDLs using the `SessionCatalog`. Specially, this corrects the behavior of setting the location of a datasource table. For datasource tables, we need to set the `locationUri` in addition to the `path` entry in the serde properties. Additionally, changing the location of a datasource table partition is not allowed.

## How was this patch tested?

`DDLSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #12186 from andrewor14/alter-table-ddl-followup.
2016-04-05 21:23:20 -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
Eric Liang 7d29c72f64 [SPARK-14359] Unit tests for java 8 lambda syntax with typed aggregates
## What changes were proposed in this pull request?

Adds unit tests for java 8 lambda syntax with typed aggregates as a follow-up to #12168

## How was this patch tested?

Unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #12181 from ericl/sc-2794-2.
2016-04-05 21:22:20 -05:00
Marcelo Vanzin d5ee9d5c24 [SPARK-529][SQL] Modify SQLConf to use new config API from core.
Because SQL keeps track of all known configs, some customization was
needed in SQLConf to allow that, since the core API does not have that
feature.

Tested via existing (and slightly updated) unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #11570 from vanzin/SPARK-529-sql.
2016-04-05 15:19:51 -07: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
Shixiong Zhu 463bac0011 [SPARK-14257][SQL] Allow multiple continuous queries to be started from the same DataFrame
## What changes were proposed in this pull request?

Make StreamingRelation store the closure to create the source in StreamExecution so that we can start multiple continuous queries from the same DataFrame.

## How was this patch tested?

`test("DataFrame reuse")`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12049 from zsxwing/df-reuse.
2016-04-05 11:12:05 -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
Eric Liang 064623014e [SPARK-14359] Create built-in functions for typed aggregates in Java
## What changes were proposed in this pull request?

This adds the corresponding Java static functions for built-in typed aggregates already exposed in Scala.

## How was this patch tested?

Unit tests.

rxin

Author: Eric Liang <ekl@databricks.com>

Closes #12168 from ericl/sc-2794.
2016-04-05 00:30:55 -05:00
Burak Yavuz ba24d1ee9a [SPARK-14287] isStreaming method for Dataset
With the addition of StreamExecution (ContinuousQuery) to Datasets, data will become unbounded. With unbounded data, the execution of some methods and operations will not make sense, e.g. `Dataset.count()`.

A simple API is required to check whether the data in a Dataset is bounded or unbounded. This will allow users to check whether their Dataset is in streaming mode or not. ML algorithms may check if the data is unbounded and throw an exception for example.

The implementation of this method is simple, however naming it is the challenge. Some possible names for this method are:
 - isStreaming
 - isContinuous
 - isBounded
 - isUnbounded

I've gone with `isStreaming` for now. We can change it before Spark 2.0 if we decide to come up with a different name. For that reason I've marked it as `Experimental`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #12080 from brkyvz/is-streaming.
2016-04-04 19:04:09 -07:00
Davies Liu 400b2f863f [SPARK-14259] [SQL] Merging small files together based on the cost of opening
## What changes were proposed in this pull request?

This PR basically re-do the things in #12068 but with a different model, which should work better in case of small files with different sizes.

## How was this patch tested?

Updated existing tests.

Ran a query on thousands of partitioned small files locally, with all default settings (the cost to open a file should be over estimated), the durations of tasks become smaller and smaller, which is good (the last few tasks will be shortest).

Author: Davies Liu <davies@databricks.com>

Closes #12095 from davies/file_cost.
2016-04-04 14:41:03 -07:00
Davies Liu cc70f17416 [SPARK-14334] [SQL] add toLocalIterator for Dataset/DataFrame
## What changes were proposed in this pull request?

RDD.toLocalIterator() could be used to fetch one partition at a time to reduce the memory usage. Right now, for Dataset/Dataframe we have to use df.rdd.toLocalIterator, which is super slow also requires lots of memory (because of the Java serializer or even Kyro serializer).

This PR introduce an optimized toLocalIterator for Dataset/DataFrame, which is much faster and requires much less memory. For a partition with 5 millions rows, `df.rdd.toIterator` took about 100 seconds, but df.toIterator took less than 7 seconds. For 10 millions row, rdd.toIterator will crash (not enough memory) with 4G heap, but df.toLocalIterator could finished in 12 seconds.

The JDBC server has been updated to use DataFrame.toIterator.

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #12114 from davies/local_iterator.
2016-04-04 13:31:44 -07:00
Shixiong Zhu 855ed44ed3 [SPARK-14176][SQL] Add DataFrameWriter.trigger to set the stream batch period
## What changes were proposed in this pull request?

Add a processing time trigger to control the batch processing speed

## How was this patch tested?

Unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11976 from zsxwing/trigger.
2016-04-04 10:54:06 -07:00
Davies Liu 745425332f [SPARK-14137] [SQL] Cleanup hash join
## What changes were proposed in this pull request?

This PR did a few cleanup on HashedRelation and HashJoin:

1) Merge HashedRelation and UniqueHashedRelation together
2) Return an iterator from HashedRelation, so we donot need a create many UnsafeRow objects.
3) Return a copy of HashedRelation for thread-safety in BroadcastJoin, so we can re-use the UnafeRow objects.
4) Cleanup HashJoin, share most of the code between BroadcastHashJoin and ShuffleHashJoin
5) Removed UniqueLongHashedRelation, which will be replaced by LongUnsafeMap (another PR).
6) Update benchmark, before this patch, the selectivity of joins are too high.

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #12102 from davies/cleanup_hash.
2016-04-04 10:01:24 -07:00
Matei Zaharia 76f3c735aa [SPARK-14356] Update spark.sql.execution.debug to work on Datasets
## What changes were proposed in this pull request?

Update DebugQuery to work on Datasets of any type, not just DataFrames.

## How was this patch tested?

Added unit tests, checked in spark-shell.

Author: Matei Zaharia <matei@databricks.com>

Closes #12140 from mateiz/debug-dataset.
2016-04-03 21:08:54 -07: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
hyukjinkwon 2262a93358 [SPARK-14231] [SQL] JSON data source infers floating-point values as a double when they do not fit in a decimal
## What changes were proposed in this pull request?

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

Currently, JSON data source supports to infer `DecimalType` for big numbers and `floatAsBigDecimal` option which reads floating-point values as `DecimalType`.

But there are few restrictions in Spark `DecimalType` below:

1. The precision cannot be bigger than 38.
2. scale cannot be bigger than precision.

Currently, both restrictions are not being handled.

This PR handles the cases by inferring them as `DoubleType`. Also, the option name was changed from `floatAsBigDecimal` to `prefersDecimal` as suggested [here](https://issues.apache.org/jira/browse/SPARK-14231?focusedCommentId=15215579&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15215579).

So, the codes below:

```scala
def doubleRecords: RDD[String] =
  sqlContext.sparkContext.parallelize(
    s"""{"a": 1${"0" * 38}, "b": 0.01}""" ::
    s"""{"a": 2${"0" * 38}, "b": 0.02}""" :: Nil)

val jsonDF = sqlContext.read
  .option("prefersDecimal", "true")
  .json(doubleRecords)
jsonDF.printSchema()
```

produces below:

- **Before**

```scala
org.apache.spark.sql.AnalysisException: Decimal scale (2) cannot be greater than precision (1).;
	at org.apache.spark.sql.types.DecimalType.<init>(DecimalType.scala:44)
	at org.apache.spark.sql.execution.datasources.json.InferSchema$.org$apache$spark$sql$execution$datasources$json$InferSchema$$inferField(InferSchema.scala:144)
	at org.apache.spark.sql.execution.datasources.json.InferSchema$.org$apache$spark$sql$execution$datasources$json$InferSchema$$inferField(InferSchema.scala:108)
	at
...
```

- **After**

```scala
root
 |-- a: double (nullable = true)
 |-- b: double (nullable = true)
```

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12030 from HyukjinKwon/SPARK-14231.
2016-04-02 23:12:04 -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
Reynold Xin a3e293542a [HOTFIX] Disable StateStoreSuite.maintenance 2016-04-02 12:44:02 -07:00
Reynold Xin 67d753516d [HOTFIX] Fix compilation break. 2016-04-02 00:00:19 -07:00
hyukjinkwon d7982a3a9a [MINOR][SQL] Fix comments styl and correct several styles and nits in CSV data source
## What changes were proposed in this pull request?

While trying to create a PR (which was not an issue at the end), I just corrected some style nits.

So, I removed the changes except for some coding style corrections.

- According to the [scala-style-guide#documentation-style](https://github.com/databricks/scala-style-guide#documentation-style), Scala style comments are discouraged.

>```scala
>/** This is a correct one-liner, short description. */
>
>/**
>  * This is correct multi-line JavaDoc comment. And
>  * this is my second line, and if I keep typing, this would be
>  * my third line.
>  */
>
>/** In Spark, we don't use the ScalaDoc style so this
>   * is not correct.
>   */
>```

- Double newlines between consecutive methods was removed. According to [scala-style-guide#blank-lines-vertical-whitespace](https://github.com/databricks/scala-style-guide#blank-lines-vertical-whitespace), single newline appears when

>Between consecutive members (or initializers) of a class: fields, constructors, methods, nested classes, static initializers, instance initializers.

- Remove uesless parentheses in tests

- Use `mapPartitions` instead of `mapPartitionsWithIndex()`.

## How was this patch tested?

Unit tests were used and `dev/run_tests` for style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12109 from HyukjinKwon/SPARK-14271.
2016-04-01 22:51:47 -07:00
Reynold Xin f414154418 [SPARK-14285][SQL] Implement common type-safe aggregate functions
## What changes were proposed in this pull request?
In the Dataset API, it is fairly difficult for users to perform simple aggregations in a type-safe way at the moment because there are no aggregators that have been implemented. This pull request adds a few common aggregate functions in expressions.scala.typed package, and also creates the expressions.java.typed package without implementation. The java implementation should probably come as a separate pull request. One challenge there is to resolve the type difference between Scala primitive types and Java boxed types.

## How was this patch tested?
Added unit tests for them.

Author: Reynold Xin <rxin@databricks.com>

Closes #12077 from rxin/SPARK-14285.
2016-04-01 22:46:56 -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
Kazuaki Ishizaki 877dc712e6 [SPARK-14138] [SQL] [MASTER] Fix generated SpecificColumnarIterator code can exceed JVM size limit for cached DataFrames
## What changes were proposed in this pull request?

This PR reduces Java byte code size of method in ```SpecificColumnarIterator``` by using a approach to make a group for  lot of ```ColumnAccessor``` instantiations or method calls (more than 200) into a method

## How was this patch tested?

Added a new unit test, which includes large instantiations and method calls, to ```InMemoryColumnarQuerySuite```

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

Closes #12108 from kiszk/SPARK-14138-master.
2016-04-01 22:38:07 -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
Shixiong Zhu 0b7d4966ca [SPARK-14316][SQL] StateStoreCoordinator should extend ThreadSafeRpcEndpoint
## What changes were proposed in this pull request?

RpcEndpoint is not thread safe and allows multiple messages to be processed at the same time. StateStoreCoordinator should use ThreadSafeRpcEndpoint.

## How was this patch tested?

Existing unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12100 from zsxwing/fix-StateStoreCoordinator.
2016-04-01 15:00:38 -07:00
Liang-Chi Hsieh 3e991dbc31 [SPARK-13674] [SQL] Add wholestage codegen support to Sample
JIRA: https://issues.apache.org/jira/browse/SPARK-13674

## What changes were proposed in this pull request?

Sample operator doesn't support wholestage codegen now. This pr is to add support to it.

## How was this patch tested?

A test is added into `BenchmarkWholeStageCodegen`. Besides, all tests should be passed.

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

Closes #11517 from viirya/add-wholestage-sample.
2016-04-01 14:02:32 -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
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
Shixiong Zhu e785402826 [SPARK-14304][SQL][TESTS] Fix tests that don't create temp files in the java.io.tmpdir folder
## What changes were proposed in this pull request?

If I press `CTRL-C` when running these tests, the temp files will be left in `sql/core` folder and I need to delete them manually. It's annoying. This PR just moves the temp files to the `java.io.tmpdir` folder and add a name prefix for them.

## How was this patch tested?

Existing Jenkins tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12093 from zsxwing/temp-file.
2016-03-31 12:17:25 -07: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
Sameer Agarwal 8d6207206c [SPARK-14263][SQL] Benchmark Vectorized HashMap for GroupBy Aggregates
## What changes were proposed in this pull request?

This PR proposes a new data-structure based on a vectorized hashmap that can be potentially _codegened_ in `TungstenAggregate` to speed up aggregates with group by. Micro-benchmarks show a 10x improvement over the current `BytesToBytes` aggregation map.

## How was this patch tested?

    Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
    BytesToBytesMap:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    hash                                      108 /  119         96.9          10.3       1.0X
    fast hash                                  63 /   70        166.2           6.0       1.7X
    arrayEqual                                 70 /   73        150.8           6.6       1.6X
    Java HashMap (Long)                       141 /  200         74.3          13.5       0.8X
    Java HashMap (two ints)                   145 /  185         72.3          13.8       0.7X
    Java HashMap (UnsafeRow)                  499 /  524         21.0          47.6       0.2X
    BytesToBytesMap (off Heap)                483 /  548         21.7          46.0       0.2X
    BytesToBytesMap (on Heap)                 485 /  562         21.6          46.2       0.2X
    Vectorized Hashmap                         54 /   60        193.7           5.2       2.0X

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12055 from sameeragarwal/vectorized-hashmap.
2016-03-31 11:53:13 -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
Cheng Lian 26445c2e47 [SPARK-14206][SQL] buildReader() implementation for CSV
## What changes were proposed in this pull request?

Major changes:

1. Implement `FileFormat.buildReader()` for the CSV data source.
1. Add an extra argument to `FileFormat.buildReader()`, `physicalSchema`, which is basically the result of `FileFormat.inferSchema` or user specified schema.

   This argument is necessary because the CSV data source needs to know all the columns of the underlying files to read the file.

## How was this patch tested?

Existing tests should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #12002 from liancheng/spark-14206-csv-build-reader.
2016-03-30 18:21:06 -07:00
Travis Crawford da54abfd87 [SPARK-14081][SQL] - Preserve DataFrame column types when filling nulls.
## What changes were proposed in this pull request?
This change resolves an issue where `DataFrameNaFunctions.fill` changes a `FloatType` column to a `DoubleType`. We also clarify the contract that replacement values will be cast to the column data type, which may change the replacement value when casting to a lower precision type.

## How was this patch tested?
This patch has associated unit tests.

Author: Travis Crawford <travis@medium.com>

Closes #11967 from traviscrawford/SPARK-14081-dataframena.
2016-03-30 16:59:52 -07:00
Takeshi YAMAMURO dadf0138b3 [SPARK-14259][SQL] Add a FileSourceStrategy option for limiting #files in a partition
## What changes were proposed in this pull request?
This pr is to add a config to control the maximum number of files as even small files have a non-trivial fixed cost. The current packing can put a lot of small files together which cases straggler tasks.

## How was this patch tested?
I added tests to check if many files get split into partitions in FileSourceStrategySuite.

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

Closes #12068 from maropu/SPARK-14259.
2016-03-30 16:02:48 -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
Eric Liang e58c4cb3c5 [SPARK-14227][SQL] Add method for printing out generated code for debugging
## What changes were proposed in this pull request?

This adds `debugCodegen` to the debug package for query execution.

## How was this patch tested?

Unit and manual testing. Output example:

```
scala> import org.apache.spark.sql.execution.debug._
import org.apache.spark.sql.execution.debug._

scala> sqlContext.range(100).groupBy("id").count().orderBy("id").debugCodegen()
Found 3 WholeStageCodegen subtrees.
== Subtree 1 / 3 ==
WholeStageCodegen
:  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L])
:     +- Range 0, 1, 1, 100, [id#0L]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ /** Codegened pipeline for:
/* 006 */ * TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L])
/* 007 */ +- Range 0, 1, 1, 100, [id#0L]
/* 008 */ */
/* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 010 */   private Object[] references;
/* 011 */   private boolean agg_initAgg;
/* 012 */   private org.apache.spark.sql.execution.aggregate.TungstenAggregate agg_plan;
/* 013 */   private org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap;
/* 014 */   private org.apache.spark.sql.execution.UnsafeKVExternalSorter agg_sorter;
/* 015 */   private org.apache.spark.unsafe.KVIterator agg_mapIter;
/* 016 */   private org.apache.spark.sql.execution.metric.LongSQLMetric range_numOutputRows;
/* 017 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue range_metricValue;
/* 018 */   private boolean range_initRange;
/* 019 */   private long range_partitionEnd;
/* 020 */   private long range_number;
/* 021 */   private boolean range_overflow;
/* 022 */   private scala.collection.Iterator range_input;
/* 023 */   private UnsafeRow range_result;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder range_holder;
/* 025 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter range_rowWriter;
/* 026 */   private UnsafeRow agg_result;
/* 027 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 028 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 029 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowJoiner agg_unsafeRowJoiner;
/* 030 */   private org.apache.spark.sql.execution.metric.LongSQLMetric wholestagecodegen_numOutputRows;
/* 031 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue wholestagecodegen_metricValue;
/* 032 */
/* 033 */   public GeneratedIterator(Object[] references) {
/* 034 */     this.references = references;
/* 035 */   }
/* 036 */
/* 037 */   public void init(scala.collection.Iterator inputs[]) {
/* 038 */     agg_initAgg = false;
/* 039 */     this.agg_plan = (org.apache.spark.sql.execution.aggregate.TungstenAggregate) references[0];
/* 040 */     agg_hashMap = agg_plan.createHashMap();
/* 041 */
/* 042 */     this.range_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1];
/* 043 */     range_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) range_numOutputRows.localValue();
/* 044 */     range_initRange = false;
/* 045 */     range_partitionEnd = 0L;
/* 046 */     range_number = 0L;
/* 047 */     range_overflow = false;
/* 048 */     range_input = inputs[0];
/* 049 */     range_result = new UnsafeRow(1);
/* 050 */     this.range_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(range_result, 0);
/* 051 */     this.range_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(range_holder, 1);
/* 052 */     agg_result = new UnsafeRow(1);
/* 053 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 054 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 055 */     agg_unsafeRowJoiner = agg_plan.createUnsafeJoiner();
/* 056 */     this.wholestagecodegen_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[2];
/* 057 */     wholestagecodegen_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) wholestagecodegen_numOutputRows.localValue();
/* 058 */   }
/* 059 */
/* 060 */   private void agg_doAggregateWithKeys() throws java.io.IOException {
/* 061 */     /*** PRODUCE: Range 0, 1, 1, 100, [id#0L] */
/* 062 */
/* 063 */     // initialize Range
/* 064 */     if (!range_initRange) {
/* 065 */       range_initRange = true;
/* 066 */       if (range_input.hasNext()) {
/* 067 */         initRange(((InternalRow) range_input.next()).getInt(0));
/* 068 */       } else {
/* 069 */         return;
/* 070 */       }
/* 071 */     }
/* 072 */
/* 073 */     while (!range_overflow && range_number < range_partitionEnd) {
/* 074 */       long range_value = range_number;
/* 075 */       range_number += 1L;
/* 076 */       if (range_number < range_value ^ 1L < 0) {
/* 077 */         range_overflow = true;
/* 078 */       }
/* 079 */
/* 080 */       /*** CONSUME: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) */
/* 081 */
/* 082 */       // generate grouping key
/* 083 */       agg_rowWriter.write(0, range_value);
/* 084 */       /* hash(input[0, bigint], 42) */
/* 085 */       int agg_value1 = 42;
/* 086 */
/* 087 */       agg_value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(range_value, agg_value1);
/* 088 */       UnsafeRow agg_aggBuffer = null;
/* 089 */       if (true) {
/* 090 */         // try to get the buffer from hash map
/* 091 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1);
/* 092 */       }
/* 093 */       if (agg_aggBuffer == null) {
/* 094 */         if (agg_sorter == null) {
/* 095 */           agg_sorter = agg_hashMap.destructAndCreateExternalSorter();
/* 096 */         } else {
/* 097 */           agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter());
/* 098 */         }
/* 099 */
/* 100 */         // the hash map had be spilled, it should have enough memory now,
/* 101 */         // try  to allocate buffer again.
/* 102 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1);
/* 103 */         if (agg_aggBuffer == null) {
/* 104 */           // failed to allocate the first page
/* 105 */           throw new OutOfMemoryError("No enough memory for aggregation");
/* 106 */         }
/* 107 */       }
/* 108 */
/* 109 */       // evaluate aggregate function
/* 110 */       /* (input[0, bigint] + 1) */
/* 111 */       /* input[0, bigint] */
/* 112 */       long agg_value4 = agg_aggBuffer.getLong(0);
/* 113 */
/* 114 */       long agg_value3 = -1L;
/* 115 */       agg_value3 = agg_value4 + 1L;
/* 116 */       // update aggregate buffer
/* 117 */       agg_aggBuffer.setLong(0, agg_value3);
/* 118 */
/* 119 */       if (shouldStop()) return;
/* 120 */     }
/* 121 */
/* 122 */     agg_mapIter = agg_plan.finishAggregate(agg_hashMap, agg_sorter);
/* 123 */   }
/* 124 */
/* 125 */   private void initRange(int idx) {
/* 126 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 127 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(1L);
/* 128 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(100L);
/* 129 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 130 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 131 */
/* 132 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 133 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 134 */       range_number = Long.MAX_VALUE;
/* 135 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 136 */       range_number = Long.MIN_VALUE;
/* 137 */     } else {
/* 138 */       range_number = st.longValue();
/* 139 */     }
/* 140 */
/* 141 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 142 */     .multiply(step).add(start);
/* 143 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 144 */       range_partitionEnd = Long.MAX_VALUE;
/* 145 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 146 */       range_partitionEnd = Long.MIN_VALUE;
/* 147 */     } else {
/* 148 */       range_partitionEnd = end.longValue();
/* 149 */     }
/* 150 */
/* 151 */     range_metricValue.add((range_partitionEnd - range_number) / 1L);
/* 152 */   }
/* 153 */
/* 154 */   protected void processNext() throws java.io.IOException {
/* 155 */     /*** PRODUCE: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) */
/* 156 */
/* 157 */     if (!agg_initAgg) {
/* 158 */       agg_initAgg = true;
/* 159 */       agg_doAggregateWithKeys();
/* 160 */     }
/* 161 */
/* 162 */     // output the result
/* 163 */     while (agg_mapIter.next()) {
/* 164 */       wholestagecodegen_metricValue.add(1);
/* 165 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 166 */       UnsafeRow agg_aggBuffer1 = (UnsafeRow) agg_mapIter.getValue();
/* 167 */
/* 168 */       UnsafeRow agg_resultRow = agg_unsafeRowJoiner.join(agg_aggKey, agg_aggBuffer1);
/* 169 */
/* 170 */       /*** CONSUME: WholeStageCodegen */
/* 171 */
/* 172 */       append(agg_resultRow);
/* 173 */
/* 174 */       if (shouldStop()) return;
/* 175 */     }
/* 176 */
/* 177 */     agg_mapIter.close();
/* 178 */     if (agg_sorter == null) {
/* 179 */       agg_hashMap.free();
/* 180 */     }
/* 181 */   }
/* 182 */ }

== Subtree 2 / 3 ==
WholeStageCodegen
:  +- Sort [id#0L ASC], true, 0
:     +- INPUT
+- Exchange rangepartitioning(id#0L ASC, 200), None
   +- WholeStageCodegen
      :  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L])
      :     +- INPUT
      +- Exchange hashpartitioning(id#0L, 200), None
         +- WholeStageCodegen
            :  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L])
            :     +- Range 0, 1, 1, 100, [id#0L]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ /** Codegened pipeline for:
/* 006 */ * Sort [id#0L ASC], true, 0
/* 007 */ +- INPUT
/* 008 */ */
/* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 010 */   private Object[] references;
/* 011 */   private boolean sort_needToSort;
/* 012 */   private org.apache.spark.sql.execution.Sort sort_plan;
/* 013 */   private org.apache.spark.sql.execution.UnsafeExternalRowSorter sort_sorter;
/* 014 */   private org.apache.spark.executor.TaskMetrics sort_metrics;
/* 015 */   private scala.collection.Iterator<UnsafeRow> sort_sortedIter;
/* 016 */   private scala.collection.Iterator inputadapter_input;
/* 017 */   private org.apache.spark.sql.execution.metric.LongSQLMetric sort_dataSize;
/* 018 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue sort_metricValue;
/* 019 */   private org.apache.spark.sql.execution.metric.LongSQLMetric sort_spillSize;
/* 020 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue sort_metricValue1;
/* 021 */
/* 022 */   public GeneratedIterator(Object[] references) {
/* 023 */     this.references = references;
/* 024 */   }
/* 025 */
/* 026 */   public void init(scala.collection.Iterator inputs[]) {
/* 027 */     sort_needToSort = true;
/* 028 */     this.sort_plan = (org.apache.spark.sql.execution.Sort) references[0];
/* 029 */     sort_sorter = sort_plan.createSorter();
/* 030 */     sort_metrics = org.apache.spark.TaskContext.get().taskMetrics();
/* 031 */
/* 032 */     inputadapter_input = inputs[0];
/* 033 */     this.sort_dataSize = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1];
/* 034 */     sort_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) sort_dataSize.localValue();
/* 035 */     this.sort_spillSize = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[2];
/* 036 */     sort_metricValue1 = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) sort_spillSize.localValue();
/* 037 */   }
/* 038 */
/* 039 */   private void sort_addToSorter() throws java.io.IOException {
/* 040 */     /*** PRODUCE: INPUT */
/* 041 */
/* 042 */     while (inputadapter_input.hasNext()) {
/* 043 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 044 */       /*** CONSUME: Sort [id#0L ASC], true, 0 */
/* 045 */
/* 046 */       sort_sorter.insertRow((UnsafeRow)inputadapter_row);
/* 047 */       if (shouldStop()) return;
/* 048 */     }
/* 049 */
/* 050 */   }
/* 051 */
/* 052 */   protected void processNext() throws java.io.IOException {
/* 053 */     /*** PRODUCE: Sort [id#0L ASC], true, 0 */
/* 054 */     if (sort_needToSort) {
/* 055 */       sort_addToSorter();
/* 056 */       Long sort_spillSizeBefore = sort_metrics.memoryBytesSpilled();
/* 057 */       sort_sortedIter = sort_sorter.sort();
/* 058 */       sort_metricValue.add(sort_sorter.getPeakMemoryUsage());
/* 059 */       sort_metricValue1.add(sort_metrics.memoryBytesSpilled() - sort_spillSizeBefore);
/* 060 */       sort_metrics.incPeakExecutionMemory(sort_sorter.getPeakMemoryUsage());
/* 061 */       sort_needToSort = false;
/* 062 */     }
/* 063 */
/* 064 */     while (sort_sortedIter.hasNext()) {
/* 065 */       UnsafeRow sort_outputRow = (UnsafeRow)sort_sortedIter.next();
/* 066 */
/* 067 */       /*** CONSUME: WholeStageCodegen */
/* 068 */
/* 069 */       append(sort_outputRow);
/* 070 */
/* 071 */       if (shouldStop()) return;
/* 072 */     }
/* 073 */   }
/* 074 */ }

== Subtree 3 / 3 ==
WholeStageCodegen
:  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L])
:     +- INPUT
+- Exchange hashpartitioning(id#0L, 200), None
   +- WholeStageCodegen
      :  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L])
      :     +- Range 0, 1, 1, 100, [id#0L]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ /** Codegened pipeline for:
/* 006 */ * TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L])
/* 007 */ +- INPUT
/* 008 */ */
/* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 010 */   private Object[] references;
/* 011 */   private boolean agg_initAgg;
/* 012 */   private org.apache.spark.sql.execution.aggregate.TungstenAggregate agg_plan;
/* 013 */   private org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap;
/* 014 */   private org.apache.spark.sql.execution.UnsafeKVExternalSorter agg_sorter;
/* 015 */   private org.apache.spark.unsafe.KVIterator agg_mapIter;
/* 016 */   private scala.collection.Iterator inputadapter_input;
/* 017 */   private UnsafeRow agg_result;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 020 */   private UnsafeRow agg_result1;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder1;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter1;
/* 023 */   private org.apache.spark.sql.execution.metric.LongSQLMetric wholestagecodegen_numOutputRows;
/* 024 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue wholestagecodegen_metricValue;
/* 025 */
/* 026 */   public GeneratedIterator(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(scala.collection.Iterator inputs[]) {
/* 031 */     agg_initAgg = false;
/* 032 */     this.agg_plan = (org.apache.spark.sql.execution.aggregate.TungstenAggregate) references[0];
/* 033 */     agg_hashMap = agg_plan.createHashMap();
/* 034 */
/* 035 */     inputadapter_input = inputs[0];
/* 036 */     agg_result = new UnsafeRow(1);
/* 037 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 038 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 039 */     agg_result1 = new UnsafeRow(2);
/* 040 */     this.agg_holder1 = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result1, 0);
/* 041 */     this.agg_rowWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder1, 2);
/* 042 */     this.wholestagecodegen_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1];
/* 043 */     wholestagecodegen_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) wholestagecodegen_numOutputRows.localValue();
/* 044 */   }
/* 045 */
/* 046 */   private void agg_doAggregateWithKeys() throws java.io.IOException {
/* 047 */     /*** PRODUCE: INPUT */
/* 048 */
/* 049 */     while (inputadapter_input.hasNext()) {
/* 050 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 051 */       /*** CONSUME: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) */
/* 052 */       /* input[0, bigint] */
/* 053 */       long inputadapter_value = inputadapter_row.getLong(0);
/* 054 */       /* input[1, bigint] */
/* 055 */       long inputadapter_value1 = inputadapter_row.getLong(1);
/* 056 */
/* 057 */       // generate grouping key
/* 058 */       agg_rowWriter.write(0, inputadapter_value);
/* 059 */       /* hash(input[0, bigint], 42) */
/* 060 */       int agg_value1 = 42;
/* 061 */
/* 062 */       agg_value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(inputadapter_value, agg_value1);
/* 063 */       UnsafeRow agg_aggBuffer = null;
/* 064 */       if (true) {
/* 065 */         // try to get the buffer from hash map
/* 066 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1);
/* 067 */       }
/* 068 */       if (agg_aggBuffer == null) {
/* 069 */         if (agg_sorter == null) {
/* 070 */           agg_sorter = agg_hashMap.destructAndCreateExternalSorter();
/* 071 */         } else {
/* 072 */           agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter());
/* 073 */         }
/* 074 */
/* 075 */         // the hash map had be spilled, it should have enough memory now,
/* 076 */         // try  to allocate buffer again.
/* 077 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1);
/* 078 */         if (agg_aggBuffer == null) {
/* 079 */           // failed to allocate the first page
/* 080 */           throw new OutOfMemoryError("No enough memory for aggregation");
/* 081 */         }
/* 082 */       }
/* 083 */
/* 084 */       // evaluate aggregate function
/* 085 */       /* (input[0, bigint] + input[2, bigint]) */
/* 086 */       /* input[0, bigint] */
/* 087 */       long agg_value4 = agg_aggBuffer.getLong(0);
/* 088 */
/* 089 */       long agg_value3 = -1L;
/* 090 */       agg_value3 = agg_value4 + inputadapter_value1;
/* 091 */       // update aggregate buffer
/* 092 */       agg_aggBuffer.setLong(0, agg_value3);
/* 093 */       if (shouldStop()) return;
/* 094 */     }
/* 095 */
/* 096 */     agg_mapIter = agg_plan.finishAggregate(agg_hashMap, agg_sorter);
/* 097 */   }
/* 098 */
/* 099 */   protected void processNext() throws java.io.IOException {
/* 100 */     /*** PRODUCE: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) */
/* 101 */
/* 102 */     if (!agg_initAgg) {
/* 103 */       agg_initAgg = true;
/* 104 */       agg_doAggregateWithKeys();
/* 105 */     }
/* 106 */
/* 107 */     // output the result
/* 108 */     while (agg_mapIter.next()) {
/* 109 */       wholestagecodegen_metricValue.add(1);
/* 110 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 111 */       UnsafeRow agg_aggBuffer1 = (UnsafeRow) agg_mapIter.getValue();
/* 112 */
/* 113 */       /* input[0, bigint] */
/* 114 */       long agg_value6 = agg_aggKey.getLong(0);
/* 115 */       /* input[0, bigint] */
/* 116 */       long agg_value7 = agg_aggBuffer1.getLong(0);
/* 117 */
/* 118 */       /*** CONSUME: WholeStageCodegen */
/* 119 */
/* 120 */       agg_rowWriter1.write(0, agg_value6);
/* 121 */
/* 122 */       agg_rowWriter1.write(1, agg_value7);
/* 123 */       append(agg_result1);
/* 124 */
/* 125 */       if (shouldStop()) return;
/* 126 */     }
/* 127 */
/* 128 */     agg_mapIter.close();
/* 129 */     if (agg_sorter == null) {
/* 130 */       agg_hashMap.free();
/* 131 */     }
/* 132 */   }
/* 133 */ }
```

rxin

Author: Eric Liang <ekl@databricks.com>

Closes #12025 from ericl/spark-14227.
2016-03-29 13:31:51 -07:00
Wenchen Fan 38326cad87 [SPARK-14205][SQL] remove trait Queryable
## What changes were proposed in this pull request?

After DataFrame and Dataset are merged, the trait `Queryable` becomes unnecessary as it has only one implementation. We should remove it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12001 from cloud-fan/df-ds.
2016-03-28 18:53:47 -07:00
Herman van Hovell 328c71161b [SPARK-14086][SQL] Add DDL commands to ANTLR4 parser
#### What changes were proposed in this pull request?

This PR adds all the current Spark SQL DDL commands to the new ANTLR 4 based SQL parser.

I have found a few inconsistencies in the current commands:
- Function has an alias field. This is actually the class name of the function.
- Partition specifications should contain nulls in some commands, and contain `None`s in others.
- `AlterTableSkewedLocation`: Should defines which columns have skewed values, and should allow us to define storage for each skewed combination of values. We currently only allow one value per field.
- `AlterTableSetFileFormat`: Should only have one file format, it currently supports both.

I have implemented all these comments like they were, and I propose to improve them in follow-up PRs.

#### How was this patch tested?

The existing DDLCommandSuite.

cc rxin andrewor14 yhuai

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

Closes #12011 from hvanhovell/SPARK-14086.
2016-03-28 16:22:02 -07:00
Davies Liu d7b58f1461 [SPARK-14052] [SQL] build a BytesToBytesMap directly in HashedRelation
## What changes were proposed in this pull request?

Currently, for the key that can not fit within a long,  we build a hash map for UnsafeHashedRelation, it's converted to BytesToBytesMap after serialization and deserialization. We should build a BytesToBytesMap directly to have better memory efficiency.

In order to do that, BytesToBytesMap should support multiple (K,V) pair with the same K,  Location.putNewKey() is renamed to Location.append(), which could append multiple values for the same key (same Location). `Location.newValue()` is added to find the next value for the same key.

## How was this patch tested?

Existing tests. Added benchmark for broadcast hash join with duplicated keys.

Author: Davies Liu <davies@databricks.com>

Closes #11870 from davies/map2.
2016-03-28 13:07:32 -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
gatorsmile a01b6a92b5 [SPARK-14177][SQL] Native Parsing for DDL Command "Describe Database" and "Alter Database"
#### What changes were proposed in this pull request?

This PR is to provide native parsing support for two DDL commands:  ```Describe Database``` and ```Alter Database Set Properties```

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

##### 1. ALTER DATABASE
**Syntax:**
```SQL
ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...)
```
 - `ALTER DATABASE` is to add new (key, value) pairs into `DBPROPERTIES`

##### 2. DESCRIBE DATABASE
**Syntax:**
```SQL
DESCRIBE DATABASE [EXTENDED] db_name
```
 - `DESCRIBE DATABASE` shows the name of the database, its comment (if one has been set), and its root location on the filesystem. When `extended` is true, it also shows the database's properties

#### How was this patch tested?
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>

This patch had conflicts when merged, resolved by
Committer: Yin Huai <yhuai@databricks.com>

Closes #11977 from gatorsmile/parseAlterDatabase.
2016-03-26 20:12:30 -07:00
Liang-Chi Hsieh bc925b73a6 [SPARK-14157][SQL] Parse Drop Function DDL command
## What changes were proposed in this pull request?
JIRA: https://issues.apache.org/jira/browse/SPARK-14157

We only parse create function command. In order to support native drop function command, we need to parse it too.

From Hive [manual](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Create/Drop/ReloadFunction), the drop function command has syntax as:

DROP [TEMPORARY] FUNCTION [IF EXISTS] function_name;

## How was this patch tested?

Added test into `DDLCommandSuite`.

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

Closes #11959 from viirya/parse-drop-func.
2016-03-26 20:09:01 -07:00
gatorsmile 8989d3a396 [SPARK-14161][SQL] Native Parsing for DDL Command Drop Database
### What changes were proposed in this pull request?
Based on the Hive DDL document https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL

The syntax of DDL command for Drop Database is
```SQL
DROP (DATABASE|SCHEMA) [IF EXISTS] database_name [RESTRICT|CASCADE];
```
 - If `IF EXISTS` is not specified, the default behavior is to issue a warning message if `database_name` does't exist
 - `RESTRICT` is the default behavior.

This PR is to provide a native parsing support for `DROP DATABASE`.

#### How was this patch tested?

Added a test case `DDLCommandSuite`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11962 from gatorsmile/parseDropDatabase.
2016-03-26 14:11:13 -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
Tathagata Das 13945dd83b [SPARK-14109][SQL] Fix HDFSMetadataLog to fallback from FileContext to FileSystem API
## What changes were proposed in this pull request?

HDFSMetadataLog uses newer FileContext API to achieve atomic renaming. However, FileContext implementations may not exist for many scheme for which there may be FileSystem implementations. In those cases, rather than failing completely, we should fallback to the FileSystem based implementation, and log warning that there may be file consistency issues in case the log directory is concurrently modified.

In addition I have also added more tests to increase the code coverage.

## How was this patch tested?

Unit test.
Tested on cluster with custom file system.

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

Closes #11925 from tdas/SPARK-14109.
2016-03-25 20:07:54 -07:00
Shixiong Zhu 24587ce433 [SPARK-14073][STREAMING][TEST-MAVEN] Move flume back to Spark
## What changes were proposed in this pull request?

This PR moves flume back to Spark as per the discussion in the dev mail-list.

## How was this patch tested?

Existing Jenkins tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11895 from zsxwing/move-flume-back.
2016-03-25 17:37:16 -07:00
Tathagata Das 11fa8741ca [SQL][HOTFIX] Fix flakiness in StateStoreRDDSuite
## What changes were proposed in this pull request?
StateStoreCoordinator.reportActiveInstance is async, so subsequence state checks must be in eventually.
## How was this patch tested?
Jenkins tests

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

Closes #11924 from tdas/state-store-flaky-fix.
2016-03-25 12:04:47 -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
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 1c70b7650f [SPARK-14145][SQL] Remove the untyped version of Dataset.groupByKey
## What changes were proposed in this pull request?
Dataset has two variants of groupByKey, one for untyped and the other for typed. It actually doesn't make as much sense to have an untyped API here, since apps that want to use untyped APIs should just use the groupBy "DataFrame" API.

## How was this patch tested?
This patch removes a method, and removes the associated tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11949 from rxin/SPARK-14145.
2016-03-24 22:56:34 -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
Michael Armbrust 6bc4be64f8 [SPARK-14078] Streaming Parquet Based FileSink
This PR adds a new `Sink` implementation that writes out Parquet files.  In order to correctly handle partial failures while maintaining exactly once semantics, the files for each batch are written out to a unique directory and then atomically appended to a metadata log.  When a parquet based `DataSource` is initialized for reading, we first check for this log directory and use it instead of file listing when present.

Unit tests are added, as well as a stress test that checks the answer after non-deterministic injected failures.

Author: Michael Armbrust <michael@databricks.com>

Closes #11897 from marmbrus/fileSink.
2016-03-23 13:03:25 -07:00
Tathagata Das 8c826880f5 [SPARK-13809][SQL] State store for streaming aggregations
## What changes were proposed in this pull request?

In this PR, I am implementing a new abstraction for management of streaming state data - State Store. It is a key-value store for persisting running aggregates for aggregate operations in streaming dataframes. The motivation and design is discussed here.

https://docs.google.com/document/d/1-ncawFx8JS5Zyfq1HAEGBx56RDet9wfVp_hDM8ZL254/edit#

## How was this patch tested?
- [x] Unit tests
- [x] Cluster tests

**Coverage from unit tests**

<img width="952" alt="screen shot 2016-03-21 at 3 09 40 pm" src="https://cloud.githubusercontent.com/assets/663212/13935872/fdc8ba86-ef76-11e5-93e8-9fa310472c7b.png">

## TODO
- [x] Fix updates() iterator to avoid duplicate updates for same key
- [x] Use Coordinator in ContinuousQueryManager
- [x] Plugging in hadoop conf and other confs
- [x] Unit tests
  - [x] StateStore object lifecycle and methods
  - [x] StateStoreCoordinator communication and logic
  - [x] StateStoreRDD fault-tolerance
  - [x] StateStoreRDD preferred location using StateStoreCoordinator
- [ ] Cluster tests
  - [ ] Whether preferred locations are set correctly
  - [ ] Whether recovery works correctly with distributed storage
  - [x] Basic performance tests
- [x] Docs

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

Closes #11645 from tdas/state-store.
2016-03-23 12:48:05 -07: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
Cheng Lian cde086cb2a [SPARK-13817][SQL][MINOR] Renames Dataset.newDataFrame to Dataset.ofRows
## What changes were proposed in this pull request?

This PR does the renaming as suggested by marmbrus in [this comment][1].

## How was this patch tested?

Existing tests.

[1]: 6d37e1eb90 (commitcomment-16654694)

Author: Cheng Lian <lian@databricks.com>

Closes #11889 from liancheng/spark-13817-follow-up.
2016-03-24 00:42:13 +08:00
Shixiong Zhu abacf5f258 [HOTFIX][SQL] Don't stop ContinuousQuery in quietly
## What changes were proposed in this pull request?

Try to fix a flaky hang

## How was this patch tested?

Existing Jenkins test

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11909 from zsxwing/hotfix2.
2016-03-23 00:00:35 -07:00
Reynold Xin 926a93e54b [SPARK-14088][SQL] Some Dataset API touch-up
## What changes were proposed in this pull request?
1. Deprecated unionAll. It is pretty confusing to have both "union" and "unionAll" when the two do the same thing in Spark but are different in SQL.
2. Rename reduce in KeyValueGroupedDataset to reduceGroups so it is more consistent with rest of the functions in KeyValueGroupedDataset. Also makes it more obvious what "reduce" and "reduceGroups" mean. Previously it was confusing because it could be reducing a Dataset, or just reducing groups.
3. Added a "name" function, which is more natural to name columns than "as" for non-SQL users.
4. Remove "subtract" function since it is just an alias for "except".

## How was this patch tested?
All changes should be covered by existing tests. Also added couple test cases to cover "name".

Author: Reynold Xin <rxin@databricks.com>

Closes #11908 from rxin/SPARK-14088.
2016-03-22 23:43:09 -07: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
Yong Tang 75dc29620e [SPARK-13401][SQL][TESTS] Fix SQL test warnings.
## What changes were proposed in this pull request?

This fix tries to fix several SQL test warnings under the sql/core/src/test directory. The fixed warnings includes "[unchecked]", "[rawtypes]", and "[varargs]".

## How was this patch tested?

All existing tests passed.

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

Closes #11857 from yongtang/SPARK-13401.
2016-03-22 21:08:11 -07:00
Shixiong Zhu d16710b4c9 [HOTFIX][SQL] Add a timeout for 'cq.stop'
## What changes were proposed in this pull request?

Fix an issue that DataFrameReaderWriterSuite may hang forever.

## How was this patch tested?

Existing tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11902 from zsxwing/hotfix.
2016-03-22 16:41:55 -07:00
Reynold Xin b2b1ad7d4c [SPARK-14060][SQL] Move StringToColumn implicit class into SQLImplicits
## What changes were proposed in this pull request?
This patch moves StringToColumn implicit class into SQLImplicits. This was kept in SQLContext.implicits object for binary backward compatibility, in the Spark 1.x series. It makes more sense for this API to be in SQLImplicits since that's the single class that defines all the SQL implicits.

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

Author: Reynold Xin <rxin@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #11878 from rxin/SPARK-14060.
2016-03-22 13:48:03 -07:00
Reynold Xin 297c20226d [SPARK-14063][SQL] SQLContext.range should return Dataset[java.lang.Long]
## What changes were proposed in this pull request?
This patch changed the return type for SQLContext.range from `Dataset[Long]` (Scala primitive) to `Dataset[java.lang.Long]` (Java boxed long).

Previously, SPARK-13894 changed the return type of range from `Dataset[Row]` to `Dataset[Long]`. The problem is that due to https://issues.scala-lang.org/browse/SI-4388, Scala compiles primitive types in generics into just Object, i.e. range at bytecode level now just returns `Dataset[Object]`. This is really bad for Java users because they are losing type safety and also need to add a type cast every time they use range.

Talked to Jason Zaugg from Lightbend (Typesafe) who suggested the best approach is to return `Dataset[java.lang.Long]`. The downside is that when Scala users want to explicitly type a closure used on the dataset returned by range, they would need to use `java.lang.Long` instead of the Scala `Long`.

## How was this patch tested?
The signature change should be covered by existing unit tests and API tests. I also added a new test case in DatasetSuite for range.

Author: Reynold Xin <rxin@databricks.com>

Closes #11880 from rxin/SPARK-14063.
2016-03-22 11:37:37 -07:00
Michael Armbrust caea152145 [SPARK-13985][SQL] Deterministic batches with ids
This PR relaxes the requirements of a `Sink` for structured streaming to only require idempotent appending of data.  Previously the `Sink` needed to be able to transactionally append data while recording an opaque offset indicated how far in a stream we have processed.

In order to do this, a new write-ahead-log has been added to stream execution, which records the offsets that will are present in each batch.  The log is created in the newly added `checkpointLocation`, which defaults to `${spark.sql.streaming.checkpointLocation}/${queryName}` but can be overriden by setting `checkpointLocation` in `DataFrameWriter`.

In addition to making sinks easier to write the addition of batchIds and a checkpoint location is done in anticipation of integration with the the `StateStore` (#11645).

Author: Michael Armbrust <michael@databricks.com>

Closes #11804 from marmbrus/batchIds.
2016-03-22 10:18:42 -07:00
Sunitha Kambhampati 0ce01635cc [SPARK-13774][SQL] - Improve error message for non-existent paths and add tests
SPARK-13774: IllegalArgumentException: Can not create a Path from an empty string for incorrect file path

**Overview:**
-	If a non-existent path is given in this call
``
scala> sqlContext.read.format("csv").load("file-path-is-incorrect.csv")
``
it throws the following error:
`java.lang.IllegalArgumentException: Can not create a Path from an empty string` …..
`It gets called from inferSchema call in org.apache.spark.sql.execution.datasources.DataSource.resolveRelation`

-	The purpose of this JIRA is to throw a better error message.
-	With the fix, you will now get a _Path does not exist_ error message.
```
scala> sqlContext.read.format("csv").load("file-path-is-incorrect.csv")
org.apache.spark.sql.AnalysisException: Path does not exist: file:/Users/ksunitha/trunk/spark/file-path-is-incorrect.csv;
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:215)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:204)
  ...
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:204)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:131)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:141)
  ... 49 elided
```

**Details**
_Changes include:_
-	Check if path exists or not in resolveRelation in DataSource, and throw an AnalysisException with message like “Path does not exist: $path”
-	AnalysisException is thrown similar to the exceptions thrown in resolveRelation.
-	The glob path and the non glob path is checked with minimal calls to path exists. If the globPath is empty, then it is a nonexistent glob pattern and an error will be thrown. In the scenario that it is not globPath, it is necessary to only check if the first element in the Seq is valid or not.

_Test modifications:_
-	Changes went in for 3 tests to account for this error checking.
-	SQLQuerySuite:test("run sql directly on files") – Error message needed to be updated.
-	2 tests failed in MetastoreDataSourcesSuite because they had a dummy path and so test is modified to give a tempdir and allow it to move past so it can continue to test the codepath it meant to test

_New Tests:_
2 new tests are added to DataFrameSuite to validate that glob and non-glob path will throw the new error message.

_Testing:_
Unit tests were run with the fix.

**Notes/Questions to reviewers:**
-	There is some code duplication in DataSource.scala in resolveRelation method and also createSource with respect to getting the paths.  I have not made any changes to the createSource codepath.  Should we make the change there as well ?

-	From other JIRAs, I know there is restructuring and changes going on in this area, not sure how that will affect these changes, but since this seemed like a starter issue, I looked into it.  If we prefer not to add the overhead of the checks, or if there is a better place to do so, let me know.

I would appreciate your review. Thanks for your time and comments.

Author: Sunitha Kambhampati <skambha@us.ibm.com>

Closes #11775 from skambha/improve_errmsg.
2016-03-22 20:47:57 +08:00
hyukjinkwon 4e09a0d5ea [SPARK-13953][SQL] Specifying the field name for corrupted record via option at JSON datasource
## What changes were proposed in this pull request?

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

Currently, JSON data source creates a new field in `PERMISSIVE` mode for storing malformed string.
This field can be renamed via `spark.sql.columnNameOfCorruptRecord` option but it is a global configuration.

This PR make that option can be applied per read and can be specified via `option()`. This will overwrites `spark.sql.columnNameOfCorruptRecord` if it is set.

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11881 from HyukjinKwon/SPARK-13953.
2016-03-22 20:30:48 +08:00
Michael Armbrust 8014a516d1 [SPARK-13883][SQL] Parquet Implementation of FileFormat.buildReader
This PR add implements the new `buildReader` interface for the Parquet `FileFormat`.  An simple implementation of `FileScanRDD` is also included.

This code should be tested by the many existing tests for parquet.

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

Closes #11709 from marmbrus/parquetReader.
2016-03-21 20:16:01 -07: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
Reynold Xin b3e5af62a1 [SPARK-13898][SQL] Merge DatasetHolder and DataFrameHolder
## What changes were proposed in this pull request?
This patch merges DatasetHolder and DataFrameHolder. This makes more sense because DataFrame/Dataset are now one class.

In addition, fixed some minor issues with pull request #11732.

## How was this patch tested?
Updated existing unit tests that test these implicits.

Author: Reynold Xin <rxin@databricks.com>

Closes #11737 from rxin/SPARK-13898.
2016-03-21 17:17:25 -07:00
Nong Li 5e86e9262f [SPARK-13916][SQL] Add a metric to WholeStageCodegen to measure duration.
## What changes were proposed in this pull request?

WholeStageCodegen naturally breaks the execution into pipelines that are easier to
measure duration. This is more granular than the task timings (a task can be multiple
pipelines) and is integrated with the web ui.

We currently report total time (across all tasks), min/mask/median to get a sense of how long each is taking.

## How was this patch tested?

Manually tested looking at the web ui.

Author: Nong Li <nong@databricks.com>

Closes #11741 from nongli/spark-13916.
2016-03-21 16:56:33 -07:00
Kazuaki Ishizaki f35df7d182 [SPARK-13805] [SQL] Generate code that get a value in each column from ColumnVector when ColumnarBatch is used
## What changes were proposed in this pull request?

This PR generates code that get a value in each column from ```ColumnVector``` instead of creating ```InternalRow``` when ```ColumnarBatch``` is accessed. This PR improves benchmark program by up to 15%.
This PR consists of two parts:

1. Get an ```ColumnVector ``` by using ```ColumnarBatch.column()``` method
2. Get a value of each column by using ```rdd_col${COLIDX}.getInt(ROWIDX)``` instead of ```rdd_row.getInt(COLIDX)```

This is a motivated example.
````
    sqlContext.conf.setConfString(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true")
    sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true")
    val values = 10
    withTempPath { dir =>
      withTempTable("t1", "tempTable") {
        sqlContext.range(values).registerTempTable("t1")
        sqlContext.sql("select id % 2 as p, cast(id as INT) as id from t1")
          .write.partitionBy("p").parquet(dir.getCanonicalPath)
        sqlContext.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable")
        sqlContext.sql("select sum(p) from tempTable").collect
      }
    }
````

The original code
````java
    ...
    /* 072 */       while (!shouldStop() && rdd_batchIdx < numRows) {
    /* 073 */         InternalRow rdd_row = rdd_batch.getRow(rdd_batchIdx++);
    /* 074 */         /*** CONSUME: TungstenAggregate(key=[], functions=[(sum(cast(p#4 as bigint)),mode=Partial,isDistinct=false)], output=[sum#10L]) */
    /* 075 */         /* input[0, int] */
    /* 076 */         boolean rdd_isNull = rdd_row.isNullAt(0);
    /* 077 */         int rdd_value = rdd_isNull ? -1 : (rdd_row.getInt(0));
    ...
````

The code generated by this PR
````java
    /* 072 */       while (!shouldStop() && rdd_batchIdx < numRows) {
    /* 073 */         org.apache.spark.sql.execution.vectorized.ColumnVector rdd_col0 = rdd_batch.column(0);
    /* 074 */         /*** CONSUME: TungstenAggregate(key=[], functions=[(sum(cast(p#4 as bigint)),mode=Partial,isDistinct=false)], output=[sum#10L]) */
    /* 075 */         /* input[0, int] */
    /* 076 */         boolean rdd_isNull = rdd_col0.getIsNull(rdd_batchIdx);
    /* 077 */         int rdd_value = rdd_isNull ? -1 : (rdd_col0.getInt(rdd_batchIdx));
    ...
    /* 128 */         rdd_batchIdx++;
    /* 129 */       }
    /* 130 */       if (shouldStop()) return;

````
Performance
Without this PR
````
model name	: Intel(R) Xeon(R) CPU E5-2667 v2  3.30GHz
Partitioned Table:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
Read data column                          434 /  488         36.3          27.6       1.0X
Read partition column                     302 /  346         52.1          19.2       1.4X
Read both columns                         588 /  643         26.8          37.4       0.7X
````
With this PR
````
model name	: Intel(R) Xeon(R) CPU E5-2667 v2  3.30GHz
Partitioned Table:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
Read data column                          392 /  516         40.1          24.9       1.0X
Read partition column                     256 /  318         61.4          16.3       1.5X
Read both columns                         523 /  539         30.1          33.3       0.7X
````

## How was this patch tested?
Tested by existing test suites and benchmark

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

Closes #11636 from kiszk/SPARK-13805.
2016-03-21 14:36:51 -07:00
Davies Liu 9b4e15ba13 [SPARK-14007] [SQL] Manage the memory used by hash map in shuffled hash join
## What changes were proposed in this pull request?

This PR try acquire the memory for hash map in shuffled hash join, fail the task if there is no enough memory (otherwise it could OOM the executor).

It also removed unused HashedRelation.

## How was this patch tested?

Existing unit tests. Manual tests with TPCDS Q78.

Author: Davies Liu <davies@databricks.com>

Closes #11826 from davies/cleanup_hash2.
2016-03-21 11:21:39 -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
hyukjinkwon e474088144 [SPARK-13764][SQL] Parse modes in JSON data source
## What changes were proposed in this pull request?

Currently, there is no way to control the behaviour when fails to parse corrupt records in JSON data source .

This PR adds the support for parse modes just like CSV data source. There are three modes below:

- `PERMISSIVE` :  When it fails to parse, this sets `null` to to field. This is a default mode when it has been this mode.
- `DROPMALFORMED`: When it fails to parse, this drops the whole record.
- `FAILFAST`: When it fails to parse, it just throws an exception.

This PR also make JSON data source share the `ParseModes` in CSV data source.

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for code style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11756 from HyukjinKwon/SPARK-13764.
2016-03-21 15:42:35 +08:00
Reynold Xin dcaa016610 [SPARK-13897][SQL] RelationalGroupedDataset and KeyValueGroupedDataset
## What changes were proposed in this pull request?
Previously, Dataset.groupBy returns a GroupedData, and Dataset.groupByKey returns a GroupedDataset. The naming is very similar, and unfortunately does not convey the real differences between the two.

Assume we are grouping by some keys (K). groupByKey is a key-value style group by, in which the schema of the returned dataset is a tuple of just two fields: key and value. groupBy, on the other hand, is a relational style group by, in which the schema of the returned dataset is flattened and contain |K| + |V| fields.

This pull request also removes the experimental tag from RelationalGroupedDataset. It has been with DataFrame since 1.3, and we have enough confidence now to stabilize it.

## How was this patch tested?
This is a rename to improve API understandability. Should be covered by all existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11841 from rxin/SPARK-13897.
2016-03-19 11:23:14 -07:00
Reynold Xin 1970d911d9 [SPARK-14018][SQL] Use 64-bit num records in BenchmarkWholeStageCodegen
## What changes were proposed in this pull request?
500L << 20 is actually pretty close to 32-bit int limit. I was trying to increase this to 500L << 23 and got negative numbers instead.

## How was this patch tested?
I'm only modifying test code.

Author: Reynold Xin <rxin@databricks.com>

Closes #11839 from rxin/SPARK-14018.
2016-03-19 00:27:23 -07:00
Sameer Agarwal 54794113a6 [SPARK-13989] [SQL] Remove non-vectorized/unsafe-row parquet record reader
## What changes were proposed in this pull request?

This PR cleans up the new parquet record reader with the following changes:

1. Removes the non-vectorized parquet reader code from `UnsafeRowParquetRecordReader`.
2. Removes the non-vectorized column reader code from `ColumnReader`.
3. Renames `UnsafeRowParquetRecordReader` to `VectorizedParquetRecordReader` and `ColumnReader` to `VectorizedColumnReader`
4. Deprecate `PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED`

## How was this patch tested?

Refactoring only; Existing tests should reveal any problems.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11799 from sameeragarwal/vectorized-parquet.
2016-03-18 14:04:42 -07:00
Davies Liu 9c23c818ca [SPARK-13977] [SQL] Brings back Shuffled hash join
## What changes were proposed in this pull request?

ShuffledHashJoin (also outer join) is removed in 1.6, in favor of SortMergeJoin, which is more robust and also fast.

ShuffledHashJoin is still useful in this case: 1) one table is much smaller than the other one, then cost to build a hash table on smaller table is smaller than sorting the larger table 2) any partition of the small table could fit in memory.

This PR brings back ShuffledHashJoin, basically revert #9645, and fix the conflict. Also merging outer join and left-semi join into the same class. This PR does not implement full outer join, because it's not implemented efficiently (requiring build hash table on both side).

A simple benchmark (one table is 5x smaller than other one) show that ShuffledHashJoin could be 2X faster than SortMergeJoin.

## How was this patch tested?

Added new unit tests for ShuffledHashJoin.

Author: Davies Liu <davies@databricks.com>

Closes #11788 from davies/shuffle_join.
2016-03-18 10:32:53 -07:00
Liang-Chi Hsieh 750ed64cd9 [SPARK-13930] [SQL] Apply fast serialization on collect limit operator
## What changes were proposed in this pull request?

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

Recently the fast serialization has been introduced to collecting DataFrame/Dataset (#11664). The same technology can be used on collect limit operator too.

## How was this patch tested?

Add a benchmark for collect limit to `BenchmarkWholeStageCodegen`.

Without this patch:

    model name      : Westmere E56xx/L56xx/X56xx (Nehalem-C)
    collect limit:                      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    collect limit 1 million                  3413 / 3768          0.3        3255.0       1.0X
    collect limit 2 millions                9728 / 10440          0.1        9277.3       0.4X

With this patch:

    model name      : Westmere E56xx/L56xx/X56xx (Nehalem-C)
    collect limit:                      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    collect limit 1 million                   833 / 1284          1.3         794.4       1.0X
    collect limit 2 millions                 3348 / 4005          0.3        3193.3       0.2X

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

Closes #11759 from viirya/execute-take.
2016-03-17 23:24:44 -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
Josh Rosen de1a84e56e [SPARK-13926] Automatically use Kryo serializer when shuffling RDDs with simple types
Because ClassTags are available when constructing ShuffledRDD we can use them to automatically use Kryo for shuffle serialization when the RDD's types are known to be compatible with Kryo.

This patch introduces `SerializerManager`, a component which picks the "best" serializer for a shuffle given the elements' ClassTags. It will automatically pick a Kryo serializer for ShuffledRDDs whose key, value, and/or combiner types are primitives, arrays of primitives, or strings. In the future we can use this class as a narrow extension point to integrate specialized serializers for other types, such as ByteBuffers.

In a planned followup patch, I will extend the BlockManager APIs so that we're able to use similar automatic serializer selection when caching RDDs (this is a little trickier because the ClassTags need to be threaded through many more places).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11755 from JoshRosen/automatically-pick-best-serializer.
2016-03-16 22:52:55 -07:00
Dongjoon Hyun c890c359b1 [MINOR][SQL][BUILD] Remove duplicated lines
## What changes were proposed in this pull request?

This PR removes three minor duplicated lines. First one is making the following unreachable code warning.
```
JoinSuite.scala:52: unreachable code
[warn]       case j: BroadcastHashJoin => j
```
The other two are just consecutive repetitions in `Seq` of MiMa filters.

## How was this patch tested?

Pass the existing Jenkins test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11773 from dongjoon-hyun/remove_duplicated_line.
2016-03-16 22:48:58 -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
hyukjinkwon 917f4000b4 [SPARK-13719][SQL] Parse JSON rows having an array type and a struct type in the same fieild
## What changes were proposed in this pull request?

This https://github.com/apache/spark/pull/2400 added the support to parse JSON rows wrapped with an array. However, this throws an exception when the given data contains array data and struct data in the same field as below:

```json
{"a": {"b": 1}}
{"a": []}
```

and the schema is given as below:

```scala
val schema =
  StructType(
    StructField("a", StructType(
      StructField("b", StringType) :: Nil
    )) :: Nil)
```

- **Before**

```scala
sqlContext.read.schema(schema).json(path).show()
```

```scala
Exception in thread "main" org.apache.spark.SparkException: Job aborted due to stage failure: Task 7 in stage 0.0 failed 4 times, most recent failure: Lost task 7.3 in stage 0.0 (TID 10, 192.168.1.170): java.lang.ClassCastException: org.apache.spark.sql.types.GenericArrayData cannot be cast to org.apache.spark.sql.catalyst.InternalRow
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getStruct(rows.scala:50)
	at org.apache.spark.sql.catalyst.expressions.GenericMutableRow.getStruct(rows.scala:247)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source)
...
```

- **After**

```scala
sqlContext.read.schema(schema).json(path).show()
```

```bash
+----+
|   a|
+----+
| [1]|
|null|
+----+
```

For other data types, in this case it converts the given values are `null` but only this case emits an exception.

This PR makes the support for wrapped rows applied only at the top level.

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for code style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11752 from HyukjinKwon/SPARK-3308-follow-up.
2016-03-16 18:20:30 -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 b90c0206fa [SPARK-13922][SQL] Filter rows with null attributes in vectorized parquet reader
# What changes were proposed in this pull request?

It's common for many SQL operators to not care about reading `null` values for correctness. Currently, this is achieved by performing `isNotNull` checks (for all relevant columns) on a per-row basis. Pushing these null filters in the vectorized parquet reader should bring considerable benefits (especially for cases when the underlying data doesn't contain any nulls or contains all nulls).

## How was this patch tested?

        Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
        String with Nulls Scan (0%):        Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
        -------------------------------------------------------------------------------------------
        SQL Parquet Vectorized                   1229 / 1648          8.5         117.2       1.0X
        PR Vectorized                             833 /  846         12.6          79.4       1.5X
        PR Vectorized (Null Filtering)            732 /  782         14.3          69.8       1.7X

        Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
        String with Nulls Scan (50%):       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
        -------------------------------------------------------------------------------------------
        SQL Parquet Vectorized                    995 / 1053         10.5          94.9       1.0X
        PR Vectorized                             732 /  772         14.3          69.8       1.4X
        PR Vectorized (Null Filtering)            725 /  790         14.5          69.1       1.4X

        Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
        String with Nulls Scan (95%):       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
        -------------------------------------------------------------------------------------------
        SQL Parquet Vectorized                    326 /  333         32.2          31.1       1.0X
        PR Vectorized                             190 /  200         55.1          18.2       1.7X
        PR Vectorized (Null Filtering)            168 /  172         62.2          16.1       1.9X

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11749 from sameeragarwal/perf-testing.
2016-03-16 16:25:40 -07:00
Cheng Hao d9670f8473 [SPARK-13894][SQL] SqlContext.range return type from DataFrame to DataSet
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13894
Change the return type of the `SQLContext.range` API from `DataFrame` to `Dataset`.

## How was this patch tested?
No additional unit test required.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #11730 from chenghao-intel/range.
2016-03-16 11:20:15 -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
hyukjinkwon 92024797a4 [SPARK-13899][SQL] Produce InternalRow instead of external Row at CSV data source
## What changes were proposed in this pull request?

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

This PR makes CSV data source produce `InternalRow` instead of `Row`.

Basically, this resembles JSON data source. It uses the same codes for casting.

## How was this patch tested?

Unit tests were used within IDE and code style was checked by `./dev/run_tests`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11717 from HyukjinKwon/SPARK-13899.
2016-03-15 23:31:46 -07:00
Davies Liu 421f6c20e8 [SPARK-13917] [SQL] generate broadcast semi join
## What changes were proposed in this pull request?

This PR brings codegen support for broadcast left-semi join.

## How was this patch tested?

Existing tests. Added benchmark, the result show 7X speedup.

Author: Davies Liu <davies@databricks.com>

Closes #11742 from davies/gen_semi.
2016-03-15 22:17:04 -07:00
Davies Liu bbd887f53c [SPARK-13918][SQL] Merge SortMergeJoin and SortMergerOuterJoin
## What changes were proposed in this pull request?

This PR just move some code from SortMergeOuterJoin into SortMergeJoin.

This is for support codegen for outer join.

## How was this patch tested?

existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #11743 from davies/gen_smjouter.
2016-03-15 19:58:49 -07:00
Reynold Xin 643649dcbf [SPARK-13895][SQL] DataFrameReader.text should return Dataset[String]
## What changes were proposed in this pull request?
This patch changes DataFrameReader.text()'s return type from DataFrame to Dataset[String].

Closes #11731.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #11739 from rxin/SPARK-13895.
2016-03-15 14:57:54 -07:00
Stavros Kontopoulos 50e3644d00 [SPARK-13896][SQL][STRING] Dataset.toJSON should return Dataset
## What changes were proposed in this pull request?
Change the return type of toJson in Dataset class
## How was this patch tested?
No additional unit test required.

Author: Stavros Kontopoulos <stavros.kontopoulos@typesafe.com>

Closes #11732 from skonto/fix_toJson.
2016-03-15 12:18:30 -07:00
Reynold Xin 5e6f2f4563 [SPARK-13893][SQL] Remove SQLContext.catalog/analyzer (internal method)
## What changes were proposed in this pull request?
Our internal code can go through SessionState.catalog and SessionState.analyzer. This brings two small benefits:
1. Reduces internal dependency on SQLContext.
2. Removes 2 public methods in Java (Java does not obey package private visibility).

More importantly, according to the design in SPARK-13485, we'd need to claim this catalog function for the user-facing public functions, rather than having an internal field.

## How was this patch tested?
Existing unit/integration test code.

Author: Reynold Xin <rxin@databricks.com>

Closes #11716 from rxin/SPARK-13893.
2016-03-15 10:12:32 -07:00
Xin Ren 10251a7457 [SPARK-13660][SQL][TESTS] ContinuousQuerySuite floods the logs with garbage
## What changes were proposed in this pull request?

Use method 'testQuietly' to avoid ContinuousQuerySuite flooding the console logs with garbage

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

## How was this patch tested?

Just check Jenkins output.

Author: Xin Ren <iamshrek@126.com>

Closes #11703 from keypointt/SPARK-13660.
2016-03-15 01:02:28 -07:00
Reynold Xin 276c2d51a3 [SPARK-13890][SQL] Remove some internal classes' dependency on SQLContext
## What changes were proposed in this pull request?
In general it is better for internal classes to not depend on the external class (in this case SQLContext) to reduce coupling between user-facing APIs and the internal implementations. This patch removes SQLContext dependency from some internal classes such as SparkPlanner, SparkOptimizer.

As part of this patch, I also removed the following internal methods from SQLContext:
```
protected[sql] def functionRegistry: FunctionRegistry
protected[sql] def optimizer: Optimizer
protected[sql] def sqlParser: ParserInterface
protected[sql] def planner: SparkPlanner
protected[sql] def continuousQueryManager
protected[sql] def prepareForExecution: RuleExecutor[SparkPlan]
```

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

Author: Reynold Xin <rxin@databricks.com>

Closes #11712 from rxin/sqlContext-planner.
2016-03-14 23:58:57 -07:00
Dongjoon Hyun a51f877b5d [SPARK-13870][SQL] Add scalastyle escaping correctly in CVSSuite.scala
## What changes were proposed in this pull request?

When initial creating `CVSSuite.scala` in SPARK-12833, there was a typo on `scalastyle:on`: `scalstyle:on`. So, it turns off ScalaStyle checking for the rest of the file mistakenly. So, it can not find a violation on the code of `SPARK-12668` added recently. This issue fixes the existing escaping correctly and adds a new escaping for `SPARK-12668` code like the following.

```scala
   test("test aliases sep and encoding for delimiter and charset") {
+    // scalastyle:off
     val cars = sqlContext
...
       .load(testFile(carsFile8859))
+    // scalastyle:on
```
This will prevent future potential problems, too.

## How was this patch tested?

Pass the Jenkins test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11700 from dongjoon-hyun/SPARK-13870.
2016-03-14 23:23:05 -07:00
Davies Liu f72743d971 [SPARK-13353][SQL] fast serialization for collecting DataFrame/Dataset
## What changes were proposed in this pull request?

When we call DataFrame/Dataset.collect(), Java serializer (or Kryo Serializer) will be used to serialize the UnsafeRows in executor, then deserialize them into UnsafeRows in driver. Java serializer (and Kyro serializer) are slow on millions rows, because they try to find out the same rows, but usually there is no same rows.

This PR will serialize the UnsafeRows as byte array by packing them together, then Java serializer (or Kyro serializer) serialize the bytes very fast (there are fewer blocks and byte array are not compared by content).

The UnsafeRow format is highly compressible, the serialized bytes are also compressed (configurable by spark.io.compression.codec).

## How was this patch tested?

Existing unit tests.

Add a benchmark for collect, before this patch:
```
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
collect:                        Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
collect 1 million                      3991 / 4311          0.3        3805.7       1.0X
collect 2 millions                  10083 / 10637          0.1        9616.0       0.4X
collect 4 millions                  29551 / 30072          0.0       28182.3       0.1X
```

```
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
collect:                        Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
collect 1 million                        775 / 1170          1.4         738.9       1.0X
collect 2 millions                     1153 / 1758          0.9        1099.3       0.7X
collect 4 millions                     4451 / 5124          0.2        4244.9       0.2X
```

We can see about 5-7X speedup.

Author: Davies Liu <davies@databricks.com>

Closes #11664 from davies/serialize_row.
2016-03-14 22:32:22 -07:00
Shixiong Zhu b5e3bd87f5 [SPARK-13791][SQL] Add MetadataLog and HDFSMetadataLog
## What changes were proposed in this pull request?

- Add a MetadataLog interface for  metadata reliably storage.
- Add HDFSMetadataLog as a MetadataLog implementation based on HDFS.
- Update FileStreamSource to use HDFSMetadataLog instead of managing metadata by itself.

## How was this patch tested?

unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11625 from zsxwing/metadata-log.
2016-03-14 19:28:13 -07:00
Reynold Xin 4bf4609795 [SPARK-13882][SQL] Remove org.apache.spark.sql.execution.local
## What changes were proposed in this pull request?
We introduced some local operators in org.apache.spark.sql.execution.local package but never fully wired the engine to actually use these. We still plan to implement a full local mode, but it's probably going to be fairly different from what the current iterator-based local mode would look like. Based on what we know right now, we might want a push-based columnar version of these operators.

Let's just remove them for now, and we can always re-introduced them in the future by looking at branch-1.6.

## How was this patch tested?
This is simply dead code removal.

Author: Reynold Xin <rxin@databricks.com>

Closes #11705 from rxin/SPARK-13882.
2016-03-14 19:22:11 -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
Andrew Or 9a1680c2c8 [SPARK-13139][SQL] Follow-ups to #11573
Addressing outstanding comments in #11573.

Jenkins, new test case in `DDLCommandSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #11667 from andrewor14/ddl-parser-followups.
2016-03-14 09:59:22 -07:00
Yin Huai 250832c733 [SPARK-13207][SQL] Make partitioning discovery ignore _SUCCESS files.
If a _SUCCESS appears in the inner partitioning dir, partition discovery will treat that _SUCCESS file as a data file. Then, partition discovery will fail because it finds that the dir structure is not valid. We should ignore those `_SUCCESS` files.

In future, it is better to ignore all files/dirs starting with `_` or `.`. This PR does not make this change. I am thinking about making this change simple, so we can consider of getting it in branch 1.6.

To ignore all files/dirs starting with `_` or `, the main change is to let ParquetRelation have another way to get metadata files. Right now, it relies on FileStatusCache's cachedLeafStatuses, which returns file statuses of both metadata files (e.g. metadata files used by parquet) and data files, which requires more changes.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #11088 from yhuai/SPARK-13207.
2016-03-14 09:03:13 -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
Cheng Lian c079420d7c [SPARK-13841][SQL] Removes Dataset.collectRows()/takeRows()
## What changes were proposed in this pull request?

This PR removes two methods, `collectRows()` and `takeRows()`, from `Dataset[T]`. These methods were added in PR #11443, and were later considered not useful.

## How was this patch tested?

Existing tests should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #11678 from liancheng/remove-collect-rows-and-take-rows.
2016-03-13 12:02:52 +08:00
Cheng Lian 4eace4d384 [SPARK-13828][SQL] Bring back stack trace of AnalysisException thrown from QueryExecution.assertAnalyzed
PR #11443 added an extra `plan: Option[LogicalPlan]` argument to `AnalysisException` and attached partially analyzed plan to thrown `AnalysisException` in `QueryExecution.assertAnalyzed()`.  However, the original stack trace wasn't properly inherited.  This PR fixes this issue by inheriting the stack trace.

A test case is added to verify that the first entry of `AnalysisException` stack trace isn't from `QueryExecution`.

Author: Cheng Lian <lian@databricks.com>

Closes #11677 from liancheng/analysis-exception-stacktrace.
2016-03-12 11:25:15 -08: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
Cheng Lian 6d37e1eb90 [SPARK-13817][BUILD][SQL] Re-enable MiMA and removes object DataFrame
## What changes were proposed in this pull request?

PR #11443 temporarily disabled MiMA check, this PR re-enables it.

One extra change is that `object DataFrame` is also removed. The only purpose of introducing `object DataFrame` was to use it as an internal factory for creating `Dataset[Row]`. By replacing this internal factory with `Dataset.newDataFrame`, both `DataFrame` and `DataFrame$` are entirely removed from the API, so that we can simply put a `MissingClassProblem` filter in `MimaExcludes.scala` for most DataFrame API  changes.

## How was this patch tested?

Tested by MiMA check triggered by Jenkins.

Author: Cheng Lian <lian@databricks.com>

Closes #11656 from liancheng/re-enable-mima.
2016-03-11 22:17:50 +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
Tathagata Das 3d2b6f56e3 [SQL][TEST] Increased timeouts to reduce flakiness in ContinuousQueryManagerSuite
## What changes were proposed in this pull request?

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

## How was this patch tested?

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

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

Closes #11638 from tdas/cqm-flaky-test.
2016-03-10 14:38:19 -08:00
Reynold Xin 8a3acb792d [SPARK-13794][SQL] Rename DataFrameWriter.stream() DataFrameWriter.startStream()
## What changes were proposed in this pull request?
The new name makes it more obvious with the verb "start" that we are actually starting some execution.

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

Author: Reynold Xin <rxin@databricks.com>

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

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

**Before**

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

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

- ORC
```
.orc
```

**After**

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

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

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

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

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

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11604 from HyukjinKwon/SPARK-13766.
2016-03-09 19:12:46 -08:00
Andrew Or 37fcda3e6c [SPARK-13747][SQL] Fix concurrent query with fork-join pool
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

New test in `SQLExecutionSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #11586 from andrewor14/fix-concurrent-sql.
2016-03-09 17:34:28 -08:00
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
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
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
Dongjoon Hyun f3201aeeb0 [SPARK-13692][CORE][SQL] Fix trivial Coverity/Checkstyle defects
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

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

Author: Dongjoon Hyun <dongjoon@apache.org>

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

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

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

Author: Jakob Odersky <jodersky@gmail.com>

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

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

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

Author: Hossein <hossein@databricks.com>

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

This PR fix the sizeInBytes of HadoopFsRelation.

## How was this patch tested?

Added regression test for that.

Author: Davies Liu <davies@databricks.com>

Closes #11590 from davies/fix_sizeInBytes.
2016-03-08 17:42:52 -08:00
Sameer Agarwal e430614eae [SPARK-13668][SQL] Reorder filter/join predicates to short-circuit isNotNull checks
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

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

Author: Sameer Agarwal <sameer@databricks.com>

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

## How was the this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11315 from HyukjinKwon/SPARK-13442.
2016-03-07 14:32:01 -08:00
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
Nong Li a6e2bd31f5 [SPARK-13255] [SQL] Update vectorized reader to directly return ColumnarBatch instead of InternalRows.
## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

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

## How was this patch tested?

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

Author: Nong Li <nong@databricks.com>

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

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

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

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

Closes #11489 from thomastechs/thomastechs-12941-master-new.
2016-03-03 20:35:40 -08:00
Davies Liu b373a88862 [SPARK-13415][SQL] Visualize subquery in SQL web UI
## What changes were proposed in this pull request?

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

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

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

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

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

The web UI looks like:

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

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

## How was this patch tested?

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

Author: Davies Liu <davies@databricks.com>

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

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

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

## How was this patch tested?

Just check Jenkins output.

Author: Shixiong Zhu <shixiong@databricks.com>

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

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

## How was this patch tested?

Jenkins.

Author: Andrew Or <andrew@databricks.com>

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

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

## How was this patch tested?

manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

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

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

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11464 from HyukjinKwon/SPARK-13543.
2016-03-03 10:30:55 -08:00
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
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
Takeshi YAMAMURO 6250cf1e00 [SPARK-13528][SQL] Make the short names of compression codecs consistent in ParquetRelation
## What changes were proposed in this pull request?
This pr to make the short names of compression codecs in `ParquetRelation` consistent against other ones. This pr comes from #11324.

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

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

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

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

Author: Nong Li <nong@databricks.com>

Closes #11454 from nongli/benchmark.
2016-03-02 15:03:19 -08:00
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
sureshthalamati e42724b12b [SPARK-13167][SQL] Include rows with null values for partition column when reading from JDBC datasources.
Rows with null values in partition column are not included in the results because none of the partition
where clause specify is null predicate on the partition column. This fix adds is null predicate on the partition column  to the first JDBC partition where clause.

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

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

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

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

## How was this patch tested?

Updated unit tests.

Author: Davies Liu <davies@databricks.com>

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

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

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

## How was this patch tested?

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

Author: Davies Liu <davies@databricks.com>

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

## What changes were proposed in this pull request?

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

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

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

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

After add wholestage codegen support:

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

## How was this patch tested?

A test is added into BenchmarkWholeStageCodegen.

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

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

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

## How was this patch tested?

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

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

Closes #11359 from sameeragarwal/sort-codegen.
2016-02-29 12:59:46 -08:00
hyukjinkwon 02aa499dfb [SPARK-13509][SPARK-13507][SQL] Support for writing CSV with a single function call
https://issues.apache.org/jira/browse/SPARK-13507
https://issues.apache.org/jira/browse/SPARK-13509

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

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

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

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

Closes #11389 from HyukjinKwon/SPARK-13507-13509.
2016-02-29 09:44:29 -08:00
Cheng Lian 916fc34f98 [SPARK-13540][SQL] Supports using nested classes within Scala objects as Dataset element type
## What changes were proposed in this pull request?

Nested classes defined within Scala objects are translated into Java static nested classes. Unlike inner classes, they don't need outer scopes. But the analyzer still thinks that an outer scope is required.

This PR fixes this issue simply by checking whether a nested class is static before looking up its outer scope.

## How was this patch tested?

A test case is added to `DatasetSuite`. It checks contents of a Dataset whose element type is a nested class declared in a Scala object.

Author: Cheng Lian <lian@databricks.com>

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

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

Closes #11194 from tanwanirahul/master.
2016-02-28 23:16:34 -08:00
Andrew Or cca79fad66 [SPARK-13526][SQL] Move SQLContext per-session states to new class
## What changes were proposed in this pull request?

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

## How was this patch tested?

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

Author: Andrew Or <andrew@databricks.com>

Closes #11405 from andrewor14/refactor-session.
2016-02-27 19:51:28 -08:00
Nong Li 0598a2b81d [SPARK-13499] [SQL] Performance improvements for parquet reader.
## What changes were proposed in this pull request?

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

## How was this patch tested?

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

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

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

Author: Nong Li <nong@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

## How was the this patch tested?

Added unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11328 from davies/nested_loop.
2016-02-26 09:58:05 -08:00
Cheng Lian 99dfcedbfd [SPARK-13457][SQL] Removes DataFrame RDD operations
## What changes were proposed in this pull request?

This is another try of PR #11323.

This PR removes DataFrame RDD operations except for `foreach` and `foreachPartitions` (they are actions rather than transformations). Original calls are now replaced by calls to methods of `DataFrame.rdd`.

PR #11323 was reverted because it introduced a regression: both `DataFrame.foreach` and `DataFrame.foreachPartitions` wrap underlying RDD operations with `withNewExecutionId` to track Spark jobs. But they are removed in #11323.

## How was the this patch tested?

No extra tests are added. Existing tests should do the work.

Author: Cheng Lian <lian@databricks.com>

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

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

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

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

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

## Changes

### BlockInfoManager and reader/writer locks

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

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

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

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

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

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

### Locking and the MemoryStore

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

### Locking and remote block transfer

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

## FAQ

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

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

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

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10705 from JoshRosen/pin-pages.
2016-02-25 17:17:56 -08:00
Davies Liu 751724b132 Revert "[SPARK-13457][SQL] Removes DataFrame RDD operations"
This reverts commit 157fe64f3e.
2016-02-25 11:53:48 -08:00
Cheng Lian 157fe64f3e [SPARK-13457][SQL] Removes DataFrame RDD operations
## What changes were proposed in this pull request?

This PR removes DataFrame RDD operations. Original calls are now replaced by calls to methods of `DataFrame.rdd`.

## How was the this patch tested?

No extra tests are added. Existing tests should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #11323 from liancheng/remove-df-rdd-ops.
2016-02-25 23:07:59 +08:00
Reynold Xin 2b2c8c3323 [SPARK-13486][SQL] Move SQLConf into an internal package
## What changes were proposed in this pull request?
This patch moves SQLConf into org.apache.spark.sql.internal package to make it very explicit that it is internal. Soon I will also submit more API work that creates implementations of interfaces in this internal package.

## How was this patch tested?
If it compiles, then the refactoring should work.

Author: Reynold Xin <rxin@databricks.com>

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

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

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

## How was this patch tested?

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

Author: Davies Liu <davies@databricks.com>

Closes #11354 from davies/fix_column_pruning.
2016-02-25 00:13:07 -08:00
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
Nong Li 5a7af9e7ac [SPARK-13250] [SQL] Update PhysicallRDD to convert to UnsafeRow if using the vectorized scanner.
Some parts of the engine rely on UnsafeRow which the vectorized parquet scanner does not want
to produce. This add a conversion in Physical RDD. In the case where codegen is used (and the
scan is the start of the pipeline), there is no requirement to use UnsafeRow. This patch adds
update PhysicallRDD to support codegen, which eliminates the need for the UnsafeRow conversion
in all cases.

The result of these changes for TPCDS-Q19 at the 10gb sf reduces the query time from 9.5 seconds
to 6.5 seconds.

Author: Nong Li <nong@databricks.com>

Closes #11141 from nongli/spark-13250.
2016-02-24 17:16:45 -08:00
Timothy Hunter 15e3015563 [SPARK-6761][SQL][ML] Fixes to API and documentation of approximate quantiles
## What changes were proposed in this pull request?

This continues  thunterdb 's work on `approxQuantile` API. It changes the signature of `approxQuantile` from `(col: String, quantile: Double, epsilon: Double): Double`  to `(col: String, probabilities: Array[Double], relativeError: Double): Array[Double]` and update API doc. It also improves the error message in tests and simplifies the merge algorithm for summaries.

## How was the this patch tested?

Use the same unit tests as before.

Closes #11325

Author: Timothy Hunter <timhunter@databricks.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #11332 from mengxr/SPARK-6761.
2016-02-23 15:31:17 -08:00
Davies Liu 9cdd867da9 [SPARK-13373] [SQL] generate sort merge join
## What changes were proposed in this pull request?

Generates code for SortMergeJoin.

## How was the this patch tested?

Unit tests and manually tested with TPCDS Q72, which showed 70% performance improvements (from 42s to 25s), but micro benchmark only show minor improvements, it may depends the distribution of data and number of columns.

Author: Davies Liu <davies@databricks.com>

Closes #11248 from davies/gen_smj.
2016-02-23 15:00:10 -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
Timothy Hunter 4fd1993692 [SPARK-6761][SQL] Approximate quantile for DataFrame
JIRA: https://issues.apache.org/jira/browse/SPARK-6761

Compute approximate quantile based on the paper Greenwald, Michael and Khanna, Sanjeev, "Space-efficient Online Computation of Quantile Summaries," SIGMOD '01.

Author: Timothy Hunter <timhunter@databricks.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #6042 from viirya/approximate_quantile.
2016-02-22 23:31:00 -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
hyukjinkwon 819b0ea029 [SPARK-13381][SQL] Support for loading CSV with a single function call
https://issues.apache.org/jira/browse/SPARK-13381

This PR adds the support to load CSV data directly by a single call with given paths.

Also, I corrected this to refer all paths rather than the first path in schema inference, which JSON datasource dose.

Several unitests were added for each functionality.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11262 from HyukjinKwon/SPARK-13381.
2016-02-21 19:11:03 -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
Shixiong Zhu 76bd98d914 [SPARK-13405][STREAMING][TESTS] Make sure no messages leak to the next test
## What changes were proposed in this pull request?

Fixed the test failure `org.apache.spark.sql.util.ContinuousQueryListenerSuite.event ordering`: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/202/testReport/junit/org.apache.spark.sql.util/ContinuousQueryListenerSuite/event_ordering/

```
      org.scalatest.exceptions.TestFailedException:
Assert failed: : null equaled null onQueryTerminated called before onQueryStarted
org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
	org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
	org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)
	org.apache.spark.sql.util.ContinuousQueryListenerSuite$QueryStatusCollector$$anonfun$onQueryTerminated$1.apply$mcV$sp(ContinuousQueryListenerSuite.scala:204)
	org.scalatest.concurrent.AsyncAssertions$Waiter.apply(AsyncAssertions.scala:349)
	org.apache.spark.sql.util.ContinuousQueryListenerSuite$QueryStatusCollector.onQueryTerminated(ContinuousQueryListenerSuite.scala:203)
	org.apache.spark.sql.execution.streaming.ContinuousQueryListenerBus.doPostEvent(ContinuousQueryListenerBus.scala:67)
	org.apache.spark.sql.execution.streaming.ContinuousQueryListenerBus.doPostEvent(ContinuousQueryListenerBus.scala:32)
	org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:63)
	org.apache.spark.sql.execution.streaming.ContinuousQueryListenerBus.postToAll(ContinuousQueryListenerBus.scala:32)
```

In the previous codes, when the test `adding and removing listener` finishes, there may be still some QueryTerminated events in the listener bus queue. Then when `event ordering` starts to run, it may see these events and throw the above exception.

This PR just added `waitUntilEmpty` in `after` to make sure all events be consumed after each test.

## How was the this patch tested?

Jenkins tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11275 from zsxwing/SPARK-13405.
2016-02-21 15:32:49 -08:00
hyukjinkwon 7eb83fefd1 [SPARK-13137][SQL] NullPoingException in schema inference for CSV when the first line is empty
https://issues.apache.org/jira/browse/SPARK-13137

This PR adds a filter in schema inference so that it does not emit NullPointException.

Also, I removed `MAX_COMMENT_LINES_IN_HEADER `but instead used a monad chaining with `filter()` and `first()`.

Lastly, I simply added a newline rather than adding a new file for this so that this is covered with the original tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11023 from HyukjinKwon/SPARK-13137.
2016-02-21 13:21:59 -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
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
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
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
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
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 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
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
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
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
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
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
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