Commit graph

1962 commits

Author SHA1 Message Date
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
Rahul Tanwani dd3b5455c6 [SPARK-13309][SQL] Fix type inference issue with CSV data
Fix type inference issue for sparse CSV data - https://issues.apache.org/jira/browse/SPARK-13309

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

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

## What changes were proposed in this pull request?

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

## How was this patch tested?

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

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

The error exception:

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

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

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

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

## How was this patch tested?

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

Author: Andrew Or <andrew@databricks.com>

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

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

## How was this patch tested?

Ran ParquetHadoopFsRelationSuite which failed before this.

Author: Nong Li <nong@databricks.com>

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

## What changes were proposed in this pull request?

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

The error exception:

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

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

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

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

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

## How was this patch tested?

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

Author: Nong Li <nong@databricks.com>

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

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

## How was this patch tested?

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

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

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

Author: Nong Li <nong@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

## How was the this patch tested?

Added unit tests.

Author: Davies Liu <davies@databricks.com>

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

This is another try of PR #11323.

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

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

## How was the this patch tested?

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

Author: Cheng Lian <lian@databricks.com>

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

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

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

## How was the this patch tested?

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

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

Closes #11306 from thomastechs/master.
2016-02-25 22:52:25 -08:00
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
Joseph K. Bradley 13ce10e954 [SPARK-13479][SQL][PYTHON] Added Python API for approxQuantile
## What changes were proposed in this pull request?

* Scala DataFrameStatFunctions: Added version of approxQuantile taking a List instead of an Array, for Python compatbility
* Python DataFrame and DataFrameStatFunctions: Added approxQuantile

## How was this patch tested?

* unit test in sql/tests.py

Documentation was copied from the existing approxQuantile exactly.

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #11356 from jkbradley/approx-quantile-python.
2016-02-24 23:15:36 -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
Wenchen Fan a60f91284c [SPARK-13467] [PYSPARK] abstract python function to simplify pyspark code
## What changes were proposed in this pull request?

When we pass a Python function to JVM side, we also need to send its context, e.g. `envVars`, `pythonIncludes`, `pythonExec`, etc. However, it's annoying to pass around so many parameters at many places. This PR abstract python function along with its context, to simplify some pyspark code and make the logic more clear.

## How was the this patch tested?

by existing unit tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11342 from cloud-fan/python-clean.
2016-02-24 12:44:54 -08:00
Reynold Xin 65805ab6ea Revert "Revert "[SPARK-13383][SQL] Keep broadcast hint after column pruning""
This reverts commit 382b27babf.
2016-02-24 12:03:45 -08:00
Reynold Xin d563c8fa01 Revert "[SPARK-13376] [SQL] improve column pruning"
This reverts commit e9533b419e.
2016-02-24 11:58:32 -08:00
Reynold Xin 382b27babf Revert "[SPARK-13383][SQL] Keep broadcast hint after column pruning"
This reverts commit f373986997.
2016-02-24 11:58:12 -08:00
Liang-Chi Hsieh f373986997 [SPARK-13383][SQL] Keep broadcast hint after column pruning
JIRA: https://issues.apache.org/jira/browse/SPARK-13383

## What changes were proposed in this pull request?

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

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

## How was the this patch tested?

Unit test is added.

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

Closes #11260 from viirya/keep-broadcasthint.
2016-02-24 10:22:40 -08:00
Davies Liu e9533b419e [SPARK-13376] [SQL] improve column pruning
## What changes were proposed in this pull request?

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

## How was the this patch tested?

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

Author: Davies Liu <davies@databricks.com>

Closes #11256 from davies/fix_column_pruning.
2016-02-23 18:19:22 -08:00
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
gatorsmile 87250580f2 [SPARK-13263][SQL] SQL Generation Support for Tablesample
In the parser, tableSample clause is part of tableSource.
```
tableSource
init { gParent.pushMsg("table source", state); }
after { gParent.popMsg(state); }
    : tabname=tableName
    ((tableProperties) => props=tableProperties)?
    ((tableSample) => ts=tableSample)?
    ((KW_AS) => (KW_AS alias=Identifier)
    |
    (Identifier) => (alias=Identifier))?
    -> ^(TOK_TABREF $tabname $props? $ts? $alias?)
    ;
```

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

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

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

Thanks! cc liancheng

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

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

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

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

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

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

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

cc liancheng

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11314 from gatorsmile/expressionToSQL.
2016-02-22 22:17:56 -08:00
Xiu Guo 2063781840 [SPARK-13422][SQL] Use HashedRelation instead of HashSet in Left Semi Joins
Use the HashedRelation which is a more optimized datastructure and reduce code complexity

Author: Xiu Guo <xguo27@gmail.com>

Closes #11291 from xguo27/SPARK-13422.
2016-02-22 16:34:02 -08:00
Michael Armbrust 173aa949c3 [SPARK-12546][SQL] Change default number of open parquet files
A common problem that users encounter with Spark 1.6.0 is that writing to a partitioned parquet table OOMs.  The root cause is that parquet allocates a significant amount of memory that is not accounted for by our own mechanisms.  As a workaround, we can ensure that only a single file is open per task unless the user explicitly asks for more.

Author: Michael Armbrust <michael@databricks.com>

Closes #11308 from marmbrus/parquetWriteOOM.
2016-02-22 15:27:29 -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 0cbadf28c9 [SPARK-13271][SQL] Better error message if 'path' is not specified
Improved the error message as per discussion in https://github.com/apache/spark/pull/11034#discussion_r52111238. Also made `path` and `metadataPath` in FileStreamSource case insensitive.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11154 from zsxwing/path.
2016-02-21 15:34:39 -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
Reynold Xin 0947f0989b [SPARK-13420][SQL] Rename Subquery logical plan to SubqueryAlias
## What changes were proposed in this pull request?
This patch renames logical.Subquery to logical.SubqueryAlias, which is a more appropriate name for this operator (versus subqueries as expressions).

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

Author: Reynold Xin <rxin@databricks.com>

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

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

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

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

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

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

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

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

Author: Cheng Lian <lian@databricks.com>

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

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

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

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

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

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

Author: Davies Liu <davies@databricks.com>

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

Closes #10527 from vectorijk/spark-12567.
2016-02-19 22:28:47 -08:00
Hossein 14844118b5 [SPARK-13261][SQL] Expose maxCharactersPerColumn as a user configurable option
This patch expose `maxCharactersPerColumn` and `maxColumns` to user in CSV data source.

Author: Hossein <hossein@databricks.com>

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

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

Author: Brandon Bradley <bradleytastic@gmail.com>

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11232 from gatorsmile/randSeed.
2016-02-18 21:19:36 -08:00
Davies Liu 95e1ab223e [SPARK-13237] [SQL] generated broadcast outer join
This PR support codegen for broadcast outer join.

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

Author: Davies Liu <davies@databricks.com>

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

Scaladoc compile error is fixed.

Author: jerryshao <sshao@hortonworks.com>

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

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

Closes #11230 from ueshin/issues/SPARK-13357.
2016-02-17 00:21:15 -08:00
Takuya UESHIN 19dc69de79 [SPARK-12976][SQL] Add LazilyGenerateOrdering and use it for RangePartitioner of Exchange.
Add `LazilyGenerateOrdering` to support generated ordering for `RangePartitioner` of `Exchange` instead of `InterpretedOrdering`.

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

Closes #10894 from ueshin/issues/SPARK-12976.
2016-02-16 10:54:44 -08:00
Josh Rosen a8bbc4f50e [SPARK-12503][SPARK-12505] Limit pushdown in UNION ALL and OUTER JOIN
This patch adds a new optimizer rule for performing limit pushdown. Limits will now be pushed down in two cases:

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

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

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11121 from JoshRosen/limit-pushdown-2.
2016-02-14 17:32:21 -08:00
Reynold Xin 354d4c24be [SPARK-13296][SQL] Move UserDefinedFunction into sql.expressions.
This pull request has the following changes:

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

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

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

Most of the diffs are just straight copy-paste.

Author: Reynold Xin <rxin@databricks.com>

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

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

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

Author: Davies Liu <davies@databricks.com>

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

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

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

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

Author: Davies Liu <davies@databricks.com>

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

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

Closes #11051 from jayadevanmurali/branch-0.2-SPARK-12982.
2016-02-11 21:21:03 +01:00
Nong Li 18bcbbdd84 [SPARK-13270][SQL] Remove extra new lines in whole stage codegen and include pipeline plan in comments.
Author: Nong Li <nong@databricks.com>

Closes #11155 from nongli/spark-13270.
2016-02-10 23:52:19 -08:00
Davies Liu 8f744fe3d9 [SPARK-13234] [SQL] remove duplicated SQL metrics
For lots of SQL operators, we have metrics for both of input and output, the number of input rows should be exactly the number of output rows of child, we could only have metrics for output rows.

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

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

The new SQL UI will looks like:

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

Author: Davies Liu <davies@databricks.com>

Closes #11163 from davies/remove_metrics.
2016-02-10 23:23:01 -08:00
Davies Liu b5761d150b [SPARK-12706] [SQL] grouping() and grouping_id()
Grouping() returns a column is aggregated or not, grouping_id() returns the aggregation levels.

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

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

Author: Davies Liu <davies@databricks.com>

Closes #10677 from davies/grouping.
2016-02-10 20:13:38 -08:00
gatorsmile 0f09f02269 [SPARK-13205][SQL] SQL Generation Support for Self Join
This PR addresses two issues:
  - Self join does not work in SQL Generation
  - When creating new instances for `LogicalRelation`, `metastoreTableIdentifier` is lost.

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

Author: gatorsmile <gatorsmile@gmail.com>

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

Author: raela <raela@databricks.com>

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

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

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

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

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

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

Closes #11030 from tdas/streaming-df-management-api.
2016-02-10 16:45:06 -08:00
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
Josh Rosen 06f0df6df2 [SPARK-8964] [SQL] Use Exchange to perform shuffle in Limit
This patch changes the implementation of the physical `Limit` operator so that it relies on the `Exchange` operator to perform data movement rather than directly using `ShuffledRDD`. In addition to improving efficiency, this lays the necessary groundwork for further optimization of limit, such as limit pushdown or whole-stage codegen.

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

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

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7334 from JoshRosen/remove-copy-in-limit.
2016-02-08 11:38:21 -08:00
Tommy YU 81da3bee66 [SPARK-5865][API DOC] Add doc warnings for methods that return local data structures
rxin srowen
I work out note message for rdd.take function, please help to review.

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

Author: Tommy YU <tummyyu@163.com>

Closes #10874 from Wenpei/spark-5865-add-warning-for-localdatastructure.
2016-02-06 17:29:09 +00:00
Jakob Odersky 6883a5120c [SPARK-13171][CORE] Replace future calls with Future
Trivial search-and-replace to eliminate deprecation warnings in Scala 2.11.
Also works with 2.10

Author: Jakob Odersky <jakob@odersky.com>

Closes #11085 from jodersky/SPARK-13171.
2016-02-05 19:00:12 -08:00
Davies Liu 875f507929 [SPARK-13215] [SQL] remove fallback in codegen
Since we remove the configuration for codegen, we are heavily reply on codegen (also TungstenAggregate require the generated MutableProjection to update UnsafeRow), should remove the fallback, which could make user confusing, see the discussion in SPARK-13116.

Author: Davies Liu <davies@databricks.com>

Closes #11097 from davies/remove_fallback.
2016-02-05 15:07:43 -08:00
Wenchen Fan 1ed354a536 [SPARK-12939][SQL] migrate encoder resolution logic to Analyzer
https://issues.apache.org/jira/browse/SPARK-12939

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

follow-ups:

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10852 from cloud-fan/bug.
2016-02-05 14:34:12 -08:00
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
gatorsmile e3c75c6398 [SPARK-12850][SQL] Support Bucket Pruning (Predicate Pushdown for Bucketed Tables)
JIRA: https://issues.apache.org/jira/browse/SPARK-12850

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

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

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10942 from gatorsmile/pruningBuckets.
2016-02-04 18:37:58 -08:00
Daoyuan Wang 0f81318ae2 [SPARK-12828][SQL] add natural join support
Jira:
https://issues.apache.org/jira/browse/SPARK-12828

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

Closes #10762 from adrian-wang/naturaljoin.
2016-02-03 21:05:53 -08:00
Holden Karau a8e2ba776b [SPARK-13152][CORE] Fix task metrics deprecation warning
Make an internal non-deprecated version of incBytesRead and incRecordsRead so we don't have unecessary deprecation warnings in our build.

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

cc andrewor14 who did the initial deprecation

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

Closes #11056 from holdenk/SPARK-13152-fix-task-metrics-deprecation-warnings.
2016-02-03 17:43:14 -08:00
Reynold Xin 915a75398e [SPARK-13166][SQL] Remove DataStreamReader/Writer
They seem redundant and we can simply use DataFrameReader/Writer. The new usage looks like:

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

Author: Reynold Xin <rxin@databricks.com>

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

Author: Davies Liu <davies@databricks.com>

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

Author: Davies Liu <davies@databricks.com>

Closes #11032 from davies/better_suffix.
2016-02-02 22:13:10 -08:00
Davies Liu 99a6e3c1e8 [SPARK-12951] [SQL] support spilling in generated aggregate
This PR add spilling support for generated TungstenAggregate.

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

The changes will be covered by TungstenAggregationQueryWithControlledFallbackSuite

Author: Davies Liu <davies@databricks.com>

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

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

Author: Nong Li <nong@databricks.com>

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

Author: Davies Liu <davies@databricks.com>

Closes #10960 from davies/stddev.
2016-02-02 11:50:14 -08:00
Reynold Xin 7f6e3ec79b [SPARK-13138][SQL] Add "logical" package prefix for ddl.scala
ddl.scala is defined in the execution package, and yet its reference of "UnaryNode" and "Command" are logical. This was fairly confusing when I was trying to understand the ddl code.

Author: Reynold Xin <rxin@databricks.com>

Closes #11021 from rxin/SPARK-13138.
2016-02-02 11:29:20 -08:00
Michael Armbrust 29d92181d0 [SPARK-13094][SQL] Add encoders for seq/array of primitives
Author: Michael Armbrust <michael@databricks.com>

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

## User-facing API

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

Author: Reynold Xin <rxin@databricks.com>

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

Fixes build for Scala 2.11.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #10946 from jaceklaskowski/SPARK-12854-fix.
2016-02-01 13:57:48 -08:00
Nong Li 064b029c6a [SPARK-13043][SQL] Implement remaining catalyst types in ColumnarBatch.
This includes: float, boolean, short, decimal and calendar interval.

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

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

Author: Nong Li <nong@databricks.com>

Closes #10961 from nongli/spark-13043.
2016-02-01 13:56:14 -08:00
Herman van Hovell 5a8b978fab [SPARK-13049] Add First/last with ignore nulls to functions.scala
This PR adds the ability to specify the ```ignoreNulls``` option to the functions dsl, e.g:
```df.select($"id", last($"value", ignoreNulls = true).over(Window.partitionBy($"id").orderBy($"other"))```

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

cc yhuai

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

Closes #10957 from hvanhovell/SPARK-13049.
2016-01-31 13:56:13 -08:00
Liang-Chi Hsieh 0e6d92d042 [SPARK-12689][SQL] Migrate DDL parsing to the newly absorbed parser
JIRA: https://issues.apache.org/jira/browse/SPARK-12689

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

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

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

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

Closes #10979 from viirya/schema-merging-failure-message.
2016-01-30 23:02:49 -08:00
Wenchen Fan dab246f7e4 [SPARK-13098] [SQL] remove GenericInternalRowWithSchema
This class is only used for serialization of Python DataFrame. However, we don't require internal row there, so `GenericRowWithSchema` can also do the job.

Author: Wenchen Fan <wenchen@databricks.com>

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

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

Author: Davies Liu <davies@databricks.com>

Closes #10855 from davies/gen_keys.
2016-01-29 20:16:11 -08:00
Andrew Or 12252d1da9 [SPARK-13071] Coalescing HadoopRDD overwrites existing input metrics
This issue is causing tests to fail consistently in master with Hadoop 2.6 / 2.7. This is because for Hadoop 2.5+ we overwrite existing values of `InputMetrics#bytesRead` in each call to `HadoopRDD#compute`. In the case of coalesce, e.g.
```
sc.textFile(..., 4).coalesce(2).count()
```
we will call `compute` multiple times in the same task, overwriting `bytesRead` values from previous calls to `compute`.

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

This was caused by #10835.

Author: Andrew Or <andrew@databricks.com>

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

Author: Andrew Or <andrew@databricks.com>

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

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

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

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

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

Author: Davies Liu <davies@databricks.com>

Closes #10977 from davies/gen_refactor.
2016-01-29 01:59:59 -08:00
Liang-Chi Hsieh 66449b8dcd [SPARK-12968][SQL] Implement command to set current database
JIRA: https://issues.apache.org/jira/browse/SPARK-12968

Implement command to set current database.

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

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

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

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

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

Closes #9940 from viirya/safe-pushdown-parquet-filters.
2016-01-28 16:25:21 -08:00
Brandon Bradley 3a40c0e575 [SPARK-12749][SQL] add json option to parse floating-point types as DecimalType
I tried to add this via `USE_BIG_DECIMAL_FOR_FLOATS` option from Jackson with no success.

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

Author: Brandon Bradley <bradleytastic@gmail.com>

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

Author: Davies Liu <davies@databricks.com>

Closes #10944 from davies/gen_refactor.
2016-01-28 13:51:55 -08:00
Tejas Patil 676803963f [SPARK-12926][SQL] SQLContext to display warning message when non-sql configs are being set
Users unknowingly try to set core Spark configs in SQLContext but later realise that it didn't work. eg. sqlContext.sql("SET spark.shuffle.memoryFraction=0.4"). This PR adds a warning message when such operations are done.

Author: Tejas Patil <tejasp@fb.com>

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

Author: Cheng Lian <lian@databricks.com>

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

Author: Nong Li <nong@databricks.com>

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

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

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

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

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

Closes #10905 from hvanhovell/SPARK-12866.
2016-01-27 13:45:00 -08:00
Wenchen Fan 680afabe78 [SPARK-12938][SQL] DataFrame API for Bloom filter
This PR integrates Bloom filter from spark-sketch into DataFrame. This version resorts to RDD.aggregate for building the filter. A more performant UDAF version can be built in future follow-up PRs.

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

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

Author: Andrew Or <andrew@databricks.com>

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

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

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

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

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

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

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

Author: Cheng Lian <lian@databricks.com>

Closes #10911 from liancheng/cms-df-api.
2016-01-26 20:12:34 -08:00
Nong Li 555127387a [SPARK-12854][SQL] Implement complex types support in ColumnarBatch
This patch adds support for complex types for ColumnarBatch. ColumnarBatch supports structs
and arrays. There is a simple mapping between the richer catalyst types to these two. Strings
are treated as an array of bytes.

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

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

Author: Nong Li <nong@databricks.com>

Closes #10820 from nongli/spark-12854.
2016-01-26 17:34:01 -08:00
Sean Owen 649e9d0f5b [SPARK-3369][CORE][STREAMING] Java mapPartitions Iterator->Iterable is inconsistent with Scala's Iterator->Iterator
Fix Java function API methods for flatMap and mapPartitions to require producing only an Iterator, not Iterable. Also fix DStream.flatMap to require a function producing TraversableOnce only, not Traversable.

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

Author: Sean Owen <sowen@cloudera.com>

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

Author: Reynold Xin <rxin@databricks.com>

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

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

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

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

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

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

Author: Davies Liu <davies@databricks.com>

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

In more details,

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

Author: hyukjinkwon <gurwls223@gmail.com>

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

Author: Cheng Lian <lian@databricks.com>

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

Author: Reynold Xin <rxin@databricks.com>

Closes #10882 from rxin/SPARK-12904-1.
2016-01-23 12:13:05 -08:00
hyukjinkwon 5af5a02160 [SPARK-12872][SQL] Support to specify the option for compression codec for JSON datasource
https://issues.apache.org/jira/browse/SPARK-12872

This PR makes the JSON 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.

As `CSVCompressionCodecs` can be shared with other datasources, it became a separate class to share as `CompressionCodecs`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #10858 from HyukjinKwon/SPARK-12872.
2016-01-22 23:53:12 -08:00
gatorsmile e13c147e74 [SPARK-12959][SQL] Writing Bucketed Data with Disabled Bucketing in SQLConf
When users turn off bucketing in SQLConf, we should issue some messages to tell users these operations will be converted to normal way.

Also added a test case for this scenario and fixed the helper function.

Do you think this PR is helpful when using bucket tables? cloud-fan Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10870 from gatorsmile/bucketTableWritingTestcases.
2016-01-22 01:03:41 -08:00
Liang-Chi Hsieh 55c7dd031b [SPARK-12747][SQL] Use correct type name for Postgres JDBC's real array
https://issues.apache.org/jira/browse/SPARK-12747

Postgres JDBC driver uses "FLOAT4" or "FLOAT8" not "real".

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

Closes #10695 from viirya/fix-postgres-jdbc.
2016-01-21 18:55:28 -08:00
Davies Liu b362239df5 [SPARK-12797] [SQL] Generated TungstenAggregate (without grouping keys)
As discussed in #10786, the generated TungstenAggregate does not support imperative functions.

For a query
```
sqlContext.range(10).filter("id > 1").groupBy().count()
```

The generated code will looks like:
```
/* 032 */     if (!initAgg0) {
/* 033 */       initAgg0 = true;
/* 034 */
/* 035 */       // initialize aggregation buffer
/* 037 */       long bufValue2 = 0L;
/* 038 */
/* 039 */
/* 040 */       // initialize Range
/* 041 */       if (!range_initRange5) {
/* 042 */         range_initRange5 = true;
       ...
/* 071 */       }
/* 072 */
/* 073 */       while (!range_overflow8 && range_number7 < range_partitionEnd6) {
/* 074 */         long range_value9 = range_number7;
/* 075 */         range_number7 += 1L;
/* 076 */         if (range_number7 < range_value9 ^ 1L < 0) {
/* 077 */           range_overflow8 = true;
/* 078 */         }
/* 079 */
/* 085 */         boolean primitive11 = false;
/* 086 */         primitive11 = range_value9 > 1L;
/* 087 */         if (!false && primitive11) {
/* 092 */           // do aggregate and update aggregation buffer
/* 099 */           long primitive17 = -1L;
/* 100 */           primitive17 = bufValue2 + 1L;
/* 101 */           bufValue2 = primitive17;
/* 105 */         }
/* 107 */       }
/* 109 */
/* 110 */       // output the result
/* 112 */       bufferHolder25.reset();
/* 114 */       rowWriter26.initialize(bufferHolder25, 1);
/* 118 */       rowWriter26.write(0, bufValue2);
/* 120 */       result24.pointTo(bufferHolder25.buffer, bufferHolder25.totalSize());
/* 121 */       currentRow = result24;
/* 122 */       return;
/* 124 */     }
/* 125 */
```

cc nongli

Author: Davies Liu <davies@databricks.com>

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

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

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

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

Closes #10814 from davies/mutable_subexpr.
2016-01-20 10:02:40 -08:00
hyukjinkwon 6844d36aea [SPARK-12871][SQL] Support to specify the option for compression codec.
https://issues.apache.org/jira/browse/SPARK-12871
This PR added an option to support to specify compression codec.
This adds the option `codec` as an alias `compression` as filed in [SPARK-12668 ](https://issues.apache.org/jira/browse/SPARK-12668).

Note that I did not add configurations for Hadoop 1.x as this `CsvRelation` is using Hadoop 2.x API and I guess it is going to drop Hadoop 1.x support.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #10805 from HyukjinKwon/SPARK-12420.
2016-01-19 20:45:52 -08:00
Andrew Or b122c861cd [SPARK-12887] Do not expose var's in TaskMetrics
This is a step in implementing SPARK-10620, which migrates TaskMetrics to accumulators.

TaskMetrics has a bunch of var's, some are fully public, some are `private[spark]`. This is bad coding style that makes it easy to accidentally overwrite previously set metrics. This has happened a few times in the past and caused bugs that were difficult to debug.

Instead, we should have get-or-create semantics, which are more readily understandable. This makes sense in the case of TaskMetrics because these are just aggregated metrics that we want to collect throughout the task, so it doesn't matter who's incrementing them.

Parent PR: #10717

Author: Andrew Or <andrew@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Author: andrewor14 <andrew@databricks.com>

Closes #10815 from andrewor14/get-or-create-metrics.
2016-01-19 10:58:51 -08:00
Wenchen Fan e14817b528 [SPARK-12870][SQL] better format bucket id in file name
for normal parquet file without bucket, it's file name ends with a jobUUID which maybe all numbers and mistakeny regarded as bucket id. This PR improves the format of bucket id in file name by using a different seperator, `_`, so that the regex is more robust.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10799 from cloud-fan/fix-bucket.
2016-01-19 10:44:51 -08:00
proflin c00744e60f [SQL][MINOR] Fix one little mismatched comment according to the codes in interface.scala
Author: proflin <proflin.me@gmail.com>

Closes #10824 from proflin/master.
2016-01-19 00:15:43 -08:00
hyukjinkwon 453dae5671 [SPARK-12668][SQL] Providing aliases for CSV options to be similar to Pandas and R
https://issues.apache.org/jira/browse/SPARK-12668

Spark CSV datasource has been being merged (filed in [SPARK-12420](https://issues.apache.org/jira/browse/SPARK-12420)). This is a quicky PR that simply renames several CSV options to  similar Pandas and R.

- Alias for delimiter ­-> sep
- charset -­> encoding

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #10800 from HyukjinKwon/SPARK-12668.
2016-01-18 21:42:07 -08:00
gatorsmile 74ba84b64c [HOT][BUILD] Changed the import order
This PR is to fix the master's build break.

The following tests failed due to the import order issues in the master.
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49651/consoleFull
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49652/consoleFull
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49653/consoleFull

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10823 from gatorsmile/importOrder.
2016-01-18 19:40:10 -08:00
Davies Liu 323d51f1da [SPARK-12700] [SQL] embed condition into SMJ and BroadcastHashJoin
Currently SortMergeJoin and BroadcastHashJoin do not support condition, the need a followed Filter for that, the result projection to generate UnsafeRow could be very expensive if they generate lots of rows and could be filtered mostly by condition.

This PR brings the support of condition for SortMergeJoin and BroadcastHashJoin, just like other outer joins do.

This could improve the performance of Q72 by 7x (from 120s to 16.5s).

Author: Davies Liu <davies@databricks.com>

Closes #10653 from davies/filter_join.
2016-01-18 17:29:54 -08:00
Reynold Xin 39ac56fc60 [SPARK-12889][SQL] Rename ParserDialect -> ParserInterface.
Based on discussions in #10801, I'm submitting a pull request to rename ParserDialect to ParserInterface.

Author: Reynold Xin <rxin@databricks.com>

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

Author: Wenchen Fan <wenchen@databricks.com>

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

Author: Reynold Xin <rxin@databricks.com>

Closes #10801 from rxin/SPARK-12855.
2016-01-18 13:55:42 -08:00
Davies Liu 3c0d2365d5 [SPARK-12796] [SQL] Whole stage codegen
This is the initial work for whole stage codegen, it support Projection/Filter/Range, we will continue work on this to support more physical operators.

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

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

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

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

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

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

Author: Davies Liu <davies@databricks.com>

Closes #10735 from davies/whole2.
2016-01-16 10:29:27 -08:00
Davies Liu 242efb7546 [SPARK-12840] [SQL] Support passing arbitrary objects (not just expressions) into code generated classes
This is a refactor to support codegen for aggregation and broadcast join.

Author: Davies Liu <davies@databricks.com>

Closes #10777 from davies/rename2.
2016-01-15 19:07:42 -08:00
Nong Li 9039333c0a [SPARK-12644][SQL] Update parquet reader to be vectorized.
This inlines a few of the Parquet decoders and adds vectorized APIs to support decoding in batch.
There are a few particulars in the Parquet encodings that make this much more efficient. In
particular, RLE encodings are very well suited for batch decoding. The Parquet 2.0 encodings are
also very suited for this.

This is a work in progress and does not affect the current execution. In subsequent patches, we will
support more encodings and types before enabling this.

Simple benchmarks indicate this can decode single ints about > 3x faster.

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

Closes #10593 from nongli/spark-12644.
2016-01-15 17:40:26 -08:00
Wenchen Fan 3b5ccb12b8 [SPARK-12649][SQL] support reading bucketed table
This PR adds the support to read bucketed tables, and correctly populate `outputPartitioning`, so that we can avoid shuffle for some cases.

TODO(follow-up PRs):

* bucket pruning
* avoid shuffle for bucketed table join when use any super-set of the bucketing key.
 (we should re-visit it after https://issues.apache.org/jira/browse/SPARK-12704 is fixed)
* recognize hive bucketed table

Author: Wenchen Fan <wenchen@databricks.com>

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

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

cc rxin viirya marmbrus yhuai cloud-fan

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

Closes #10745 from hvanhovell/SPARK-12575-2.
2016-01-15 15:19:10 -08:00
Julien Baley 0bb73554a9 Fix typo
disvoered => discovered

Author: Julien Baley <julien.baley@gmail.com>

Closes #10773 from julienbaley/patch-1.
2016-01-15 13:53:20 -08:00
Yin Huai 513266c042 [SPARK-12833][HOT-FIX] Fix scala 2.11 compilation.
Seems 5f83c6991c breaks scala 2.11 compilation.

Author: Yin Huai <yhuai@databricks.com>

Closes #10774 from yhuai/fixScala211Compile.
2016-01-15 13:17:29 -08:00
Hossein 5f83c6991c [SPARK-12833][SQL] Initial import of spark-csv
CSV is the most common data format in the "small data" world. It is often the first format people want to try when they see Spark on a single node. Having to rely on a 3rd party component for this leads to poor user experience for new users. This PR merges the popular spark-csv data source package (https://github.com/databricks/spark-csv) with SparkSQL.

This is a first PR to bring the functionality to spark 2.0 master. We will complete items outlines in the design document (see JIRA attachment) in follow up pull requests.

Author: Hossein <hossein@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

Author: Reynold Xin <rxin@databricks.com>

Closes #10734 from rxin/simplify-case.
2016-01-13 12:44:35 -08:00
Liang-Chi Hsieh 63eee86cc6 [SPARK-9297] [SQL] Add covar_pop and covar_samp
JIRA: https://issues.apache.org/jira/browse/SPARK-9297

Add two aggregation functions: covar_pop and covar_samp.

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

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

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

Closes #10732 from sarutak/SPARK-12692-followup-sql.
2016-01-12 22:25:20 -08:00
Nong Li 9247084962 [SPARK-12785][SQL] Add ColumnarBatch, an in memory columnar format for execution.
There are many potential benefits of having an efficient in memory columnar format as an alternate
to UnsafeRow. This patch introduces ColumnarBatch/ColumnarVector which starts this effort. The
remaining implementation can be done as follow up patches.

As stated in the in the JIRA, there are useful external components that operate on memory in a
simple columnar format. ColumnarBatch would serve that purpose and could server as a
zero-serialization/zero-copy exchange for this use case.

This patch supports running the underlying data either on heap or off heap. On heap runs a bit
faster but we would need offheap for zero-copy exchanges. Currently, this mode is hidden behind one
interface (ColumnVector).

This differs from Parquet or the existing columnar cache because this is *not* intended to be used
as a storage format. The focus is entirely on CPU efficiency as we expect to only have 1 of these
batches in memory per task. The layout of the values is just dense arrays of the value type.

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

Closes #10628 from nongli/spark-12635.
2016-01-12 18:21:04 -08:00
Cheng Lian 8ed5f12d2b [SPARK-12724] SQL generation support for persisted data source tables
This PR implements SQL generation support for persisted data source tables.  A new field `metastoreTableIdentifier: Option[TableIdentifier]` is added to `LogicalRelation`.  When a `LogicalRelation` representing a persisted data source relation is created, this field holds the database name and table name of the relation.

Author: Cheng Lian <lian@databricks.com>

Closes #10712 from liancheng/spark-12724-datasources-sql-gen.
2016-01-12 14:19:53 -08:00
Reynold Xin 0d543b98f3 Revert "[SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before "," or ":")"
This reverts commit 8cfa218f4f.
2016-01-12 12:56:52 -08:00
Robert Kruszewski 508592b1ba [SPARK-9843][SQL] Make catalyst optimizer pass pluggable at runtime
Let me know whether you'd like to see it in other place

Author: Robert Kruszewski <robertk@palantir.com>

Closes #10210 from robert3005/feature/pluggable-optimizer.
2016-01-12 11:09:28 -08:00
Kousuke Saruta 8cfa218f4f [SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before "," or ":")
Fix the style violation (space before , and :).
This PR is a followup for #10643.

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

Closes #10718 from sarutak/SPARK-12692-followup-sql.
2016-01-12 00:51:00 -08:00
Anatoliy Plastinin 9559ac5f74 [SPARK-12744][SQL] Change parsing JSON integers to timestamps to treat integers as number of seconds
JIRA: https://issues.apache.org/jira/browse/SPARK-12744

This PR makes parsing JSON integers to timestamps consistent with casting behavior.

Author: Anatoliy Plastinin <anatoliy.plastinin@gmail.com>

Closes #10687 from antlypls/fix-json-timestamp-parsing.
2016-01-11 10:28:57 -08:00
Wenchen Fan f253feff62 [SPARK-12539][FOLLOW-UP] always sort in partitioning writer
address comments in #10498 , especially https://github.com/apache/spark/pull/10498#discussion_r49021259

Author: Wenchen Fan <wenchen@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@databricks.com>

Closes #10638 from cloud-fan/bucket-write.
2016-01-11 00:44:33 -08:00
Marcelo Vanzin 6439a82503 [SPARK-3873][BUILD] Enable import ordering error checking.
Turn import ordering violations into build errors, plus a few adjustments
to account for how the checker behaves. I'm a little on the fence about
whether the existing code is right, but it's easier to appease the checker
than to discuss what's the more correct order here.

Plus a few fixes to imports that cropped in since my recent cleanups.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10612 from vanzin/SPARK-3873-enable.
2016-01-10 20:04:50 -08:00
Reynold Xin b23c4521f5 [SPARK-12340] Fix overflow in various take functions.
This is a follow-up for the original patch #10562.

Author: Reynold Xin <rxin@databricks.com>

Closes #10670 from rxin/SPARK-12340.
2016-01-09 11:21:58 -08:00
Cheng Lian d9447cac74 [SPARK-12593][SQL] Converts resolved logical plan back to SQL
This PR tries to enable Spark SQL to convert resolved logical plans back to SQL query strings.  For now, the major use case is to canonicalize Spark SQL native view support.  The major entry point is `SQLBuilder.toSQL`, which returns an `Option[String]` if the logical plan is recognized.

The current version is still in WIP status, and is quite limited.  Known limitations include:

1.  The logical plan must be analyzed but not optimized

    The optimizer erases `Subquery` operators, which contain necessary scope information for SQL generation.  Future versions should be able to recover erased scope information by inserting subqueries when necessary.

1.  The logical plan must be created using HiveQL query string

    Query plans generated by composing arbitrary DataFrame API combinations are not supported yet.  Operators within these query plans need to be rearranged into a canonical form that is more suitable for direct SQL generation.  For example, the following query plan

    ```
    Filter (a#1 < 10)
     +- MetastoreRelation default, src, None
    ```

    need to be canonicalized into the following form before SQL generation:

    ```
    Project [a#1, b#2, c#3]
     +- Filter (a#1 < 10)
         +- MetastoreRelation default, src, None
    ```

    Otherwise, the SQL generation process will have to handle a large number of special cases.

1.  Only a fraction of expressions and basic logical plan operators are supported in this PR

    Currently, 95.7% (1720 out of 1798) query plans in `HiveCompatibilitySuite` can be successfully converted to SQL query strings.

    Known unsupported components are:

    - Expressions
      - Part of math expressions
      - Part of string expressions (buggy?)
      - Null expressions
      - Calendar interval literal
      - Part of date time expressions
      - Complex type creators
      - Special `NOT` expressions, e.g. `NOT LIKE` and `NOT IN`
    - Logical plan operators/patterns
      - Cube, rollup, and grouping set
      - Script transformation
      - Generator
      - Distinct aggregation patterns that fit `DistinctAggregationRewriter` analysis rule
      - Window functions

    Support for window functions, generators, and cubes etc. will be added in follow-up PRs.

This PR leverages `HiveCompatibilitySuite` for testing SQL generation in a "round-trip" manner:

*   For all select queries, we try to convert it back to SQL
*   If the query plan is convertible, we parse the generated SQL into a new logical plan
*   Run the new logical plan instead of the original one

If the query plan is inconvertible, the test case simply falls back to the original logic.

TODO

- [x] Fix failed test cases
- [x] Support for more basic expressions and logical plan operators (e.g. distinct aggregation etc.)
- [x] Comments and documentation

Author: Cheng Lian <lian@databricks.com>

Closes #10541 from liancheng/sql-generation.
2016-01-08 14:08:13 -08:00
Sean Owen b9c8353378 [SPARK-12618][CORE][STREAMING][SQL] Clean up build warnings: 2.0.0 edition
Fix most build warnings: mostly deprecated API usages. I'll annotate some of the changes below. CC rxin who is leading the charge to remove the deprecated APIs.

Author: Sean Owen <sowen@cloudera.com>

Closes #10570 from srowen/SPARK-12618.
2016-01-08 17:47:44 +00:00
Kevin Yu 5028a001d5 [SPARK-12317][SQL] Support units (m,k,g) in SQLConf
This PR is continue from previous closed PR 10314.

In this PR, SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE will be taken memory string conventions as input.

For example, the user can now specify 10g for SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE in SQLConf file.

marmbrus srowen : Can you help review this code changes ? Thanks.

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

Closes #10629 from kevinyu98/spark-12317.
2016-01-07 21:13:17 -08:00
Jacek Laskowski 07b314a57a [MINOR] Fix for BUILD FAILURE for Scala 2.11
It was introduced in 917d3fc069

/cc cloud-fan rxin

Author: Jacek Laskowski <jacek@japila.pl>

Closes #10636 from jaceklaskowski/fix-for-build-failure-2.11.
2016-01-07 10:39:46 -08:00
Sameer Agarwal f194d9911a [SPARK-12662][SQL] Fix DataFrame.randomSplit to avoid creating overlapping splits
https://issues.apache.org/jira/browse/SPARK-12662

cc yhuai

Author: Sameer Agarwal <sameer@databricks.com>

Closes #10626 from sameeragarwal/randomsplit.
2016-01-07 10:37:15 -08:00
Davies Liu 6a1c864ab6 [SPARK-12295] [SQL] external spilling for window functions
This PR manage the memory used by window functions (buffered rows), also enable external spilling.

After this PR, we can run window functions on a partition with hundreds of millions of rows with only 1G.

Author: Davies Liu <davies@databricks.com>

Closes #10605 from davies/unsafe_window.
2016-01-06 23:21:52 -08:00
Wenchen Fan 917d3fc069 [SPARK-12539][SQL] support writing bucketed table
This PR adds bucket write support to Spark SQL. User can specify bucketing columns, numBuckets and sorting columns with or without partition columns. For example:
```
df.write.partitionBy("year").bucketBy(8, "country").sortBy("amount").saveAsTable("sales")
```

When bucketing is used, we will calculate bucket id for each record, and group the records by bucket id. For each group, we will create a file with bucket id in its name, and write data into it. For each bucket file, if sorting columns are specified, the data will be sorted before write.

Note that there may be multiply files for one bucket, as the data is distributed.

Currently we store the bucket metadata at hive metastore in a non-hive-compatible way. We use different bucketing hash function compared to hive, so we can't be compatible anyway.

Limitations:

* Can't write bucketed data without hive metastore.
* Can't insert bucketed data into existing hive tables.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10498 from cloud-fan/bucket-write.
2016-01-06 16:58:10 -08:00
Herman van Hovell ea489f14f1 [SPARK-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst
This PR moves a major part of the new SQL parser to Catalyst. This is a prelude to start using this parser for all of our SQL parsing. The following key changes have been made:

The ANTLR Parser & Supporting classes have been moved to the Catalyst project. They are now part of the ```org.apache.spark.sql.catalyst.parser``` package. These classes contained quite a bit of code that was originally from the Hive project, I have added aknowledgements whenever this applied. All Hive dependencies have been factored out. I have also taken this chance to clean-up the ```ASTNode``` class, and to improve the error handling.

The HiveQl object that provides the functionality to convert an AST into a LogicalPlan has been refactored into three different classes, one for every SQL sub-project:
- ```CatalystQl```: This implements Query and Expression parsing functionality.
- ```SparkQl```: This is a subclass of CatalystQL and provides SQL/Core only functionality such as Explain and Describe.
- ```HiveQl```: This is a subclass of ```SparkQl``` and this adds Hive-only functionality to the parser such as Analyze, Drop, Views, CTAS & Transforms. This class still depends on Hive.

cc rxin

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

Closes #10583 from hvanhovell/SPARK-12575.
2016-01-06 11:16:53 -08:00
Yash Datta 9061e777fd [SPARK-11878][SQL] Eliminate distribute by in case group by is present with exactly the same grouping expressi
For queries like :
select <> from table group by a distribute by a
we can eliminate distribute by ; since group by will anyways do a hash partitioning
Also applicable when user uses Dataframe API

Author: Yash Datta <Yash.Datta@guavus.com>

Closes #9858 from saucam/eliminatedistribute.
2016-01-06 10:37:53 -08:00
QiangCai 5d871ea43e [SPARK-12340][SQL] fix Int overflow in the SparkPlan.executeTake, RDD.take and AsyncRDDActions.takeAsync
I have closed pull request https://github.com/apache/spark/pull/10487. And I create this pull request to resolve the problem.

spark jira
https://issues.apache.org/jira/browse/SPARK-12340

Author: QiangCai <david.caiq@gmail.com>

Closes #10562 from QiangCai/bugfix.
2016-01-06 18:13:07 +09:00
sureshthalamati 0d42292f6a [SPARK-12504][SQL] Masking credentials in the sql plan explain output for JDBC data sources.
This fix masks JDBC  credentials in the explain output.  URL patterns to specify credential seems to be vary between different databases. Added a new method to dialect to mask the credentials according to the database specific URL pattern.

While adding tests I noticed explain output includes array variable for partitions ([Lorg.apache.spark.Partition;3ff74546,).  Modified the code to include the first, and last partition information.

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

Closes #10452 from sureshthalamati/mask_jdbc_credentials_spark-12504.
2016-01-05 17:48:05 -08:00
Marcelo Vanzin df8bd97520 [SPARK-3873][SQL] Import ordering fixes.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10573 from vanzin/SPARK-3873-sql.
2016-01-05 16:48:59 -08:00
Nong c26d174265 [SPARK-12636] [SQL] Update UnsafeRowParquetRecordReader to support reading files directly.
As noted in the code, this change is to make this component easier to test in isolation.

Author: Nong <nongli@gmail.com>

Closes #10581 from nongli/spark-12636.
2016-01-05 13:47:24 -08:00
Wenchen Fan 76768337be [SPARK-12480][FOLLOW-UP] use a single column vararg for hash
address comments in #10435

This makes the API easier to use if user programmatically generate the call to hash, and they will get analysis exception if the arguments of hash is empty.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10588 from cloud-fan/hash.
2016-01-05 10:23:36 -08:00
Wenchen Fan b1a771231e [SPARK-12480][SQL] add Hash expression that can calculate hash value for a group of expressions
just write the arguments into unsafe row and use murmur3 to calculate hash code

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10435 from cloud-fan/hash-expr.
2016-01-04 18:49:41 -08:00
Reynold Xin 77ab49b857 [SPARK-12600][SQL] Remove deprecated methods in Spark SQL
Author: Reynold Xin <rxin@databricks.com>

Closes #10559 from rxin/remove-deprecated-sql.
2016-01-04 18:02:38 -08:00
Narine Kokhlikyan fdfac22d08 [SPARK-12509][SQL] Fixed error messages for DataFrame correlation and covariance
Currently, when we call corr or cov on dataframe with invalid input we see these error messages for both corr and cov:
   -  "Currently cov supports calculating the covariance between two columns"
   -  "Covariance calculation for columns with dataType "[DataType Name]" not supported."

I've fixed this issue by passing the function name as an argument. We could also do the input checks separately for each function. I avoided doing that because of code duplication.

Thanks!

Author: Narine Kokhlikyan <narine.kokhlikyan@gmail.com>

Closes #10458 from NarineK/sparksqlstatsmessages.
2016-01-04 16:14:49 -08:00
Nong Li 34de24abb5 [SPARK-12589][SQL] Fix UnsafeRowParquetRecordReader to properly set the row length.
The reader was previously not setting the row length meaning it was wrong if there were variable
length columns. This problem does not manifest usually, since the value in the column is correct and
projecting the row fixes the issue.

Author: Nong Li <nong@databricks.com>

Closes #10576 from nongli/spark-12589.
2016-01-04 14:58:24 -08:00
Davies Liu d084a2de32 [SPARK-12541] [SQL] support cube/rollup as function
This PR enable cube/rollup as function, so they can be used as this:
```
select a, b, sum(c) from t group by rollup(a, b)
```

Author: Davies Liu <davies@databricks.com>

Closes #10522 from davies/rollup.
2016-01-04 14:26:56 -08:00
tedyu 40d03960d7 [DOC] Adjust coverage for partitionBy()
This is the related thread: http://search-hadoop.com/m/q3RTtO3ReeJ1iF02&subj=Re+partitioning+json+data+in+spark

Michael suggested fixing the doc.

Please review.

Author: tedyu <yuzhihong@gmail.com>

Closes #10499 from ted-yu/master.
2016-01-04 12:38:04 -08:00
Xiu Guo 573ac55d74 [SPARK-12512][SQL] support column name with dot in withColumn()
Author: Xiu Guo <xguo27@gmail.com>

Closes #10500 from xguo27/SPARK-12512.
2016-01-04 12:34:04 -08:00
Josh Rosen 6c83d938cc [SPARK-12579][SQL] Force user-specified JDBC driver to take precedence
Spark SQL's JDBC data source allows users to specify an explicit JDBC driver to load (using the `driver` argument), but in the current code it's possible that the user-specified driver will not be used when it comes time to actually create a JDBC connection.

In a nutshell, the problem is that you might have multiple JDBC drivers on the classpath that claim to be able to handle the same subprotocol, so simply registering the user-provided driver class with the our `DriverRegistry` and JDBC's `DriverManager` is not sufficient to ensure that it's actually used when creating the JDBC connection.

This patch addresses this issue by first registering the user-specified driver with the DriverManager, then iterating over the driver manager's loaded drivers in order to obtain the correct driver and use it to create a connection (previously, we just called `DriverManager.getConnection()` directly).

If a user did not specify a JDBC driver to use, then we call `DriverManager.getDriver` to figure out the class of the driver to use, then pass that class's name to executors; this guards against corner-case bugs in situations where the driver and executor JVMs might have different sets of JDBC drivers on their classpaths (previously, there was the (rare) potential for `DriverManager.getConnection()` to use different drivers on the driver and executors if the user had not explicitly specified a JDBC driver class and the classpaths were different).

This patch is inspired by a similar patch that I made to the `spark-redshift` library (https://github.com/databricks/spark-redshift/pull/143), which contains its own modified fork of some of Spark's JDBC data source code (for cross-Spark-version compatibility reasons).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10519 from JoshRosen/jdbc-driver-precedence.
2016-01-04 10:39:42 -08:00
Xiu Guo 84f8492c15 [SPARK-12562][SQL] DataFrame.write.format(text) requires the column name to be called value
Author: Xiu Guo <xguo27@gmail.com>

Closes #10515 from xguo27/SPARK-12562.
2016-01-03 20:48:56 -08:00
Cazen b8410ff9ce [SPARK-12537][SQL] Add option to accept quoting of all character backslash quoting mechanism
We can provides the option to choose JSON parser can be enabled to accept quoting of all character or not.

Author: Cazen <Cazen@korea.com>
Author: Cazen Lee <cazen.lee@samsung.com>
Author: Cazen Lee <Cazen@korea.com>
Author: cazen.lee <cazen.lee@samsung.com>

Closes #10497 from Cazen/master.
2016-01-03 17:01:19 -08:00
Reynold Xin 6c5bbd628a Revert "Revert "[SPARK-12286][SPARK-12290][SPARK-12294][SPARK-12284][SQL] always output UnsafeRow""
This reverts commit 44ee920fd4.
2016-01-02 22:39:25 -08:00
Reynold Xin 513e3b092c [SPARK-12599][MLLIB][SQL] Remove the use of callUDF in MLlib
callUDF has been deprecated. However, we do not have an alternative for users to specify the output data type without type tags. This pull request introduced a new API for that, and replaces the invocation of the deprecated callUDF with that.

Author: Reynold Xin <rxin@databricks.com>

Closes #10547 from rxin/SPARK-12599.
2016-01-02 22:31:39 -08:00
Sean Owen 15bd73627e [SPARK-12481][CORE][STREAMING][SQL] Remove usage of Hadoop deprecated APIs and reflection that supported 1.x
Remove use of deprecated Hadoop APIs now that 2.2+ is required

Author: Sean Owen <sowen@cloudera.com>

Closes #10446 from srowen/SPARK-12481.
2016-01-02 13:15:53 +00:00
hyukjinkwon 94f7a12b3c [SPARK-10180][SQL] JDBC datasource are not processing EqualNullSafe filter
This PR is followed by https://github.com/apache/spark/pull/8391.
Previous PR fixes JDBCRDD to support null-safe equality comparison for JDBC datasource. This PR fixes the problem that it can actually return null as a result of the comparison resulting error as using the value of that comparison.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: HyukjinKwon <gurwls223@gmail.com>

Closes #8743 from HyukjinKwon/SPARK-10180.
2016-01-02 00:04:48 -08:00
Reynold Xin 44ee920fd4 Revert "[SPARK-12286][SPARK-12290][SPARK-12294][SPARK-12284][SQL] always output UnsafeRow"
This reverts commit 0da7bd50dd.
2016-01-01 19:23:06 -08:00
Davies Liu 0da7bd50dd [SPARK-12286][SPARK-12290][SPARK-12294][SPARK-12284][SQL] always output UnsafeRow
It's confusing that some operator output UnsafeRow but some not, easy to make mistake.

This PR change to only output UnsafeRow for all the operators (SparkPlan), removed the rule to insert Unsafe/Safe conversions. For those that can't output UnsafeRow directly, added UnsafeProjection into them.

Closes #10330

cc JoshRosen rxin

Author: Davies Liu <davies@databricks.com>

Closes #10511 from davies/unsafe_row.
2016-01-01 13:39:20 -08:00
Liang-Chi Hsieh ad5b7cfcca [SPARK-12409][SPARK-12387][SPARK-12391][SQL] Refactor filter pushdown for JDBCRDD and add few filters
This patch refactors the filter pushdown for JDBCRDD and also adds few filters.

Added filters are basically from #10468 with some refactoring. Test cases are from #10468.

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

Closes #10470 from viirya/refactor-jdbc-filter.
2016-01-01 00:54:25 -08:00
Davies Liu e6c77874b9 [SPARK-12585] [SQL] move numFields to constructor of UnsafeRow
Right now, numFields will be passed in by pointTo(), then bitSetWidthInBytes is calculated, making pointTo() a little bit heavy.

It should be part of constructor of UnsafeRow.

Author: Davies Liu <davies@databricks.com>

Closes #10528 from davies/numFields.
2015-12-30 22:16:37 -08:00
Takeshi YAMAMURO 5c2682b0c8 [SPARK-12409][SPARK-12387][SPARK-12391][SQL] Support AND/OR/IN/LIKE push-down filters for JDBC
This is rework from #10386 and add more tests and LIKE push-down support.

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

Closes #10468 from maropu/SupportMorePushdownInJdbc.
2015-12-30 13:34:37 -08:00
Reynold Xin 270a659584 [SPARK-12549][SQL] Take Option[Seq[DataType]] in UDF input type specification.
In Spark we allow UDFs to declare its expected input types in order to apply type coercion. The expected input type parameter takes a Seq[DataType] and uses Nil when no type coercion is applied. It makes more sense to take Option[Seq[DataType]] instead, so we can differentiate a no-arg function vs function with no expected input type specified.

Author: Reynold Xin <rxin@databricks.com>

Closes #10504 from rxin/SPARK-12549.
2015-12-29 16:58:23 -08:00
Hossein f6ecf14333 [SPARK-11199][SPARKR] Improve R context management story and add getOrCreate
* Changes api.r.SQLUtils to use ```SQLContext.getOrCreate``` instead of creating a new context.
* Adds a simple test

[SPARK-11199] #comment link with JIRA

Author: Hossein <hossein@databricks.com>

Closes #9185 from falaki/SPARK-11199.
2015-12-29 11:44:20 -08:00
Takeshi YAMAMURO 73862a1eb9 [SPARK-11394][SQL] Throw IllegalArgumentException for unsupported types in postgresql
If DataFrame has BYTE types, throws an exception:
org.postgresql.util.PSQLException: ERROR: type "byte" does not exist

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

Closes #9350 from maropu/FixBugInPostgreJdbc.
2015-12-28 21:28:32 -08:00
Reynold Xin 1a91be8078 [SPARK-12547][SQL] Tighten scala style checker enforcement for UDF registration
We use scalastyle:off to turn off style checks in certain places where it is not possible to follow the style guide. This is usually ok. However, in udf registration, we disable the checker for a large amount of code simply because some of them exceed 100 char line limit. It is better to just disable the line limit check rather than everything.

In this pull request, I only disabled line length check, and fixed a problem (lack explicit types for public methods).

Author: Reynold Xin <rxin@databricks.com>

Closes #10501 from rxin/SPARK-12547.
2015-12-28 20:43:06 -08:00
gatorsmile 043135819c [SPARK-12522][SQL][MINOR] Add the missing document strings for the SQL configuration
Fixing the missing the document for the configuration. We can see the missing messages "TODO" when issuing the command "SET -V".
```
spark.sql.columnNameOfCorruptRecord
spark.sql.hive.verifyPartitionPath
spark.sql.sources.parallelPartitionDiscovery.threshold
spark.sql.hive.convertMetastoreParquet.mergeSchema
spark.sql.hive.convertCTAS
spark.sql.hive.thriftServer.async
```

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10471 from gatorsmile/commandDesc.
2015-12-28 17:22:18 -08:00
Shixiong Zhu 710b411729 [SPARK-12489][CORE][SQL][MLIB] Fix minor issues found by FindBugs
Include the following changes:

1. Close `java.sql.Statement`
2. Fix incorrect `asInstanceOf`.
3. Remove unnecessary `synchronized` and `ReentrantLock`.

Author: Shixiong Zhu <shixiong@databricks.com>

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

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

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

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

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

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

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

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

Closes #10393 from gatorsmile/generateExplain.
2015-12-28 12:48:30 -08:00
gatorsmile e01c6c8664 [SPARK-12287][SQL] Support UnsafeRow in MapPartitions/MapGroups/CoGroup
Support Unsafe Row in MapPartitions/MapGroups/CoGroup.

Added a test case for MapPartitions. Since MapGroups and CoGroup are built on AppendColumns, all the related dataset test cases already can verify the correctness when MapGroups and CoGroup processing unsafe rows.

davies cloud-fan Not sure if my understanding is right, please correct me. Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10398 from gatorsmile/unsafeRowMapGroup.
2015-12-28 12:23:28 -08:00
Kevin Yu fd50df413f [SPARK-12231][SQL] create a combineFilters' projection when we call buildPartitionedTableScan
Hello Michael & All:

We have some issues to submit the new codes in the other PR(#10299), so we closed that PR and open this one with the fix.

The reason for the previous failure is that the projection for the scan when there is a filter that is not pushed down (the "left-over" filter) could be different, in elements or ordering, from the original projection.

With this new codes, the approach to solve this problem is:

Insert a new Project if the "left-over" filter is nonempty and (the original projection is not empty and the projection for the scan has more than one elements which could otherwise cause different ordering in projection).

We create 3 test cases to cover the otherwise failure cases.

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

Closes #10388 from kevinyu98/spark-12231.
2015-12-28 11:58:33 -08:00
felixcheung 5aa2710c1e [SPARK-12515][SQL][DOC] minor doc update for read.jdbc
Author: felixcheung <felixcheung_m@hotmail.com>

Closes #10465 from felixcheung/dfreaderjdbcdoc.
2015-12-28 10:22:45 +00:00
CK50 502476e45c [SPARK-12010][SQL] Spark JDBC requires support for column-name-free INSERT syntax
In the past Spark JDBC write only worked with technologies which support the following INSERT statement syntax (JdbcUtils.scala: insertStatement()):

INSERT INTO $table VALUES ( ?, ?, ..., ? )

But some technologies require a list of column names:

INSERT INTO $table ( $colNameList ) VALUES ( ?, ?, ..., ? )

This was blocking the use of e.g. the Progress JDBC Driver for Cassandra.

Another limitation is that syntax 1 relies no the dataframe field ordering match that of the target table. This works fine, as long as the target table has been created by writer.jdbc().

If the target table contains more columns (not created by writer.jdbc()), then the insert fails due mismatch of number of columns or their data types.

This PR switches to the recommended second INSERT syntax. Column names are taken from datafram field names.

Author: CK50 <christian.kurz@oracle.com>

Closes #10380 from CK50/master-SPARK-12010-2.
2015-12-24 13:39:11 +00:00
Liang-Chi Hsieh 50301c0a28 [SPARK-11164][SQL] Add InSet pushdown filter back for Parquet
When the filter is ```"b in ('1', '2')"```, the filter is not pushed down to Parquet. Thanks!

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

Closes #10278 from gatorsmile/parquetFilterNot.
2015-12-23 14:08:29 +08:00
Cheng Lian 42bfde2983 [SPARK-12371][SQL] Runtime nullability check for NewInstance
This PR adds a new expression `AssertNotNull` to ensure non-nullable fields of products and case classes don't receive null values at runtime.

Author: Cheng Lian <lian@databricks.com>

Closes #10331 from liancheng/dataset-nullability-check.
2015-12-22 19:41:44 +08:00
Takeshi YAMAMURO 8c1b867cee [SPARK-12446][SQL] Add unit tests for JDBCRDD internal functions
No tests done for JDBCRDD#compileFilter.

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

Closes #10409 from maropu/AddTestsInJdbcRdd.
2015-12-22 00:50:05 -08:00
Alex Bozarth b0849b8aea [SPARK-12339][SPARK-11206][WEBUI] Added a null check that was removed in
Updates made in SPARK-11206 missed an edge case which cause's a NullPointerException when a task is killed. In some cases when a task ends in failure taskMetrics is initialized as null (see JobProgressListener.onTaskEnd()). To address this a null check was added. Before the changes in SPARK-11206 this null check was called at the start of the updateTaskAccumulatorValues() function.

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #10405 from ajbozarth/spark12339.
2015-12-21 14:06:36 -08:00
gatorsmile 4883a5087d [SPARK-12374][SPARK-12150][SQL] Adding logical/physical operators for Range
Based on the suggestions from marmbrus , added logical/physical operators for Range for improving the performance.

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

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

Thank you very much!

Author: gatorsmile <gatorsmile@gmail.com>

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

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

example json:

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

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

Closes #10373 from dilipbiswal/spark-12398.
2015-12-21 12:46:06 -08:00
Yin Huai 41ee7c57ab [SPARK-12218][SQL] Invalid splitting of nested AND expressions in Data Source filter API
JIRA: https://issues.apache.org/jira/browse/SPARK-12218

When creating filters for Parquet/ORC, we should not push nested AND expressions partially.

Author: Yin Huai <yhuai@databricks.com>

Closes #10362 from yhuai/SPARK-12218.
2015-12-18 10:53:13 -08:00
Davies Liu 4af647c77d [SPARK-12054] [SQL] Consider nullability of expression in codegen
This could simplify the generated code for expressions that is not nullable.

This PR fix lots of bugs about nullability.

Author: Davies Liu <davies@databricks.com>

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

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

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

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

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

cc rxin / yhuai

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

Closes #9819 from hvanhovell/SPARK-8641-2.
2015-12-17 15:16:35 -08:00
Reynold Xin e096a652b9 [SPARK-12397][SQL] Improve error messages for data sources when they are not found
Point users to spark-packages.org to find them.

Author: Reynold Xin <rxin@databricks.com>

Closes #10351 from rxin/SPARK-12397.
2015-12-17 14:16:49 -08:00
Yanbo Liang 6e0771665b [SQL] Update SQLContext.read.text doc
Since we rename the column name from ```text``` to ```value``` for DataFrame load by ```SQLContext.read.text```, we need to update doc.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10349 from yanboliang/text-value.
2015-12-17 09:19:46 -08:00
Davies Liu a170d34a1b [SPARK-12395] [SQL] fix resulting columns of outer join
For API DataFrame.join(right, usingColumns, joinType), if the joinType is right_outer or full_outer, the resulting join columns could be wrong (will be null).

The order of columns had been changed to match that with MySQL and PostgreSQL [1].

This PR also fix the nullability of output for outer join.

[1] http://www.postgresql.org/docs/9.2/static/queries-table-expressions.html

Author: Davies Liu <davies@databricks.com>

Closes #10353 from davies/fix_join.
2015-12-17 08:04:11 -08:00
Yin Huai 9d66c4216a [SPARK-12057][SQL] Prevent failure on corrupt JSON records
This PR makes JSON parser and schema inference handle more cases where we have unparsed records. It is based on #10043. The last commit fixes the failed test and updates the logic of schema inference.

Regarding the schema inference change, if we have something like
```
{"f1":1}
[1,2,3]
```
originally, we will get a DF without any column.
After this change, we will get a DF with columns `f1` and `_corrupt_record`. Basically, for the second row, `[1,2,3]` will be the value of `_corrupt_record`.

When merge this PR, please make sure that the author is simplyianm.

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

Closes #10043

Author: Ian Macalinao <me@ian.pw>
Author: Yin Huai <yhuai@databricks.com>

Closes #10288 from yhuai/handleCorruptJson.
2015-12-16 23:18:53 -08:00
gatorsmile edf65cd961 [SPARK-12164][SQL] Decode the encoded values and then display
Based on the suggestions from marmbrus cloud-fan in https://github.com/apache/spark/pull/10165 , this PR is to print the decoded values(user objects) in `Dataset.show`
```scala
    implicit val kryoEncoder = Encoders.kryo[KryoClassData]
    val ds = Seq(KryoClassData("a", 1), KryoClassData("b", 2), KryoClassData("c", 3)).toDS()
    ds.show(20, false);
```
The current output is like
```
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|value                                                                                                                                                                                 |
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|[1, 0, 111, 114, 103, 46, 97, 112, 97, 99, 104, 101, 46, 115, 112, 97, 114, 107, 46, 115, 113, 108, 46, 75, 114, 121, 111, 67, 108, 97, 115, 115, 68, 97, 116, -31, 1, 1, -126, 97, 2]|
|[1, 0, 111, 114, 103, 46, 97, 112, 97, 99, 104, 101, 46, 115, 112, 97, 114, 107, 46, 115, 113, 108, 46, 75, 114, 121, 111, 67, 108, 97, 115, 115, 68, 97, 116, -31, 1, 1, -126, 98, 4]|
|[1, 0, 111, 114, 103, 46, 97, 112, 97, 99, 104, 101, 46, 115, 112, 97, 114, 107, 46, 115, 113, 108, 46, 75, 114, 121, 111, 67, 108, 97, 115, 115, 68, 97, 116, -31, 1, 1, -126, 99, 6]|
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
```
After the fix, it will be like the below if and only if the users override the `toString` function in the class `KryoClassData`
```scala
override def toString: String = s"KryoClassData($a, $b)"
```
```
+-------------------+
|value              |
+-------------------+
|KryoClassData(a, 1)|
|KryoClassData(b, 2)|
|KryoClassData(c, 3)|
+-------------------+
```

If users do not override the `toString` function, the results will be like
```
+---------------------------------------+
|value                                  |
+---------------------------------------+
|org.apache.spark.sql.KryoClassData68ef|
|org.apache.spark.sql.KryoClassData6915|
|org.apache.spark.sql.KryoClassData693b|
+---------------------------------------+
```

Question: Should we add another optional parameter in the function `show`? It will decide if the function `show` will display the hex values or the object values?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10215 from gatorsmile/showDecodedValue.
2015-12-16 13:22:34 -08:00
Davies Liu 54c512ba90 [SPARK-8745] [SQL] remove GenerateProjection
cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #10316 from davies/remove_generate_projection.
2015-12-16 10:22:48 -08:00
Reynold Xin 1a3d0cd9f0 Revert "[SPARK-12105] [SQL] add convenient show functions"
This reverts commit 31b391019f.
2015-12-16 00:57:34 -08:00
Reynold Xin 18ea11c3a8 Revert "[HOTFIX] Compile error from commit 31b3910"
This reverts commit 840bd2e008.
2015-12-16 00:57:07 -08:00
hyukjinkwon 2aad2d3724 [SPARK-12315][SQL] isnotnull operator not pushed down for JDBC datasource.
https://issues.apache.org/jira/browse/SPARK-12315
`IsNotNull` filter is not being pushed down for JDBC datasource.

It looks it is SQL standard according to [SQL-92](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt), SQL:1999, [SQL:2003](http://www.wiscorp.com/sql_2003_standard.zip) and [SQL:201x](http://www.wiscorp.com/sql20nn.zip) and I believe most databases support this.

In this PR, I simply added the case for `IsNotNull` filter to produce a proper filter string.

Author: hyukjinkwon <gurwls223@gmail.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@databricks.com>

Closes #10287 from HyukjinKwon/SPARK-12315.
2015-12-15 22:30:35 -08:00
hyukjinkwon 7f443a6879 [SPARK-12314][SQL] isnull operator not pushed down for JDBC datasource.
https://issues.apache.org/jira/browse/SPARK-12314
`IsNull` filter is not being pushed down for JDBC datasource.

It looks it is SQL standard according to [SQL-92](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt), SQL:1999, [SQL:2003](http://www.wiscorp.com/sql_2003_standard.zip) and [SQL:201x](http://www.wiscorp.com/sql20nn.zip) and I believe most databases support this.

In this PR, I simply added the case for `IsNull` filter to produce a proper filter string.

Author: hyukjinkwon <gurwls223@gmail.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@databricks.com>

Closes #10286 from HyukjinKwon/SPARK-12314.
2015-12-15 22:25:08 -08:00
hyukjinkwon 0f6936b5f1 [SPARK-12249][SQL] JDBC non-equality comparison operator not pushed down.
https://issues.apache.org/jira/browse/SPARK-12249
Currently `!=` operator is not pushed down correctly.
I simply added a case for this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #10233 from HyukjinKwon/SPARK-12249.
2015-12-15 22:22:49 -08:00
tedyu f725b2ec1a [SPARK-12056][CORE] Part 2 Create a TaskAttemptContext only after calling setConf
This is continuation of SPARK-12056 where change is applied to SqlNewHadoopRDD.scala

andrewor14
FYI

Author: tedyu <yuzhihong@gmail.com>

Closes #10164 from tedyu/master.
2015-12-15 18:15:10 -08:00
Andrew Or 840bd2e008 [HOTFIX] Compile error from commit 31b3910 2015-12-15 18:11:53 -08:00
Jean-Baptiste Onofré 31b391019f [SPARK-12105] [SQL] add convenient show functions
Author: Jean-Baptiste Onofré <jbonofre@apache.org>

Closes #10130 from jbonofre/SPARK-12105.
2015-12-15 18:08:09 -08:00
gatorsmile 606f99b942 [SPARK-12288] [SQL] Support UnsafeRow in Coalesce/Except/Intersect.
Support UnsafeRow for the Coalesce/Except/Intersect.

Could you review if my code changes are ok? davies Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10285 from gatorsmile/unsafeSupportCIE.
2015-12-14 19:42:16 -08:00
gatorsmile d13ff82cba [SPARK-12188][SQL][FOLLOW-UP] Code refactoring and comment correction in Dataset APIs
marmbrus This PR is to address your comment. Thanks for your review!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10214 from gatorsmile/followup12188.
2015-12-14 18:33:45 -08:00
yucai ed87f6d3b4 [SPARK-12275][SQL] No plan for BroadcastHint in some condition
When SparkStrategies.BasicOperators's "case BroadcastHint(child) => apply(child)" is hit, it only recursively invokes BasicOperators.apply with this "child". It makes many strategies have no change to process this plan, which probably leads to "No plan" issue, so we use planLater to go through all strategies.

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

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

Closes #10265 from yucai/broadcast_hint.
2015-12-13 23:08:21 -08:00
Davies Liu 834e71489b [SPARK-12213][SQL] use multiple partitions for single distinct query
Currently, we could generate different plans for query with single distinct (depends on spark.sql.specializeSingleDistinctAggPlanning), one works better on low cardinality columns, the other
works better for high cardinality column (default one).

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

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

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

cc yhuai nongli marmbrus

Author: Davies Liu <davies@databricks.com>

Closes #10228 from davies/single_distinct.
2015-12-13 22:57:01 -08:00
Ankur Dave 1e799d617a [SPARK-12298][SQL] Fix infinite loop in DataFrame.sortWithinPartitions
Modifies the String overload to call the Column overload and ensures this is called in a test.

Author: Ankur Dave <ankurdave@gmail.com>

Closes #10271 from ankurdave/SPARK-12298.
2015-12-11 19:07:48 -08:00
Josh Rosen 23a9e62bad [SPARK-12251] Document and improve off-heap memory configurations
This patch adds documentation for Spark configurations that affect off-heap memory and makes some naming and validation improvements for those configs.

- Change `spark.memory.offHeapSize` to `spark.memory.offHeap.size`. This is fine because this configuration has not shipped in any Spark release yet (it's new in Spark 1.6).
- Deprecated `spark.unsafe.offHeap` in favor of a new `spark.memory.offHeap.enabled` configuration. The motivation behind this change is to gather all memory-related configurations under the same prefix.
- Add a check which prevents users from setting `spark.memory.offHeap.enabled=true` when `spark.memory.offHeap.size == 0`. After SPARK-11389 (#9344), which was committed in Spark 1.6, Spark enforces a hard limit on the amount of off-heap memory that it will allocate to tasks. As a result, enabling off-heap execution memory without setting `spark.memory.offHeap.size` will lead to immediate OOMs. The new configuration validation makes this scenario easier to diagnose, helping to avoid user confusion.
- Document these configurations on the configuration page.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10237 from JoshRosen/SPARK-12251.
2015-12-10 15:29:04 -08:00
Yin Huai bc5f56aa60 [SPARK-12250][SQL] Allow users to define a UDAF without providing details of its inputSchema
https://issues.apache.org/jira/browse/SPARK-12250

Author: Yin Huai <yhuai@databricks.com>

Closes #10236 from yhuai/SPARK-12250.
2015-12-10 12:03:29 -08:00
Reynold Xin 76540b6df5 [SPARK-12242][SQL] Add DataFrame.transform method
Author: Reynold Xin <rxin@databricks.com>

Closes #10226 from rxin/df-transform.
2015-12-10 22:23:10 +08:00
Cheng Lian 6e1c55eac4 [SPARK-12012][SQL] Show more comprehensive PhysicalRDD metadata when visualizing SQL query plan
This PR adds a `private[sql]` method `metadata` to `SparkPlan`, which can be used to describe detail information about a physical plan during visualization. Specifically, this PR uses this method to provide details of `PhysicalRDD`s translated from a data source relation. For example, a `ParquetRelation` converted from Hive metastore table `default.psrc` is now shown as the following screenshot:

![image](https://cloud.githubusercontent.com/assets/230655/11526657/e10cb7e6-9916-11e5-9afa-f108932ec890.png)

And here is the screenshot for a regular `ParquetRelation` (not converted from Hive metastore table) loaded from a really long path:

![output](https://cloud.githubusercontent.com/assets/230655/11680582/37c66460-9e94-11e5-8f50-842db5309d5a.png)

Author: Cheng Lian <lian@databricks.com>

Closes #10004 from liancheng/spark-12012.physical-rdd-metadata.
2015-12-09 23:30:42 +08:00
Michael Armbrust 3959489423 [SPARK-12069][SQL] Update documentation with Datasets
Author: Michael Armbrust <michael@databricks.com>

Closes #10060 from marmbrus/docs.
2015-12-08 15:58:35 -08:00
gatorsmile 5d96a710a5 [SPARK-12188][SQL] Code refactoring and comment correction in Dataset APIs
This PR contains the following updates:

- Created a new private variable `boundTEncoder` that can be shared by multiple functions, `RDD`, `select` and `collect`.
- Replaced all the `queryExecution.analyzed` by the function call `logicalPlan`
- A few API comments are using wrong class names (e.g., `DataFrame`) or parameter names (e.g., `n`)
- A few API descriptions are wrong. (e.g., `mapPartitions`)

marmbrus rxin cloud-fan Could you take a look and check if they are appropriate? Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10184 from gatorsmile/datasetClean.
2015-12-08 10:25:57 -08:00
tedyu 84b809445f [SPARK-11884] Drop multiple columns in the DataFrame API
See the thread Ben started:
http://search-hadoop.com/m/q3RTtveEuhjsr7g/

This PR adds drop() method to DataFrame which accepts multiple column names

Author: tedyu <yuzhihong@gmail.com>

Closes #9862 from ted-yu/master.
2015-12-07 14:58:09 -08:00
gcc 04b6799932 [SPARK-12048][SQL] Prevent to close JDBC resources twice
Author: gcc <spark-src@condor.rhaag.ip>

Closes #10101 from rh99/master.
2015-12-06 16:27:40 +00:00
Shixiong Zhu 3af53e61fd [SPARK-12084][CORE] Fix codes that uses ByteBuffer.array incorrectly
`ByteBuffer` doesn't guarantee all contents in `ByteBuffer.array` are valid. E.g, a ByteBuffer returned by `ByteBuffer.slice`. We should not use the whole content of `ByteBuffer` unless we know that's correct.

This patch fixed all places that use `ByteBuffer.array` incorrectly.

Author: Shixiong Zhu <shixiong@databricks.com>

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

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

Author: Josh Rosen <joshrosen@databricks.com>

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

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

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

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

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

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

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

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

cr: JoshRosen, pwendell

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

Author: Dmitry Erastov <derastov@gmail.com>

Closes #9867 from dskrvk/master.
2015-12-04 12:03:45 -08:00
Carson Wang b6e9963ee4 [SPARK-11206] Support SQL UI on the history server (resubmit)
Resubmit #9297 and #9991
On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution.

To support SQL UI on the history server:
1. I added an onOtherEvent method to the SparkListener trait and post all SQL related events to the same event bus.
2. Two SQL events SparkListenerSQLExecutionStart and SparkListenerSQLExecutionEnd are defined in the sql module.
3. The new SQL events are written to event log using Jackson.
4. A new trait SparkHistoryListenerFactory is added to allow the history server to feed events to the SQL history listener. The SQL implementation is loaded at runtime using java.util.ServiceLoader.

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

Closes #10061 from carsonwang/SqlHistoryUI.
2015-12-03 16:39:12 -08:00
Huaxin Gao 5349851f36 [SPARK-12088][SQL] check connection.isClosed before calling connection…
In Java Spec java.sql.Connection, it has
boolean getAutoCommit() throws SQLException
Throws:
SQLException - if a database access error occurs or this method is called on a closed connection

So if conn.getAutoCommit is called on a closed connection, a SQLException will be thrown. Even though the code catch the SQLException and program can continue, I think we should check conn.isClosed before calling conn.getAutoCommit to avoid the unnecessary SQLException.

Author: Huaxin Gao <huaxing@oc0558782468.ibm.com>

Closes #10095 from huaxingao/spark-12088.
2015-12-03 08:42:21 +00:00
Yadong Qi d0d7ec5330 [SPARK-12093][SQL] Fix the error of comment in DDLParser
Author: Yadong Qi <qiyadong2010@gmail.com>

Closes #10096 from watermen/patch-1.
2015-12-03 08:48:49 +08:00
Davies Liu 96691feae0 [SPARK-12077][SQL] change the default plan for single distinct
Use try to match the behavior for single distinct aggregation with Spark 1.5, but that's not scalable, we should be robust by default, have a flag to address performance regression for low cardinality aggregation.

cc yhuai nongli

Author: Davies Liu <davies@databricks.com>

Closes #10075 from davies/agg_15.
2015-12-01 20:17:12 -08:00
Huaxin Gao 5a8b5fdd6f [SPARK-11788][SQL] surround timestamp/date value with quotes in JDBC data source
When query the Timestamp or Date column like the following
val filtered = jdbcdf.where($"TIMESTAMP_COLUMN" >= beg && $"TIMESTAMP_COLUMN" < end)
The generated SQL query is "TIMESTAMP_COLUMN >= 2015-01-01 00:00:00.0"
It should have quote around the Timestamp/Date value such as "TIMESTAMP_COLUMN >= '2015-01-01 00:00:00.0'"

Author: Huaxin Gao <huaxing@oc0558782468.ibm.com>

Closes #9872 from huaxingao/spark-11788.
2015-12-01 15:32:57 -08:00
Nong Li 47a0abc343 [SPARK-11328][SQL] Improve error message when hitting this issue
The issue is that the output commiter is not idempotent and retry attempts will
fail because the output file already exists. It is not safe to clean up the file
as this output committer is by design not retryable. Currently, the job fails
with a confusing file exists error. This patch is a stop gap to tell the user
to look at the top of the error log for the proper message.

This is difficult to test locally as Spark is hardcoded not to retry. Manually
verified by upping the retry attempts.

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

Closes #10080 from nongli/spark-11328.
2015-12-01 15:30:21 -08:00
gatorsmile 0a7bca2da0 [SPARK-11905][SQL] Support Persist/Cache and Unpersist in Dataset APIs
Persist and Unpersist exist in both RDD and Dataframe APIs. I think they are still very critical in Dataset APIs. Not sure if my understanding is correct? If so, could you help me check if the implementation is acceptable?

Please provide your opinions. marmbrus rxin cloud-fan

Thank you very much!

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

Closes #9889 from gatorsmile/persistDS.
2015-12-01 10:38:59 -08:00
Wenchen Fan 8ddc55f1d5 [SPARK-12068][SQL] use a single column in Dataset.groupBy and count will fail
The reason is that, for a single culumn `RowEncoder`(or a single field product encoder), when we use it as the encoder for grouping key, we should also combine the grouping attributes, although there is only one grouping attribute.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10059 from cloud-fan/bug.
2015-12-01 10:22:55 -08:00
Cheng Lian 69dbe6b40d [SPARK-12046][DOC] Fixes various ScalaDoc/JavaDoc issues
This PR backports PR #10039 to master

Author: Cheng Lian <lian@databricks.com>

Closes #10063 from liancheng/spark-12046.doc-fix.master.
2015-12-01 10:21:31 -08:00
Davies Liu ecc00ec3fa fix Maven build 2015-11-30 15:42:10 -08:00
Josh Rosen 2c5dee0fb8 Revert "[SPARK-11206] Support SQL UI on the history server"
This reverts commit cc243a079b / PR #9297

I'm reverting this because it broke SQLListenerMemoryLeakSuite in the master Maven builds.

See #9991 for a discussion of why this broke the tests.
2015-11-30 13:42:35 -08:00
Davies Liu 8df584b020 [SPARK-11982] [SQL] improve performance of cartesian product
This PR improve the performance of CartesianProduct by caching the result of right plan.

After this patch, the query time of TPC-DS Q65 go down to 4 seconds from 28 minutes (420X faster).

cc nongli

Author: Davies Liu <davies@databricks.com>

Closes #9969 from davies/improve_cartesian.
2015-11-30 11:54:18 -08:00
Davies Liu 17275fa99c [SPARK-11700] [SQL] Remove thread local SQLContext in SparkPlan
In 1.6, we introduce a public API to have a SQLContext for current thread, SparkPlan should use that.

Author: Davies Liu <davies@databricks.com>

Closes #9990 from davies/leak_context.
2015-11-30 10:32:13 -08:00
CK50 2db4662fe2 [SPARK-11989][SQL] Only use commit in JDBC data source if the underlying database supports transactions
Fixes [SPARK-11989](https://issues.apache.org/jira/browse/SPARK-11989)

Author: CK50 <christian.kurz@oracle.com>
Author: Christian Kurz <christian.kurz@oracle.com>

Closes #9973 from CK50/branch-1.6_non-transactional.

(cherry picked from commit a589736a1b)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-11-30 20:09:05 +08:00
Herman van Hovell 3d28081e53 [SPARK-12024][SQL] More efficient multi-column counting.
In https://github.com/apache/spark/pull/9409 we enabled multi-column counting. The approach taken in that PR introduces a bit of overhead by first creating a row only to check if all of the columns are non-null.

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

cc yhuai

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

Closes #10015 from hvanhovell/SPARK-12024.
2015-11-29 14:13:11 -08:00
Dilip Biswal a374e20b54 [SPARK-11997] [SQL] NPE when save a DataFrame as parquet and partitioned by long column
Check for partition column null-ability while building the partition spec.

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

Closes #10001 from dilipbiswal/spark-11997.
2015-11-26 21:04:40 -08:00
Reynold Xin 10e315c28c Fix style violation for b63938a8b0 2015-11-26 19:36:43 -08:00
mariusvniekerk b63938a8b0 [SPARK-11881][SQL] Fix for postgresql fetchsize > 0
Reference: https://jdbc.postgresql.org/documentation/head/query.html#query-with-cursor
In order for PostgreSQL to honor the fetchSize non-zero setting, its Connection.autoCommit needs to be set to false. Otherwise, it will just quietly ignore the fetchSize setting.

This adds a new side-effecting dialect specific beforeFetch method that will fire before a select query is ran.

Author: mariusvniekerk <marius.v.niekerk@gmail.com>

Closes #9861 from mariusvniekerk/SPARK-11881.
2015-11-26 19:13:16 -08:00
Yanbo Liang 6f6bb0e893 [SPARK-12011][SQL] Stddev/Variance etc should support columnName as arguments
Spark SQL aggregate function:
```Java
stddev
stddev_pop
stddev_samp
variance
var_pop
var_samp
skewness
kurtosis
collect_list
collect_set
```
should support ```columnName``` as arguments like other aggregate function(max/min/count/sum).

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #9994 from yanboliang/SPARK-12011.
2015-11-26 19:00:36 -08:00
Carson Wang cc243a079b [SPARK-11206] Support SQL UI on the history server
On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution.

To support SQL UI on the history server:
1. I added an `onOtherEvent` method to the `SparkListener` trait and post all SQL related events to the same event bus.
2. Two SQL events `SparkListenerSQLExecutionStart` and `SparkListenerSQLExecutionEnd` are defined in the sql module.
3. The new SQL events are written to event log using Jackson.
4.  A new trait `SparkHistoryListenerFactory` is added to allow the history server to feed events to the SQL history listener. The SQL implementation is loaded at runtime using `java.util.ServiceLoader`.

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

Closes #9297 from carsonwang/SqlHistoryUI.
2015-11-25 15:13:13 -08:00
Reynold Xin ecac283545 Fix Aggregator documentation (rename present to finish). 2015-11-25 13:45:41 -08:00
Davies Liu dc1d324fdf [SPARK-11969] [SQL] [PYSPARK] visualization of SQL query for pyspark
Currently, we does not have visualization for SQL query from Python, this PR fix that.

cc zsxwing

Author: Davies Liu <davies@databricks.com>

Closes #9949 from davies/pyspark_sql_ui.
2015-11-25 11:11:39 -08:00
felixcheung faabdfa2bd [SPARK-11984][SQL][PYTHON] Fix typos in doc for pivot for scala and python
Author: felixcheung <felixcheung_m@hotmail.com>

Closes #9967 from felixcheung/pypivotdoc.
2015-11-25 10:36:35 -08:00
Reynold Xin a0f1a11837 [SPARK-11981][SQL] Move implementations of methods back to DataFrame from Queryable
Also added show methods to Dataset.

Author: Reynold Xin <rxin@databricks.com>

Closes #9964 from rxin/SPARK-11981.
2015-11-25 01:03:18 -08:00
gatorsmile 2610e06124 [SPARK-11970][SQL] Adding JoinType into JoinWith and support Sample in Dataset API
Except inner join, maybe the other join types are also useful when users are using the joinWith function. Thus, added the joinType into the existing joinWith call in Dataset APIs.

Also providing another joinWith interface for the cartesian-join-like functionality.

Please provide your opinions. marmbrus rxin cloud-fan Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9921 from gatorsmile/joinWith.
2015-11-25 01:02:36 -08:00
Reynold Xin 151d7c2baf [SPARK-10621][SQL] Consistent naming for functions in SQL, Python, Scala
Author: Reynold Xin <rxin@databricks.com>

Closes #9948 from rxin/SPARK-10621.
2015-11-24 21:30:53 -08:00
Reynold Xin 4d6bbbc03d [SPARK-11947][SQL] Mark deprecated methods with "This will be removed in Spark 2.0."
Also fixed some documentation as I saw them.

Author: Reynold Xin <rxin@databricks.com>

Closes #9930 from rxin/SPARK-11947.
2015-11-24 18:58:55 -08:00
Reynold Xin 25bbd3c16e [SPARK-11967][SQL] Consistent use of varargs for multiple paths in DataFrameReader
This patch makes it consistent to use varargs in all DataFrameReader methods, including Parquet, JSON, text, and the generic load function.

Also added a few more API tests for the Java API.

Author: Reynold Xin <rxin@databricks.com>

Closes #9945 from rxin/SPARK-11967.
2015-11-24 18:16:07 -08:00
gatorsmile 238ae51b66 [SPARK-11914][SQL] Support coalesce and repartition in Dataset APIs
This PR is to provide two common `coalesce` and `repartition` in Dataset APIs.

After reading the comments of SPARK-9999, I am unclear about the plan for supporting re-partitioning in Dataset APIs. Currently, both RDD APIs and Dataframe APIs provide users such a flexibility to control the number of partitions.

In most traditional RDBMS, they expose the number of partitions, the partitioning columns, the table partitioning methods to DBAs for performance tuning and storage planning. Normally, these parameters could largely affect the query performance. Since the actual performance depends on the workload types, I think it is almost impossible to automate the discovery of the best partitioning strategy for all the scenarios.

I am wondering if Dataset APIs are planning to hide these APIs from users? Feel free to reject my PR if it does not match the plan.

Thank you for your answers. marmbrus rxin cloud-fan

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9899 from gatorsmile/coalesce.
2015-11-24 15:54:10 -08:00
Reynold Xin 34ca392da7 Added a line of comment to explain why the extra sort exists in pivot. 2015-11-24 14:51:01 -08:00
Reynold Xin f315272279 [SPARK-11946][SQL] Audit pivot API for 1.6.
Currently pivot's signature looks like

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

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

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

I also made similar changes for Python.

Author: Reynold Xin <rxin@databricks.com>

Closes #9929 from rxin/SPARK-11946.
2015-11-24 12:54:37 -08:00
Wenchen Fan e5aaae6e11 [SPARK-11942][SQL] fix encoder life cycle for CoGroup
we should pass in resolved encodera to logical `CoGroup` and bind them in physical `CoGroup`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9928 from cloud-fan/cogroup.
2015-11-24 09:28:39 -08:00
Xiu Guo 12eea834d7 [SPARK-11897][SQL] Add @scala.annotations.varargs to sql functions
Author: Xiu Guo <xguo27@gmail.com>

Closes #9918 from xguo27/SPARK-11897.
2015-11-24 00:07:40 -08:00
Mikhail Bautin 4021a28ac3 [SPARK-10707][SQL] Fix nullability computation in union output
Author: Mikhail Bautin <mbautin@gmail.com>

Closes #9308 from mbautin/SPARK-10707.
2015-11-23 22:26:08 -08:00
Reynold Xin 8d57524662 [SPARK-11933][SQL] Rename mapGroup -> mapGroups and flatMapGroup -> flatMapGroups.
Based on feedback from Matei, this is more consistent with mapPartitions in Spark.

Also addresses some of the cleanups from a previous commit that renames the type variables.

Author: Reynold Xin <rxin@databricks.com>

Closes #9919 from rxin/SPARK-11933.
2015-11-23 22:22:15 -08:00
Wenchen Fan 946b406519 [SPARK-11913][SQL] support typed aggregate with complex buffer schema
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9898 from cloud-fan/agg.
2015-11-23 10:39:33 -08:00
Reynold Xin ff442bbcff [SPARK-11899][SQL] API audit for GroupedDataset.
1. Renamed map to mapGroup, flatMap to flatMapGroup.
2. Renamed asKey -> keyAs.
3. Added more documentation.
4. Changed type parameter T to V on GroupedDataset.
5. Added since versions for all functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #9880 from rxin/SPARK-11899.
2015-11-21 15:00:37 -08:00
Reynold Xin 596710268e [SPARK-11901][SQL] API audit for Aggregator.
Author: Reynold Xin <rxin@databricks.com>

Closes #9882 from rxin/SPARK-11901.
2015-11-21 00:54:18 -08:00
Michael Armbrust 968acf3bd9 [SPARK-11889][SQL] Fix type inference for GroupedDataset.agg in REPL
In this PR I delete a method that breaks type inference for aggregators (only in the REPL)

The error when this method is present is:
```
<console>:38: error: missing parameter type for expanded function ((x$2) => x$2._2)
              ds.groupBy(_._1).agg(sum(_._2), sum(_._3)).collect()
```

Author: Michael Armbrust <michael@databricks.com>

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

Author: Nong Li <nong@databricks.com>

Closes #9845 from nongli/spark-11787.
2015-11-20 15:30:53 -08:00
Jean-Baptiste Onofré 03ba56d78f [SPARK-11716][SQL] UDFRegistration just drops the input type when re-creating the UserDefinedFunction
https://issues.apache.org/jira/browse/SPARK-11716

This is one is #9739 and a regression test. When commit it, please make sure the author is jbonofre.

You can find the original PR at https://github.com/apache/spark/pull/9739

closes #9739

Author: Jean-Baptiste Onofré <jbonofre@apache.org>
Author: Yin Huai <yhuai@databricks.com>

Closes #9868 from yhuai/SPARK-11716.
2015-11-20 14:45:40 -08:00
gatorsmile bef361c589 [SPARK-11876][SQL] Support printSchema in DataSet API
DataSet APIs look great! However, I am lost when doing multiple level joins.  For example,
```
val ds1 = Seq(("a", 1), ("b", 2)).toDS().as("a")
val ds2 = Seq(("a", 1), ("b", 2)).toDS().as("b")
val ds3 = Seq(("a", 1), ("b", 2)).toDS().as("c")

ds1.joinWith(ds2, $"a._2" === $"b._2").as("ab").joinWith(ds3, $"ab._1._2" === $"c._2").printSchema()
```

The printed schema is like
```
root
 |-- _1: struct (nullable = true)
 |    |-- _1: struct (nullable = true)
 |    |    |-- _1: string (nullable = true)
 |    |    |-- _2: integer (nullable = true)
 |    |-- _2: struct (nullable = true)
 |    |    |-- _1: string (nullable = true)
 |    |    |-- _2: integer (nullable = true)
 |-- _2: struct (nullable = true)
 |    |-- _1: string (nullable = true)
 |    |-- _2: integer (nullable = true)
```

Personally, I think we need the printSchema function. Sometimes, I do not know how to specify the column, especially when their data types are mixed. For example, if I want to write the following select for the above multi-level join, I have to know the schema:
```
newDS.select(expr("_1._2._2 + 1").as[Int]).collect()
```

marmbrus rxin cloud-fan  Do you have the same feeling?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9855 from gatorsmile/printSchemaDataSet.
2015-11-20 11:20:47 -08:00
Dilip Biswal 7ee7d5a3c4 [SPARK-11544][SQL][TEST-HADOOP1.0] sqlContext doesn't use PathFilter
Apply the user supplied pathfilter while retrieving the files from fs.

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

Closes #9830 from dilipbiswal/spark-11544.
2015-11-19 19:46:10 -08:00
Reynold Xin 014c0f7a9d [SPARK-11858][SQL] Move sql.columnar into sql.execution.
In addition, tightened visibility of a lot of classes in the columnar package from private[sql] to private[columnar].

Author: Reynold Xin <rxin@databricks.com>

Closes #9842 from rxin/SPARK-11858.
2015-11-19 14:48:18 -08:00
Huaxin Gao 4700074530 [SPARK-11778][SQL] parse table name before it is passed to lookupRelation
Fix a bug in DataFrameReader.table (table with schema name such as "db_name.table" doesn't work)
Use SqlParser.parseTableIdentifier to parse the table name before lookupRelation.

Author: Huaxin Gao <huaxing@oc0558782468.ibm.com>

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9726 from cloud-fan/follow.
2015-11-19 12:54:25 -08:00
gatorsmile 7d4aba1872 [SPARK-11848][SQL] Support EXPLAIN in DataSet APIs
When debugging DataSet API, I always need to print the logical and physical plans.

I am wondering if we should provide a simple API for EXPLAIN?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9832 from gatorsmile/explainDS.
2015-11-19 12:46:36 -08:00
gatorsmile 276a7e1302 [SPARK-11633][SQL] LogicalRDD throws TreeNode Exception : Failed to Copy Node
When handling self joins, the implementation did not consider the case insensitivity of HiveContext. It could cause an exception as shown in the JIRA:
```
TreeNodeException: Failed to copy node.
```

The fix is low risk. It avoids unnecessary attribute replacement. It should not affect the existing behavior of self joins. Also added the test case to cover this case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9762 from gatorsmile/joinMakeCopy.
2015-11-19 12:45:04 -08:00
Yin Huai 9c0654d36c Revert "[SPARK-11544][SQL] sqlContext doesn't use PathFilter"
This reverts commit 54db797025.
2015-11-18 18:41:40 -08:00
Nong Li 6d0848b53b [SPARK-11787][SQL] Improve Parquet scan performance when using flat schemas.
This patch adds an alternate to the Parquet RecordReader from the parquet-mr project
that is much faster for flat schemas. Instead of using the general converter mechanism
from parquet-mr, this directly uses the lower level APIs from parquet-columnar and a
customer RecordReader that directly assembles into UnsafeRows.

This is optionally disabled and only used for supported schemas.

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

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

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

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

Author: Nong Li <nong@databricks.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

Closes #9602 from marmbrus/dataset-replClasses.
2015-11-18 16:48:09 -08:00
Dilip Biswal 54db797025 [SPARK-11544][SQL] sqlContext doesn't use PathFilter
Apply the user supplied pathfilter while retrieving the files from fs.

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

Closes #9652 from dilipbiswal/spark-11544.
2015-11-18 14:05:18 -08:00
Davies Liu 94624eacb0 [SPARK-11739][SQL] clear the instantiated SQLContext
Currently, if the first SQLContext is not removed after stopping SparkContext, a SQLContext could set there forever. This patch make this more robust.

Author: Davies Liu <davies@databricks.com>

Closes #9706 from davies/clear_context.
2015-11-18 11:53:28 -08:00
Yin Huai 6f99522d13 [SPARK-11792] [SQL] [FOLLOW-UP] Change SizeEstimation to KnownSizeEstimation and make estimatedSize return Long instead of Option[Long]
https://issues.apache.org/jira/browse/SPARK-11792

The main changes include:
* Renaming `SizeEstimation` to `KnownSizeEstimation`. Hopefully this new name has more information.
* Making `estimatedSize` return `Long` instead of `Option[Long]`.
* In `UnsaveHashedRelation`, `estimatedSize` will delegate the work to `SizeEstimator` if we have not created a `BytesToBytesMap`.

Since we will put `UnsaveHashedRelation` to `BlockManager`, it is generally good to let it provide a more accurate size estimation. Also, if we do not put `BytesToBytesMap` directly into `BlockerManager`, I feel it is not really necessary to make `BytesToBytesMap` extends `KnownSizeEstimation`.

Author: Yin Huai <yhuai@databricks.com>

Closes #9813 from yhuai/SPARK-11792-followup.
2015-11-18 11:49:12 -08:00
Wenchen Fan dbf428c87a [SPARK-11795][SQL] combine grouping attributes into a single NamedExpression
we use `ExpressionEncoder.tuple` to build the result encoder, which assumes the input encoder should point to a struct type field if it’s non-flat.
However, our keyEncoder always point to a flat field/fields: `groupingAttributes`, we should combine them into a single `NamedExpression`.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9792 from cloud-fan/agg.
2015-11-18 10:33:17 -08:00
Wenchen Fan cffb899c43 [SPARK-11803][SQL] fix Dataset self-join
When we resolve the join operator, we may change the output of right side if self-join is detected. So in `Dataset.joinWith`, we should resolve the join operator first, and then get the left output and right output from it, instead of using `left.output` and `right.output` directly.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9806 from cloud-fan/self-join.
2015-11-18 10:15:50 -08:00
somideshmukh b8f4379ba1 [SPARK-10946][SQL] JDBC - Use Statement.executeUpdate instead of PreparedStatement.executeUpdate for DDLs
New changes with JDBCRDD

Author: somideshmukh <somilde@us.ibm.com>

Closes #9733 from somideshmukh/SomilBranch-1.1.
2015-11-18 08:51:01 +00:00
Yin Huai 1714350bdd [SPARK-11792][SQL] SizeEstimator cannot provide a good size estimation of UnsafeHashedRelations
https://issues.apache.org/jira/browse/SPARK-11792

Right now, SizeEstimator will "think" a small UnsafeHashedRelation is several GBs.

Author: Yin Huai <yhuai@databricks.com>

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

Author: Reynold Xin <rxin@databricks.com>

Closes #9789 from rxin/SPARK-11802.
2015-11-18 00:09:29 -08:00
Reynold Xin 91f4b6f2db [SPARK-11797][SQL] collect, first, and take should use encoders for serialization
They were previously using Spark's default serializer for serialization.

Author: Reynold Xin <rxin@databricks.com>

Closes #9787 from rxin/SPARK-11797.
2015-11-17 21:40:58 -08:00
Reynold Xin ed8d1531f9 [SPARK-11793][SQL] Dataset should set the resolved encoders internally for maps.
I also wrote a test case -- but unfortunately the test case is not working due to SPARK-11795.

Author: Reynold Xin <rxin@databricks.com>

Closes #9784 from rxin/SPARK-11503.
2015-11-17 19:02:44 -08:00
Davies Liu 5aca6ad00c [SPARK-11767] [SQL] limit the size of caced batch
Currently the size of cached batch in only controlled by `batchSize` (default value is 10000), which does not work well with the size of serialized columns (for example, complex types). The memory used to build the batch is not accounted, it's easy to OOM (especially after unified memory management).

This PR introduce a hard limit as 4M for total columns (up to 50 columns of uncompressed primitive columns).

This also change the way to grow buffer, double it each time, then trim it once finished.

cc liancheng

Author: Davies Liu <davies@databricks.com>

Closes #9760 from davies/cache_limit.
2015-11-17 12:50:01 -08:00
Wenchen Fan d925149664 [SPARK-10186][SQL] support postgre array type in JDBCRDD
Add ARRAY support to `PostgresDialect`.

Nested ARRAY is not allowed for now because it's hard to get the array dimension info. See http://stackoverflow.com/questions/16619113/how-to-get-array-base-type-in-postgres-via-jdbc

Thanks for the initial work from mariusvniekerk !

Close https://github.com/apache/spark/pull/9137

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9662 from cloud-fan/postgre.
2015-11-17 11:29:02 -08:00
Wenchen Fan fd14936be7 [SPARK-11625][SQL] add java test for typed aggregate
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9591 from cloud-fan/agg-test.
2015-11-16 15:32:49 -08:00
Zee Chen 985b38dd2f [SPARK-11390][SQL] Query plan with/without filterPushdown indistinguishable
…ishable

Propagate pushed filters to PhyicalRDD in DataSourceStrategy.apply

Author: Zee Chen <zeechen@us.ibm.com>

Closes #9679 from zeocio/spark-11390.
2015-11-16 14:21:28 -08:00
hyukjinkwon e388b39d10 [SPARK-11692][SQL] Support for Parquet logical types, JSON and BSON (embedded types)
Parquet supports some JSON and BSON datatypes. They are represented as binary for BSON and string (UTF-8) for JSON internally.

I searched a bit and found Apache drill also supports both in this way, [link](https://drill.apache.org/docs/parquet-format/).

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

Closes #9658 from HyukjinKwon/SPARK-11692.
2015-11-16 21:59:33 +08:00
hyukjinkwon 7f8eb3bf6e [SPARK-11044][SQL] Parquet writer version fixed as version1
https://issues.apache.org/jira/browse/SPARK-11044

Spark writes a parquet file only with writer version1 ignoring the writer version given by user.

So, in this PR, it keeps the writer version if given or sets version1 as default.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: HyukjinKwon <gurwls223@gmail.com>

Closes #9060 from HyukjinKwon/SPARK-11044.
2015-11-16 21:30:10 +08:00
Reynold Xin 42de5253f3 [SPARK-11745][SQL] Enable more JSON parsing options
This patch adds the following options to the JSON data source, for dealing with non-standard JSON files:
* `allowComments` (default `false`): ignores Java/C++ style comment in JSON records
* `allowUnquotedFieldNames` (default `false`): allows unquoted JSON field names
* `allowSingleQuotes` (default `true`): allows single quotes in addition to double quotes
* `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers (e.g. 00012)

To avoid passing a lot of options throughout the json package, I introduced a new JSONOptions case class to define all JSON config options.

Also updated documentation to explain these options.

Scala

![screen shot 2015-11-15 at 6 12 12 pm](https://cloud.githubusercontent.com/assets/323388/11172965/e3ace6ec-8bc4-11e5-805e-2d78f80d0ed6.png)

Python

![screen shot 2015-11-15 at 6 11 28 pm](https://cloud.githubusercontent.com/assets/323388/11172964/e23ed6ee-8bc4-11e5-8216-312f5983acd5.png)

Author: Reynold Xin <rxin@databricks.com>

Closes #9724 from rxin/SPARK-11745.
2015-11-16 00:06:14 -08:00
gatorsmile b58765caa6 [SPARK-9928][SQL] Removal of LogicalLocalTable
LogicalLocalTable in ExistingRDD.scala is replaced by localRelation in LocalRelation.scala?

Do you know any reason why we still keep this class?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9717 from gatorsmile/LogicalLocalTable.
2015-11-15 21:10:46 -08:00
Reynold Xin d22fc10887 [SPARK-11734][SQL] Rename TungstenProject -> Project, TungstenSort -> Sort
I didn't remove the old Sort operator, since we still use it in randomized tests. I moved it into test module and renamed it ReferenceSort.

Author: Reynold Xin <rxin@databricks.com>

Closes #9700 from rxin/SPARK-11734.
2015-11-15 10:33:53 -08:00
nitin goyal c939c70ac1 [SPARK-7970] Skip closure cleaning for SQL operations
Also introduces new spark private API in RDD.scala with name 'mapPartitionsInternal' which doesn't closure cleans the RDD elements.

Author: nitin goyal <nitin.goyal@guavus.com>
Author: nitin.goyal <nitin.goyal@guavus.com>

Closes #9253 from nitin2goyal/master.
2015-11-13 18:09:08 -08:00
Wenchen Fan d7b2b97ad6 [SPARK-11727][SQL] Split ExpressionEncoder into FlatEncoder and ProductEncoder
also add more tests for encoders, and fix bugs that I found:

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

other issues that still in investigation:

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

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

Author: Wenchen Fan <wenchen@databricks.com>

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9688 from cloud-fan/follow.
2015-11-13 11:13:09 -08:00
Yin Huai 7b5d9051cf [SPARK-11678][SQL] Partition discovery should stop at the root path of the table.
https://issues.apache.org/jira/browse/SPARK-11678

The change of this PR is to pass root paths of table to the partition discovery logic. So, the process of partition discovery stops at those root paths instead of going all the way to the root path of the file system.

Author: Yin Huai <yhuai@databricks.com>

Closes #9651 from yhuai/SPARK-11678.
2015-11-13 18:36:56 +08:00
Michael Armbrust 41bbd23004 [SPARK-11654][SQL] add reduce to GroupedDataset
This PR adds a new method, `reduce`, to `GroupedDataset`, which allows similar operations to `reduceByKey` on a traditional `PairRDD`.

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

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

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

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

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

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

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

Author: JihongMa <linlin200605@gmail.com>

Closes #9380 from JihongMA/SPARK-11420.
2015-11-12 13:47:34 -08:00
hyukjinkwon f5a9526fec [SPARK-10113][SQL] Explicit error message for unsigned Parquet logical types
Parquet supports some unsigned datatypes. However, Since Spark does not support unsigned datatypes, it needs to emit an exception with a clear message rather then with the one saying illegal datatype.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9646 from HyukjinKwon/SPARK-10113.
2015-11-12 12:29:50 -08:00
Reynold Xin 30e7433643 [SPARK-11673][SQL] Remove the normal Project physical operator (and keep TungstenProject)
Also make full outer join being able to produce UnsafeRows.

Author: Reynold Xin <rxin@databricks.com>

Closes #9643 from rxin/SPARK-11673.
2015-11-12 08:14:08 -08:00
Yin Huai 14cf753704 [SPARK-11661][SQL] Still pushdown filters returned by unhandledFilters.
https://issues.apache.org/jira/browse/SPARK-11661

Author: Yin Huai <yhuai@databricks.com>

Closes #9634 from yhuai/unhandledFilters.
2015-11-12 16:47:00 +08:00
Reynold Xin e49e723392 [SPARK-11675][SQL] Remove shuffle hash joins.
Author: Reynold Xin <rxin@databricks.com>

Closes #9645 from rxin/SPARK-11675.
2015-11-11 19:32:52 -08:00
Andrew Ray b8ff6888e7 [SPARK-8992][SQL] Add pivot to dataframe api
This adds a pivot method to the dataframe api.

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

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

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

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

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

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

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

Closes #7841 from aray/sql-pivot.
2015-11-11 16:23:24 -08:00
Reynold Xin a9a6b80c71 [SPARK-11645][SQL] Remove OpenHashSet for the old aggregate.
Author: Reynold Xin <rxin@databricks.com>

Closes #9621 from rxin/SPARK-11645.
2015-11-11 12:48:51 -08:00
Reynold Xin df97df2b39 [SPARK-11644][SQL] Remove the option to turn off unsafe and codegen.
Author: Reynold Xin <rxin@databricks.com>

Closes #9618 from rxin/SPARK-11644.
2015-11-11 12:47:02 -08:00
Wenchen Fan e71ba56586 [SQL][MINOR] remove newLongEncoder in functions
it may shadows the one from implicits in some case.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9629 from cloud-fan/minor.
2015-11-11 11:04:04 -08:00
Wenchen Fan ec2b807212 [SPARK-11564][SQL][FOLLOW-UP] clean up java tuple encoder
We need to support custom classes like java beans and combine them into tuple, and it's very hard to do it with the  TypeTag-based approach.
We should keep only the compose-based way to create tuple encoder.

This PR also move `Encoder` to `org.apache.spark.sql`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9567 from cloud-fan/java.
2015-11-11 10:52:23 -08:00
Wenchen Fan 9c57bc0efc [SPARK-11656][SQL] support typed aggregate in project list
insert `aEncoder` like we do in `agg`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9630 from cloud-fan/select.
2015-11-11 10:21:53 -08:00
Wenchen Fan c964fc1015 [SQL][MINOR] rename present to finish in Aggregator
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9617 from cloud-fan/tmp.
2015-11-11 10:19:09 -08:00
hyukjinkwon 1bc41125ee [SPARK-11500][SQL] Not deterministic order of columns when using merging schemas.
https://issues.apache.org/jira/browse/SPARK-11500

As filed in SPARK-11500, if merging schemas is enabled, the order of files to touch is a matter which might affect the ordering of the output columns.

This was mostly because of the use of `Set` and `Map` so I replaced them to `LinkedHashSet` and `LinkedHashMap` to keep the insertion order.

Also, I changed `reduceOption` to `reduceLeftOption`, and replaced the order of `filesToTouch` from `metadataStatuses ++ commonMetadataStatuses ++ needMerged` to  `needMerged ++ metadataStatuses ++ commonMetadataStatuses` in order to touch the part-files first which always have the schema in footers whereas the others might not exist.

One nit is, If merging schemas is not enabled, but when multiple files are given, there is no guarantee of the output order, since there might not be a summary file for the first file, which ends up putting ahead the columns of the other files.

However, I thought this should be okay since disabling merging schemas means (assumes) all the files have the same schemas.

In addition, in the test code for this, I only checked the names of fields.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9517 from HyukjinKwon/SPARK-11500.
2015-11-11 16:46:04 +08:00
Marc Prud'hommeaux 745e45d5ff [MINOR] License header formatting fix
The header wasn't indented properly.

Author: Marc Prud'hommeaux <mwp1@cornell.edu>

Closes #9312 from mprudhom/patch-1.
2015-11-10 16:57:12 -08:00
tedyu 9009175416 [SPARK-11615] Drop @VisibleForTesting annotation
See http://search-hadoop.com/m/q3RTtjpe8r1iRbTj2 for discussion.

Summary: addition of VisibleForTesting annotation resulted in spark-shell malfunctioning.

Author: tedyu <yuzhihong@gmail.com>

Closes #9585 from tedyu/master.
2015-11-10 16:52:59 -08:00
Yin Huai 3121e78168 [SPARK-9830][SPARK-11641][SQL][FOLLOW-UP] Remove AggregateExpression1 and update toString of Exchange
https://issues.apache.org/jira/browse/SPARK-9830

This is the follow-up pr for https://github.com/apache/spark/pull/9556 to address davies' comments.

Author: Yin Huai <yhuai@databricks.com>

Closes #9607 from yhuai/removeAgg1-followup.
2015-11-10 16:25:22 -08:00
Michael Armbrust 724cf7a38c [SPARK-11616][SQL] Improve toString for Dataset
Author: Michael Armbrust <michael@databricks.com>

Closes #9586 from marmbrus/dataset-toString.
2015-11-10 14:30:19 -08:00
Nong Li 87aedc48c0 [SPARK-10371][SQL] Implement subexpr elimination for UnsafeProjections
This patch adds the building blocks for codegening subexpr elimination and implements
it end to end for UnsafeProjection. The building blocks can be used to do the same thing
for other operators.

It introduces some utilities to compute common sub expressions. Expressions can be added to
this data structure. The expr and its children will be recursively matched against existing
expressions (ones previously added) and grouped into common groups. This is built using
the existing `semanticEquals`. It does not understand things like commutative or associative
expressions. This can be done as future work.

After building this data structure, the codegen process takes advantage of it by:
  1. Generating a helper function in the generated class that computes the common
     subexpression. This is done for all common subexpressions that have at least
     two occurrences and the expression tree is sufficiently complex.
  2. When generating the apply() function, if the helper function exists, call that
     instead of regenerating the expression tree. Repeated calls to the helper function
     shortcircuit the evaluation logic.

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

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

Closes #9480 from nongli/spark-10371.
2015-11-10 11:28:53 -08:00
Wenchen Fan 53600854c2 [SPARK-11590][SQL] use native json_tuple in lateral view
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9562 from cloud-fan/json-tuple.
2015-11-10 11:21:31 -08:00
Wenchen Fan dfcfcbcc04 [SPARK-11578][SQL][FOLLOW-UP] complete the user facing api for typed aggregation
Currently the user facing api for typed aggregation has some limitations:

* the customized typed aggregation must be the first of aggregation list
* the customized typed aggregation can only use long as buffer type
* the customized typed aggregation can only use flat type as result type

This PR tries to remove these limitations.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9599 from cloud-fan/agg.
2015-11-10 11:14:25 -08:00
Yin Huai e0701c7560 [SPARK-9830][SQL] Remove AggregateExpression1 and Aggregate Operator used to evaluate AggregateExpression1s
https://issues.apache.org/jira/browse/SPARK-9830

This PR contains the following main changes.
* Removing `AggregateExpression1`.
* Removing `Aggregate` operator, which is used to evaluate `AggregateExpression1`.
* Removing planner rule used to plan `Aggregate`.
* Linking `MultipleDistinctRewriter` to analyzer.
* Renaming `AggregateExpression2` to `AggregateExpression` and `AggregateFunction2` to `AggregateFunction`.
* Updating places where we create aggregate expression. The way to create aggregate expressions is `AggregateExpression(aggregateFunction, mode, isDistinct)`.
* Changing `val`s in `DeclarativeAggregate`s that touch children of this function to `lazy val`s (when we create aggregate expression in DataFrame API, children of an aggregate function can be unresolved).

Author: Yin Huai <yhuai@databricks.com>

Closes #9556 from yhuai/removeAgg1.
2015-11-10 11:06:29 -08:00
Reynold Xin 675c7e723c [SPARK-11564][SQL] Fix documentation for DataFrame.take/collect
Author: Reynold Xin <rxin@databricks.com>

Closes #9557 from rxin/SPARK-11564-1.
2015-11-09 16:22:15 -08:00
Michael Armbrust 9c740a9ddf [SPARK-11578][SQL] User API for Typed Aggregation
This PR adds a new interface for user-defined aggregations, that can be used in `DataFrame` and `Dataset` operations to take all of the elements of a group and reduce them to a single value.

For example, the following aggregator extracts an `int` from a specific class and adds them up:

```scala
  case class Data(i: Int)

  val customSummer =  new Aggregator[Data, Int, Int] {
    def prepare(d: Data) = d.i
    def reduce(l: Int, r: Int) = l + r
    def present(r: Int) = r
  }.toColumn()

  val ds: Dataset[Data] = ...
  val aggregated = ds.select(customSummer)
```

By using helper functions, users can make a generic `Aggregator` that works on any input type:

```scala
/** An `Aggregator` that adds up any numeric type returned by the given function. */
class SumOf[I, N : Numeric](f: I => N) extends Aggregator[I, N, N] with Serializable {
  val numeric = implicitly[Numeric[N]]
  override def zero: N = numeric.zero
  override def reduce(b: N, a: I): N = numeric.plus(b, f(a))
  override def present(reduction: N): N = reduction
}

def sum[I, N : Numeric : Encoder](f: I => N): TypedColumn[I, N] = new SumOf(f).toColumn
```

These aggregators can then be used alongside other built-in SQL aggregations.

```scala
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
ds
  .groupBy(_._1)
  .agg(
    sum(_._2),                // The aggregator defined above.
    expr("sum(_2)").as[Int],  // A built-in dynatically typed aggregation.
    count("*"))               // A built-in statically typed aggregation.
  .collect()

res0: ("a", 30, 30, 2L), ("b", 3, 3, 2L), ("c", 1, 1, 1L)
```

The current implementation focuses on integrating this into the typed API, but currently only supports running aggregations that return a single long value as explained in `TypedAggregateExpression`.  This will be improved in a followup PR.

Author: Michael Armbrust <michael@databricks.com>

Closes #9555 from marmbrus/dataset-useragg.
2015-11-09 16:11:00 -08:00
Wenchen Fan fcb57e9c73 [SPARK-11564][SQL][FOLLOW-UP] improve java api for GroupedDataset
created `MapGroupFunction`, `FlatMapGroupFunction`, `CoGroupFunction`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9564 from cloud-fan/map.
2015-11-09 15:16:47 -08:00
Nick Buroojy f138cb8733 [SPARK-9301][SQL] Add collect_set and collect_list aggregate functions
For now they are thin wrappers around the corresponding Hive UDAFs.

One limitation with these in Hive 0.13.0 is they only support aggregating primitive types.

I chose snake_case here instead of camelCase because it seems to be used in the majority of the multi-word fns.

Do we also want to add these to `functions.py`?

This approach was recommended here: https://github.com/apache/spark/pull/8592#issuecomment-154247089

marmbrus rxin

Author: Nick Buroojy <nick.buroojy@civitaslearning.com>

Closes #9526 from nburoojy/nick/udaf-alias.

(cherry picked from commit a6ee4f989d)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-11-09 14:30:52 -08:00
Wenchen Fan d8b50f7029 [SPARK-11453][SQL] append data to partitioned table will messes up the result
The reason is that:

1. For partitioned hive table, we will move the partitioned columns after data columns. (e.g. `<a: Int, b: Int>` partition by `a` will become `<b: Int, a: Int>`)
2. When append data to table, we use position to figure out how to match input columns to table's columns.

So when we append data to partitioned table, we will match wrong columns between input and table. A solution is reordering the input columns before match by position, like what we did for [`InsertIntoHadoopFsRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala#L101-L105)

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9408 from cloud-fan/append.
2015-11-08 21:01:53 -08:00
Reynold Xin 97b7080cf2 [SPARK-11564][SQL] Dataset Java API audit
A few changes:

1. Removed fold, since it can be confusing for distributed collections.
2. Created specific interfaces for each Dataset function (e.g. MapFunction, ReduceFunction, MapPartitionsFunction)
3. Added more documentation and test cases.

The other thing I'm considering doing is to have a "collector" interface for FlatMapFunction and MapPartitionsFunction, similar to MapReduce's map function.

Author: Reynold Xin <rxin@databricks.com>

Closes #9531 from rxin/SPARK-11564.
2015-11-08 20:57:09 -08:00
Wenchen Fan b2d195e137 [SPARK-11554][SQL] add map/flatMap to GroupedDataset
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9521 from cloud-fan/map.
2015-11-08 12:59:35 -08:00
Liang-Chi Hsieh 4b69a42eda [SPARK-11362] [SQL] Use Spark BitSet in BroadcastNestedLoopJoin
JIRA: https://issues.apache.org/jira/browse/SPARK-11362

We use scala.collection.mutable.BitSet in BroadcastNestedLoopJoin now. We should use Spark's BitSet.

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

Closes #9316 from viirya/use-spark-bitset.
2015-11-07 19:44:45 -08:00
Herman van Hovell 6d0ead322e [SPARK-9241][SQL] Supporting multiple DISTINCT columns (2) - Rewriting Rule
The second PR for SPARK-9241, this adds support for multiple distinct columns to the new aggregation code path.

This PR solves the multiple DISTINCT column problem by rewriting these Aggregates into an Expand-Aggregate-Aggregate combination. See the [JIRA ticket](https://issues.apache.org/jira/browse/SPARK-9241) for some information on this. The advantages over the - competing - [first PR](https://github.com/apache/spark/pull/9280) are:
- This can use the faster TungstenAggregate code path.
- It is impossible to OOM due to an ```OpenHashSet``` allocating to much memory. However, this will multiply the number of input rows by the number of distinct clauses (plus one), and puts a lot more memory pressure on the aggregation code path itself.

The location of this Rule is a bit funny, and should probably change when the old aggregation path is changed.

cc yhuai - Could you also tell me where to add tests for this?

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

Closes #9406 from hvanhovell/SPARK-9241-rewriter.
2015-11-06 16:04:20 -08:00
Wenchen Fan 7e9a9e603a [SPARK-11269][SQL] Java API support & test cases for Dataset
This simply brings https://github.com/apache/spark/pull/9358 up-to-date.

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

Closes #9528 from rxin/dataset-java.
2015-11-06 15:37:07 -08:00
Reynold Xin 3a652f691b [SPARK-11561][SQL] Rename text data source's column name to value.
Author: Reynold Xin <rxin@databricks.com>

Closes #9527 from rxin/SPARK-11561.
2015-11-06 14:47:41 -08:00
Herman van Hovell f328fedafd [SPARK-11450] [SQL] Add Unsafe Row processing to Expand
This PR enables the Expand operator to process and produce Unsafe Rows.

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

Closes #9414 from hvanhovell/SPARK-11450.
2015-11-06 12:21:53 -08:00
Yin Huai 8211aab079 [SPARK-9858][SQL] Add an ExchangeCoordinator to estimate the number of post-shuffle partitions for aggregates and joins (follow-up)
https://issues.apache.org/jira/browse/SPARK-9858

This PR is the follow-up work of https://github.com/apache/spark/pull/9276. It addresses JoshRosen's comments.

Author: Yin Huai <yhuai@databricks.com>

Closes #9453 from yhuai/numReducer-followUp.
2015-11-06 11:13:51 -08:00
Reynold Xin bc5d6c0389 [SPARK-11541][SQL] Break JdbcDialects.scala into multiple files and mark various dialects as private.
Author: Reynold Xin <rxin@databricks.com>

Closes #9511 from rxin/SPARK-11541.
2015-11-05 22:03:26 -08:00
Michael Armbrust 363a476c3f [SPARK-11528] [SQL] Typed aggregations for Datasets
This PR adds the ability to do typed SQL aggregations.  We will likely also want to provide an interface to allow users to do aggregations on objects, but this is deferred to another PR.

```scala
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
ds.groupBy(_._1).agg(sum("_2").as[Int]).collect()

res0: Array(("a", 30), ("b", 3), ("c", 1))
```

Author: Michael Armbrust <michael@databricks.com>

Closes #9499 from marmbrus/dataset-agg.
2015-11-05 21:42:32 -08:00
Davies Liu eec74ba8bd [SPARK-7542][SQL] Support off-heap index/sort buffer
This brings the support of off-heap memory for array inside BytesToBytesMap and InMemorySorter, then we could allocate all the memory from off-heap for execution.

Closes #8068

Author: Davies Liu <davies@databricks.com>

Closes #9477 from davies/unsafe_timsort.
2015-11-05 19:02:18 -08:00
Reynold Xin 3cc2c053b5 [SPARK-11540][SQL] API audit for QueryExecutionListener.
Author: Reynold Xin <rxin@databricks.com>

Closes #9509 from rxin/SPARK-11540.
2015-11-05 18:12:54 -08:00
Reynold Xin 6091e91fca Revert "[SPARK-11469][SQL] Allow users to define nondeterministic udfs."
This reverts commit 9cf56c96b7.
2015-11-05 17:10:35 -08:00
Reynold Xin b6974f8fed [SPARK-11536][SQL] Remove the internal implicit conversion from Expression to Column in functions.scala
Author: Reynold Xin <rxin@databricks.com>

Closes #9505 from rxin/SPARK-11536.
2015-11-05 15:34:05 -08:00
Wenchen Fan d9e30c59ce [SPARK-10656][SQL] completely support special chars in DataFrame
the main problem is: we interpret column name with special handling of `.` for DataFrame. This enables us to write something like `df("a.b")` to get the field `b` of `a`. However, we don't need this feature in `DataFrame.apply("*")` or `DataFrame.withColumnRenamed`. In these 2 cases, the column name is the final name already, we don't need extra process to interpret it.

The solution is simple, use `queryExecution.analyzed.output` to get resolved column directly, instead of using `DataFrame.resolve`.

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9462 from cloud-fan/special-chars.
2015-11-05 14:53:16 -08:00
Reynold Xin 8a5314efd1 [SPARK-11532][SQL] Remove implicit conversion from Expression to Column
Author: Reynold Xin <rxin@databricks.com>

Closes #9500 from rxin/SPARK-11532.
2015-11-05 13:34:36 -08:00
Travis Hegner 14ee0f5726 [SPARK-10648] Oracle dialect to handle nonspecific numeric types
This is the alternative/agreed upon solution to PR #8780.

Creating an OracleDialect to handle the nonspecific numeric types that can be defined in oracle.

Author: Travis Hegner <thegner@trilliumit.com>

Closes #9495 from travishegner/OracleDialect.
2015-11-05 12:36:57 -08:00
Reynold Xin 6b87acd664 [SPARK-11513][SQL] Remove implicit conversion from LogicalPlan to DataFrame
This internal implicit conversion has been a source of confusion for a lot of new developers.

Author: Reynold Xin <rxin@databricks.com>

Closes #9479 from rxin/SPARK-11513.
2015-11-05 11:58:13 -08:00
Huaxin Gao b072ff4d1d [SPARK-11474][SQL] change fetchSize to fetchsize
In DefaultDataSource.scala, it has
override def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String]): BaseRelation
The parameters is CaseInsensitiveMap.
After this line
parameters.foreach(kv => properties.setProperty(kv._1, kv._2))
properties is set to all lower case key/value pairs and fetchSize becomes fetchsize.
However, in compute method in JDBCRDD, it has
val fetchSize = properties.getProperty("fetchSize", "0").toInt
so fetchSize value is always 0 and never gets set correctly.

Author: Huaxin Gao <huaxing@oc0558782468.ibm.com>

Closes #9473 from huaxingao/spark-11474.
2015-11-05 09:41:14 -08:00
Davies Liu 81498dd5c8 [SPARK-11425] [SPARK-11486] Improve hybrid aggregation
After aggregation, the dataset could be smaller than inputs, so it's better to do hash based aggregation for all inputs, then using sort based aggregation to merge them.

Author: Davies Liu <davies@databricks.com>

Closes #9383 from davies/fix_switch.
2015-11-04 21:30:21 -08:00
Zhenhua Wang a752ddad7f [SPARK-11398] [SQL] unnecessary def dialectClassName in HiveContext, and misleading dialect conf at the start of spark-sql
1. def dialectClassName in HiveContext is unnecessary.
In HiveContext, if conf.dialect == "hiveql", getSQLDialect() will return new HiveQLDialect(this);
else it will use super.getSQLDialect(). Then in super.getSQLDialect(), it calls dialectClassName, which is overriden in HiveContext and still return super.dialectClassName.
So we'll never reach the code "classOf[HiveQLDialect].getCanonicalName" of def dialectClassName in HiveContext.

2. When we start bin/spark-sql, the default context is HiveContext, and the corresponding dialect is hiveql.
However, if we type "set spark.sql.dialect;", the result is "sql", which is inconsistent with the actual dialect and is misleading. For example, we can use sql like "create table" which is only allowed in hiveql, but this dialect conf shows it's "sql".
Although this problem will not cause any execution error, it's misleading to spark sql users. Therefore I think we should fix it.
In this pr, while procesing “set spark.sql.dialect” in SetCommand, I use "conf.dialect" instead of "getConf()" for the case of key == SQLConf.DIALECT.key, so that it will return the right dialect conf.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #9349 from wzhfy/dialect.
2015-11-04 17:16:00 -08:00
Reynold Xin abf5e4285d [SPARK-11504][SQL] API audit for distributeBy and localSort
1. Renamed localSort -> sortWithinPartitions to avoid ambiguity in "local"
2. distributeBy -> repartition to match the existing repartition.

Author: Reynold Xin <rxin@databricks.com>

Closes #9470 from rxin/SPARK-11504.
2015-11-04 12:33:47 -08:00
Liang-Chi Hsieh de289bf279 [SPARK-10304][SQL] Following up checking valid dir structure for partition discovery
This patch follows up #8840.

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

Closes #9459 from viirya/detect_invalid_part_dir_following.
2015-11-04 10:56:32 -08:00
Reynold Xin 3bd6f5d2ae [SPARK-11490][SQL] variance should alias var_samp instead of var_pop.
stddev is an alias for stddev_samp. variance should be consistent with stddev.

Also took the chance to remove internal Stddev and Variance, and only kept StddevSamp/StddevPop and VarianceSamp/VariancePop.

Author: Reynold Xin <rxin@databricks.com>

Closes #9449 from rxin/SPARK-11490.
2015-11-04 09:34:52 -08:00
Reynold Xin cd1df66238 [SPARK-11485][SQL] Make DataFrameHolder and DatasetHolder public.
These two classes should be public, since they are used in public code.

Author: Reynold Xin <rxin@databricks.com>

Closes #9445 from rxin/SPARK-11485.
2015-11-04 09:32:30 -08:00
Wenchen Fan 2692bdb7db [SPARK-11455][SQL] fix case sensitivity of partition by
depend on `caseSensitive` to do column name equality check, instead of just `==`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9410 from cloud-fan/partition.
2015-11-03 20:25:58 -08:00
Nong e352de0db2 [SPARK-11329] [SQL] Cleanup from spark-11329 fix.
Author: Nong <nong@cloudera.com>

Closes #9442 from nongli/spark-11483.
2015-11-03 16:44:37 -08:00
Reynold Xin 5051262d4c [SPARK-11489][SQL] Only include common first order statistics in GroupedData
We added a bunch of higher order statistics such as skewness and kurtosis to GroupedData. I don't think they are common enough to justify being listed, since users can always use the normal statistics aggregate functions.

That is to say, after this change, we won't support
```scala
df.groupBy("key").kurtosis("colA", "colB")
```

However, we will still support
```scala
df.groupBy("key").agg(kurtosis(col("colA")), kurtosis(col("colB")))
```

Author: Reynold Xin <rxin@databricks.com>

Closes #9446 from rxin/SPARK-11489.
2015-11-03 16:27:56 -08:00
Wenchen Fan f6fcb4874c [SPARK-11477] [SQL] support create Dataset from RDD
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9434 from cloud-fan/rdd2ds and squashes the following commits:

0892d72 [Wenchen Fan] support create Dataset from RDD
2015-11-04 00:15:50 +01:00
Davies Liu 1d04dc95c0 [SPARK-11467][SQL] add Python API for stddev/variance
Add Python API for stddev/stddev_pop/stddev_samp/variance/var_pop/var_samp/skewness/kurtosis

Author: Davies Liu <davies@databricks.com>

Closes #9424 from davies/py_var.
2015-11-03 13:33:46 -08:00
Cheng Lian ebf8b0b48d [SPARK-10978][SQL] Allow data sources to eliminate filters
This PR adds a new method `unhandledFilters` to `BaseRelation`. Data sources which implement this method properly may avoid the overhead of defensive filtering done by Spark SQL.

Author: Cheng Lian <lian@databricks.com>

Closes #9399 from liancheng/spark-10978.unhandled-filters.
2015-11-03 10:07:45 -08:00
Liang-Chi Hsieh d6035d97c9 [SPARK-10304] [SQL] Partition discovery should throw an exception if the dir structure is invalid
JIRA: https://issues.apache.org/jira/browse/SPARK-10304

This patch detects if the structure of partition directories is not valid.

The test cases are from #8547. Thanks zhzhan.

cc liancheng

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

Closes #8840 from viirya/detect_invalid_part_dir.
2015-11-03 07:41:50 -08:00
Michael Armbrust b86f2cab67 [SPARK-11404] [SQL] Support for groupBy using column expressions
This PR adds a new method `groupBy(cols: Column*)` to `Dataset` that allows users to group using column expressions instead of a lambda function.  Since the return type of these expressions is not known at compile time, we just set the key type as a generic `Row`.  If the user would like to work the key in a type-safe way, they can call `grouped.asKey[Type]`, which is also added in this PR.

```scala
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
val grouped = ds.groupBy($"_1").asKey[String]
val agged = grouped.mapGroups { case (g, iter) =>
  Iterator((g, iter.map(_._2).sum))
}

agged.collect()

res0: Array(("a", 30), ("b", 3), ("c", 1))
```

Author: Michael Armbrust <michael@databricks.com>

Closes #9359 from marmbrus/columnGroupBy and squashes the following commits:

bbcb03b [Michael Armbrust] Update DatasetSuite.scala
8fd2908 [Michael Armbrust] Update DatasetSuite.scala
0b0e2f8 [Michael Armbrust] [SPARK-11404] [SQL] Support for groupBy using column expressions
2015-11-03 13:02:17 +01:00
Wenchen Fan 425ff03f5a [SPARK-11436] [SQL] rebind right encoder when join 2 datasets
When we join 2 datasets, we will combine 2 encoders into a tupled one, and use it as the encoder for the jioned dataset. Assume both of the 2 encoders are flat, their `constructExpression`s both reference to the first element of input row. However, when we combine 2 encoders, the schema of input row changed,  now the right encoder should reference to second element of input row. So we should rebind right encoder to let it know the new schema of input row before combine it.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9391 from cloud-fan/join and squashes the following commits:

846d3ab [Wenchen Fan] rebind right encoder when join 2 datasets
2015-11-03 12:47:39 +01:00
Yin Huai d728d5c986 [SPARK-9858][SPARK-9859][SPARK-9861][SQL] Add an ExchangeCoordinator to estimate the number of post-shuffle partitions for aggregates and joins
https://issues.apache.org/jira/browse/SPARK-9858
https://issues.apache.org/jira/browse/SPARK-9859
https://issues.apache.org/jira/browse/SPARK-9861

Author: Yin Huai <yhuai@databricks.com>

Closes #9276 from yhuai/numReducer.
2015-11-03 00:12:49 -08:00
navis.ryu c34c27fe92 [SPARK-9034][SQL] Reflect field names defined in GenericUDTF
Hive GenericUDTF#initialize() defines field names in a returned schema though,
the current HiveGenericUDTF drops these names.
We might need to reflect these in a logical plan tree.

Author: navis.ryu <navis@apache.org>

Closes #8456 from navis/SPARK-9034.
2015-11-02 23:52:36 -08:00
Yin Huai 9cf56c96b7 [SPARK-11469][SQL] Allow users to define nondeterministic udfs.
This is the first task (https://issues.apache.org/jira/browse/SPARK-11469) of https://issues.apache.org/jira/browse/SPARK-11438

Author: Yin Huai <yhuai@databricks.com>

Closes #9393 from yhuai/udfNondeterministic.
2015-11-02 21:18:38 -08:00
Nong Li 9cb5c731da [SPARK-11329][SQL] Support star expansion for structs.
1. Supporting expanding structs in Projections. i.e.
  "SELECT s.*" where s is a struct type.
  This is fixed by allowing the expand function to handle structs in addition to tables.

2. Supporting expanding * inside aggregate functions of structs.
   "SELECT max(struct(col1, structCol.*))"
   This requires recursively expanding the expressions. In this case, it it the aggregate
   expression "max(...)" and we need to recursively expand its children inputs.

Author: Nong Li <nongli@gmail.com>

Closes #9343 from nongli/spark-11329.
2015-11-02 20:32:08 -08:00
Nong Li 2cef1bb0b5 [SPARK-5354][SQL] Cached tables should preserve partitioning and ord…
…ering.

For cached tables, we can just maintain the partitioning and ordering from the
source relation.

Author: Nong Li <nongli@gmail.com>

Closes #9404 from nongli/spark-5354.
2015-11-02 19:18:45 -08:00
Liang-Chi Hsieh 3e770a64a4 [SPARK-9298][SQL] Add pearson correlation aggregation function
JIRA: https://issues.apache.org/jira/browse/SPARK-9298

This patch adds pearson correlation aggregation function based on `AggregateExpression2`.

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

Closes #8587 from viirya/corr_aggregation.
2015-11-01 18:37:27 -08:00
Nong Li 046e32ed84 [SPARK-11410][SQL] Add APIs to provide functionality similar to Hive's DISTRIBUTE BY and SORT BY.
DISTRIBUTE BY allows the user to hash partition the data by specified exprs. It also allows for
optioning sorting within each resulting partition. There is no required relationship between the
exprs for partitioning and sorting (i.e. one does not need to be a prefix of the other).

This patch adds to APIs to DataFrames which can be used together to provide this functionality:
  1. distributeBy() which partitions the data frame into a specified number of partitions using the
     partitioning exprs.
  2. localSort() which sorts each partition using the provided sorting exprs.

To get the DISTRIBUTE BY functionality, the user simply does: df.distributeBy(...).localSort(...)

Author: Nong Li <nongli@gmail.com>

Closes #9364 from nongli/spark-11410.
2015-11-01 14:34:06 -08:00
Cheng Lian aa494a9c2e [SPARK-11117] [SPARK-11345] [SQL] Makes all HadoopFsRelation data sources produce UnsafeRow
This PR fixes two issues:

1.  `PhysicalRDD.outputsUnsafeRows` is always `false`

    Thus a `ConvertToUnsafe` operator is often required even if the underlying data source relation does output `UnsafeRow`.

1.  Internal/external row conversion for `HadoopFsRelation` is kinda messy

    Currently we're using `HadoopFsRelation.needConversion` and [dirty type erasure hacks][1] to indicate whether the relation outputs external row or internal row and apply external-to-internal conversion when necessary.  Basically, all builtin `HadoopFsRelation` data sources, i.e. Parquet, JSON, ORC, and Text output `InternalRow`, while typical external `HadoopFsRelation` data sources, e.g. spark-avro and spark-csv, output `Row`.

This PR adds a `private[sql]` interface method `HadoopFsRelation.buildInternalScan`, which by default invokes `HadoopFsRelation.buildScan` and converts `Row`s to `UnsafeRow`s (which are also `InternalRow`s).  All builtin `HadoopFsRelation` data sources override this method and directly output `UnsafeRow`s.  In this way, now `HadoopFsRelation` always produces `UnsafeRow`s. Thus `PhysicalRDD.outputsUnsafeRows` can be properly set by checking whether the underlying data source is a `HadoopFsRelation`.

A remaining question is that, can we assume that all non-builtin `HadoopFsRelation` data sources output external rows?  At least all well known ones do so.  However it's possible that some users implemented their own `HadoopFsRelation` data sources that leverages `InternalRow` and thus all those unstable internal data representations.  If this assumption is safe, we can deprecate `HadoopFsRelation.needConversion` and cleanup some more conversion code (like [here][2] and [here][3]).

This PR supersedes #9125.

Follow-ups:

1.  Makes JSON and ORC data sources output `UnsafeRow` directly

1.  Makes `HiveTableScan` output `UnsafeRow` directly

    This is related to 1 since ORC data source shares the same `Writable` unwrapping code with `HiveTableScan`.

[1]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala#L353
[2]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala#L331-L335
[3]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala#L630-L669

Author: Cheng Lian <lian@databricks.com>

Closes #9305 from liancheng/spark-11345.unsafe-hadoop-fs-relation.
2015-10-31 21:16:09 -07:00
Jeff Zhang 97b3c8fb47 [SPARK-11226][SQL] Empty line in json file should be skipped
Currently the empty line in json file will be parsed into Row with all null field values. But in json, "{}" represents a json object, empty line is supposed to be skipped.

Make a trivial change for this.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #9211 from zjffdu/SPARK-11226.
2015-10-31 11:10:37 +00:00
Davies Liu 45029bfdea [SPARK-11423] remove MapPartitionsWithPreparationRDD
Since we do not need to preserve a page before calling compute(), MapPartitionsWithPreparationRDD is not needed anymore.

This PR basically revert #8543, #8511, #8038, #8011

Author: Davies Liu <davies@databricks.com>

Closes #9381 from davies/remove_prepare2.
2015-10-30 15:47:40 -07:00
Wenchen Fan 14d08b9908 [SPARK-11393] [SQL] CoGroupedIterator should respect the fact that GroupedIterator.hasNext is not idempotent
When we cogroup 2 `GroupedIterator`s in `CoGroupedIterator`, if the right side is smaller, we will consume right data and keep the left data unchanged. Then we call `hasNext` which will call `left.hasNext`. This will make `GroupedIterator` generate an extra group as the previous one has not been comsumed yet.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9346 from cloud-fan/cogroup and squashes the following commits:

9be67c8 [Wenchen Fan] SPARK-11393
2015-10-30 12:17:51 +01:00
hyukjinkwon 59db9e9c38 [SPARK-11103][SQL] Filter applied on Merged Parquet shema with new column fail
When enabling mergedSchema and predicate filter, this fails since Parquet does not accept filters pushed down when the columns of the filters do not exist in the schema.
This is related with Parquet issue (https://issues.apache.org/jira/browse/PARQUET-389).

For now, it just simply disables predicate push down when using merged schema in this PR.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9327 from HyukjinKwon/SPARK-11103.
2015-10-30 18:17:35 +08:00
Davies Liu 56419cf11f [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPARK-10929] [SQL] Cooperative memory management
This PR introduce a mechanism to call spill() on those SQL operators that support spilling (for example, BytesToBytesMap, UnsafeExternalSorter and ShuffleExternalSorter) if there is not enough memory for execution. The preserved first page is needed anymore, so removed.

Other Spillable objects in Spark core (ExternalSorter and AppendOnlyMap) are not included in this PR, but those could benefit from this (trigger others' spilling).

The PrepareRDD may be not needed anymore, could be removed in follow up PR.

The following script will fail with OOM before this PR, finished in 150 seconds with 2G heap (also works in 1.5 branch, with similar duration).

```python
sqlContext.setConf("spark.sql.shuffle.partitions", "1")
df = sqlContext.range(1<<25).selectExpr("id", "repeat(id, 2) as s")
df2 = df.select(df.id.alias('id2'), df.s.alias('s2'))
j = df.join(df2, df.id==df2.id2).groupBy(df.id).max("id", "id2")
j.explain()
print j.count()
```

For thread-safety, here what I'm got:

1) Without calling spill(), the operators should only be used by single thread, no safety problems.

2) spill() could be triggered in two cases, triggered by itself, or by other operators. we can check trigger == this in spill(), so it's still in the same thread, so safety problems.

3) if it's triggered by other operators (right now cache will not trigger spill()), we only spill the data into disk when it's in scanning stage (building is finished), so the in-memory sorter or memory pages are read-only, we only need to synchronize the iterator and change it.

4) During scanning, the iterator will only use one record in one page, we can't free this page, because the downstream is currently using it (used by UnsafeRow or other objects). In BytesToBytesMap, we just skip the current page, and dump all others into disk. In UnsafeExternalSorter, we keep the page that is used by current record (having the same baseObject), free it when loading the next record. In ShuffleExternalSorter, the spill() will not trigger during scanning.

5) In order to avoid deadlock, we didn't call acquireMemory during spill (so we reused the pointer array in InMemorySorter).

Author: Davies Liu <davies@databricks.com>

Closes #9241 from davies/force_spill.
2015-10-29 23:38:06 -07:00
Wenchen Fan 96cf87f66d [SPARK-11301] [SQL] fix case sensitivity for filter on partitioned columns
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9271 from cloud-fan/filter.
2015-10-29 16:36:52 -07:00
sethah a01cbf5daa [SPARK-10641][SQL] Add Skewness and Kurtosis Support
Implementing skewness and kurtosis support based on following algorithm:
https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics

Author: sethah <seth.hendrickson16@gmail.com>

Closes #9003 from sethah/SPARK-10641.
2015-10-29 11:58:39 -07:00
xin Wu f7a51deeba [SPARK-11246] [SQL] Table cache for Parquet broken in 1.5
The root cause is that when spark.sql.hive.convertMetastoreParquet=true by default, the cached InMemoryRelation of the ParquetRelation can not be looked up from the cachedData of CacheManager because the key comparison fails even though it is the same LogicalPlan representing the Subquery that wraps the ParquetRelation.
The solution in this PR is overriding the LogicalPlan.sameResult function in Subquery case class to eliminate subquery node first before directly comparing the child (ParquetRelation), which will find the key  to the cached InMemoryRelation.

Author: xin Wu <xinwu@us.ibm.com>

Closes #9326 from xwu0226/spark-11246-commit.
2015-10-29 07:42:46 -07:00
Wenchen Fan f79ebf2a9e [SPARK-11370] [SQL] fix a bug in GroupedIterator and create unit test for it
Before this PR, user has to consume the iterator of one group before process next group, or we will get into infinite loops.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9330 from cloud-fan/group.
2015-10-29 11:49:45 +01:00
Cheng Lian e5b89978ed [SPARK-11376][SQL] Removes duplicated mutableRow field
This PR fixes a mistake in the code generated by `GenerateColumnAccessor`. Interestingly, although the code is illegal in Java (the class has two fields with the same name), Janino accepts it happily and accidentally works properly.

Author: Cheng Lian <lian@databricks.com>

Closes #9335 from liancheng/spark-11376.fix-generated-code.
2015-10-29 11:34:54 +08:00
Liang-Chi Hsieh 20dfd46743 [SPARK-11363] [SQL] LeftSemiJoin should be LeftSemi in SparkStrategies
JIRA: https://issues.apache.org/jira/browse/SPARK-11363

In SparkStrategies some places use LeftSemiJoin. It should be LeftSemi.

cc chenghao-intel liancheng

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

Closes #9318 from viirya/no-left-semi-join.
2015-10-28 15:57:01 -07:00
Wenchen Fan 075ce4914f [SPARK-11313][SQL] implement cogroup on DataSets (support 2 datasets)
A simpler version of https://github.com/apache/spark/pull/9279, only support 2 datasets.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9324 from cloud-fan/cogroup2.
2015-10-28 13:58:52 +01:00
Cheng Hao d9c6039897 [SPARK-10484] [SQL] Optimize the cartesian join with broadcast join for some cases
In some cases, we can broadcast the smaller relation in cartesian join, which improve the performance significantly.

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

Closes #8652 from chenghao-intel/cartesian.
2015-10-27 20:26:38 -07:00
Michael Armbrust 5a5f65905a [SPARK-11347] [SQL] Support for joinWith in Datasets
This PR adds a new operation `joinWith` to a `Dataset`, which returns a `Tuple` for each pair where a given `condition` evaluates to true.

```scala
case class ClassData(a: String, b: Int)

val ds1 = Seq(ClassData("a", 1), ClassData("b", 2)).toDS()
val ds2 = Seq(("a", 1), ("b", 2)).toDS()

> ds1.joinWith(ds2, $"_1" === $"a").collect()
res0: Array((ClassData("a", 1), ("a", 1)), (ClassData("b", 2), ("b", 2)))
```

This operation is similar to the relation `join` function with one important difference in the result schema. Since `joinWith` preserves objects present on either side of the join, the result schema is similarly nested into a tuple under the column names `_1` and `_2`.

This type of join can be useful both for preserving type-safety with the original object types as well as working with relational data where either side of the join has column names in common.

## Required Changes to Encoders
In the process of working on this patch, several deficiencies to the way that we were handling encoders were discovered.  Specifically, it turned out to be very difficult to `rebind` the non-expression based encoders to extract the nested objects from the results of joins (and also typed selects that return tuples).

As a result the following changes were made.
 - `ClassEncoder` has been renamed to `ExpressionEncoder` and has been improved to also handle primitive types.  Additionally, it is now possible to take arbitrary expression encoders and rewrite them into a single encoder that returns a tuple.
 - All internal operations on `Dataset`s now require an `ExpressionEncoder`.  If the users tries to pass a non-`ExpressionEncoder` in, an error will be thrown.  We can relax this requirement in the future by constructing a wrapper class that uses expressions to project the row to the expected schema, shielding the users code from the required remapping.  This will give us a nice balance where we don't force user encoders to understand attribute references and binding, but still allow our native encoder to leverage runtime code generation to construct specific encoders for a given schema that avoid an extra remapping step.
 - Additionally, the semantics for different types of objects are now better defined.  As stated in the `ExpressionEncoder` scaladoc:
  - Classes will have their sub fields extracted by name using `UnresolvedAttribute` expressions
  and `UnresolvedExtractValue` expressions.
  - Tuples will have their subfields extracted by position using `BoundReference` expressions.
  - Primitives will have their values extracted from the first ordinal with a schema that defaults
  to the name `value`.
 - Finally, the binding lifecycle for `Encoders` has now been unified across the codebase.  Encoders are now `resolved` to the appropriate schema in the constructor of `Dataset`.  This process replaces an unresolved expressions with concrete `AttributeReference` expressions.  Binding then happens on demand, when an encoder is going to be used to construct an object.  This closely mirrors the lifecycle for standard expressions when executing normal SQL or `DataFrame` queries.

Author: Michael Armbrust <michael@databricks.com>

Closes #9300 from marmbrus/datasets-tuples.
2015-10-27 13:28:52 -07:00
Stephen De Gennaro 82464fb2e0 [SPARK-10947] [SQL] With schema inference from JSON into a Dataframe, add option to infer all primitive object types as strings
Currently, when a schema is inferred from a JSON file using sqlContext.read.json, the primitive object types are inferred as string, long, boolean, etc.

However, if the inferred type is too specific (JSON obviously does not enforce types itself), this can cause issues with merging dataframe schemas.

This pull request adds the option "primitivesAsString" to the JSON DataFrameReader which when true (defaults to false if not set) will infer all primitives as strings.

Below is an example usage of this new functionality.
```
val jsonDf = sqlContext.read.option("primitivesAsString", "true").json(sampleJsonFile)

scala> jsonDf.printSchema()
root
|-- bigInteger: string (nullable = true)
|-- boolean: string (nullable = true)
|-- double: string (nullable = true)
|-- integer: string (nullable = true)
|-- long: string (nullable = true)
|-- null: string (nullable = true)
|-- string: string (nullable = true)
```

Author: Stephen De Gennaro <stepheng@realitymine.com>

Closes #9249 from stephend-realitymine/stephend-primitives.
2015-10-26 19:55:10 -07:00
Nong Li d4c397a64a [SPARK-11325] [SQL] Alias 'alias' in Scala's DataFrame API
Author: Nong Li <nongli@gmail.com>

Closes #9286 from nongli/spark-11325.
2015-10-26 18:27:02 -07:00
Alexander Slesarenko 4bb2b3698f [SQL][DOC] Minor document fixes in interfaces.scala
rxin just noticed this while reading the code.

Author: Alexander Slesarenko <avslesarenko@gmail.com>

Closes #9284 from aslesarenko/doc-typos.
2015-10-26 23:49:14 +01:00
Frank Rosner b60aab8a95 [SPARK-11258] Converting a Spark DataFrame into an R data.frame is slow / requires a lot of memory
https://issues.apache.org/jira/browse/SPARK-11258

I was not able to locate an existing unit test for this function so I wrote one.

Author: Frank Rosner <frank@fam-rosner.de>

Closes #9222 from FRosner/master.
2015-10-26 15:46:59 -07:00
Wenchen Fan 07ced43424 [SPARK-11253] [SQL] reset all accumulators in physical operators before execute an action
With this change, our query execution listener can get the metrics correctly.

The UI still looks good after this change.
<img width="257" alt="screen shot 2015-10-23 at 11 25 14 am" src="https://cloud.githubusercontent.com/assets/3182036/10683834/d516f37e-7978-11e5-8118-343ed40eb824.png">
<img width="494" alt="screen shot 2015-10-23 at 11 25 01 am" src="https://cloud.githubusercontent.com/assets/3182036/10683837/e1fa60da-7978-11e5-8ec8-178b88f27764.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9215 from cloud-fan/metric.
2015-10-25 22:47:39 -07:00
Josh Rosen 85e654c5ec [SPARK-10984] Simplify *MemoryManager class structure
This patch refactors the MemoryManager class structure. After #9000, Spark had the following classes:

- MemoryManager
- StaticMemoryManager
- ExecutorMemoryManager
- TaskMemoryManager
- ShuffleMemoryManager

This is fairly confusing. To simplify things, this patch consolidates several of these classes:

- ShuffleMemoryManager and ExecutorMemoryManager were merged into MemoryManager.
- TaskMemoryManager is moved into Spark Core.

**Key changes and tasks**:

- [x] Merge ExecutorMemoryManager into MemoryManager.
  - [x] Move pooling logic into Allocator.
- [x] Move TaskMemoryManager from `spark-unsafe` to `spark-core`.
- [x] Refactor the existing Tungsten TaskMemoryManager interactions so Tungsten code use only this and not both this and ShuffleMemoryManager.
- [x] Refactor non-Tungsten code to use the TaskMemoryManager instead of ShuffleMemoryManager.
- [x] Merge ShuffleMemoryManager into MemoryManager.
  - [x] Move code
  - [x] ~~Simplify 1/n calculation.~~ **Will defer to followup, since this needs more work.**
- [x] Port ShuffleMemoryManagerSuite tests.
- [x] Move classes from `unsafe` package to `memory` package.
- [ ] Figure out how to handle the hacky use of the memory managers in HashedRelation's broadcast variable construction.
- [x] Test porting and cleanup: several tests relied on mock functionality (such as `TestShuffleMemoryManager.markAsOutOfMemory`) which has been changed or broken during the memory manager consolidation
  - [x] AbstractBytesToBytesMapSuite
  - [x] UnsafeExternalSorterSuite
  - [x] UnsafeFixedWidthAggregationMapSuite
  - [x] UnsafeKVExternalSorterSuite

**Compatiblity notes**:

- This patch introduces breaking changes in `ExternalAppendOnlyMap`, which is marked as `DevloperAPI` (likely for legacy reasons): this class now cannot be used outside of a task.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9127 from JoshRosen/SPARK-10984.
2015-10-25 21:19:52 -07:00
Reynold Xin e1a897b657 [SPARK-11274] [SQL] Text data source support for Spark SQL.
This adds API for reading and writing text files, similar to SparkContext.textFile and RDD.saveAsTextFile.
```
SQLContext.read.text("/path/to/something.txt")
DataFrame.write.text("/path/to/write.txt")
```

Using the new Dataset API, this also supports
```
val ds: Dataset[String] = SQLContext.read.text("/path/to/something.txt").as[String]
```

Author: Reynold Xin <rxin@databricks.com>

Closes #9240 from rxin/SPARK-11274.
2015-10-23 13:04:06 -07:00
Reynold Xin cdea0174e3 [SPARK-11273][SQL] Move ArrayData/MapData/DataTypeParser to catalyst.util package
Author: Reynold Xin <rxin@databricks.com>

Closes #9239 from rxin/types-private.
2015-10-23 00:00:21 -07:00
Michael Armbrust 53e83a3a77 [SPARK-11116][SQL] First Draft of Dataset API
*This PR adds a new experimental API to Spark, tentitively named Datasets.*

A `Dataset` is a strongly-typed collection of objects that can be transformed in parallel using functional or relational operations.  Example usage is as follows:

### Functional
```scala
> val ds: Dataset[Int] = Seq(1, 2, 3).toDS()
> ds.filter(_ % 1 == 0).collect()
res1: Array[Int] = Array(1, 2, 3)
```

### Relational
```scala
scala> ds.toDF().show()
+-----+
|value|
+-----+
|    1|
|    2|
|    3|
+-----+

> ds.select(expr("value + 1").as[Int]).collect()
res11: Array[Int] = Array(2, 3, 4)
```

## Comparison to RDDs
 A `Dataset` differs from an `RDD` in the following ways:
  - The creation of a `Dataset` requires the presence of an explicit `Encoder` that can be
    used to serialize the object into a binary format.  Encoders are also capable of mapping the
    schema of a given object to the Spark SQL type system.  In contrast, RDDs rely on runtime
    reflection based serialization.
  - Internally, a `Dataset` is represented by a Catalyst logical plan and the data is stored
    in the encoded form.  This representation allows for additional logical operations and
    enables many operations (sorting, shuffling, etc.) to be performed without deserializing to
    an object.

A `Dataset` can be converted to an `RDD` by calling the `.rdd` method.

## Comparison to DataFrames

A `Dataset` can be thought of as a specialized DataFrame, where the elements map to a specific
JVM object type, instead of to a generic `Row` container. A DataFrame can be transformed into
specific Dataset by calling `df.as[ElementType]`.  Similarly you can transform a strongly-typed
`Dataset` to a generic DataFrame by calling `ds.toDF()`.

## Implementation Status and TODOs

This is a rough cut at the least controversial parts of the API.  The primary purpose here is to get something committed so that we can better parallelize further work and get early feedback on the API.  The following is being deferred to future PRs:
 - Joins and Aggregations (prototype here f11f91e6f0)
 - Support for Java

Additionally, the responsibility for binding an encoder to a given schema is currently done in a fairly ad-hoc fashion.  This is an internal detail, and what we are doing today works for the cases we care about.  However, as we add more APIs we'll probably need to do this in a more principled way (i.e. separate resolution from binding as we do in DataFrames).

## COMPATIBILITY NOTE
Long term we plan to make `DataFrame` extend `Dataset[Row]`.  However,
making this change to che class hierarchy would break the function signatures for the existing
function operations (map, flatMap, etc).  As such, this class should be considered a preview
of the final API.  Changes will be made to the interface after Spark 1.6.

Author: Michael Armbrust <michael@databricks.com>

Closes #9190 from marmbrus/dataset-infra.
2015-10-22 15:20:17 -07:00
Cheng Hao d4950e6be4 [SPARK-9735][SQL] Respect the user specified schema than the infer partition schema for HadoopFsRelation
To enable the unit test of `hadoopFsRelationSuite.Partition column type casting`. It previously threw exception like below, as we treat the auto infer partition schema with higher priority than the user specified one.

```
java.lang.ClassCastException: java.lang.Integer cannot be cast to org.apache.spark.unsafe.types.UTF8String
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:45)
	at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getUTF8String(rows.scala:220)
	at org.apache.spark.sql.catalyst.expressions.JoinedRow.getUTF8String(JoinedRow.scala:102)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(generated.java:62)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
	at scala.collection.AbstractIterator.to(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
07:44:01.344 ERROR org.apache.spark.executor.Executor: Exception in task 14.0 in stage 3.0 (TID 206)
java.lang.ClassCastException: java.lang.Integer cannot be cast to org.apache.spark.unsafe.types.UTF8String
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:45)
	at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getUTF8String(rows.scala:220)
	at org.apache.spark.sql.catalyst.expressions.JoinedRow.getUTF8String(JoinedRow.scala:102)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(generated.java:62)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
	at scala.collection.AbstractIterator.to(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
```

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

Closes #8026 from chenghao-intel/partition_discovery.
2015-10-22 13:11:37 -07:00
Josh Rosen f6d06adf05 [SPARK-10708] Consolidate sort shuffle implementations
There's a lot of duplication between SortShuffleManager and UnsafeShuffleManager. Given that these now provide the same set of functionality, now that UnsafeShuffleManager supports large records, I think that we should replace SortShuffleManager's serialized shuffle implementation with UnsafeShuffleManager's and should merge the two managers together.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8829 from JoshRosen/consolidate-sort-shuffle-implementations.
2015-10-22 09:46:30 -07:00
Davies Liu 1d97332715 [SPARK-11243][SQL] output UnsafeRow from columnar cache
This PR change InMemoryTableScan to output UnsafeRow, and optimize the unrolling and scanning by coping the bytes for var-length types between UnsafeRow and ByteBuffer directly without creating the wrapper objects. When scanning the decimals in TPC-DS store_sales table, it's 80% faster (copy it as long without create Decimal objects).

Author: Davies Liu <davies@databricks.com>

Closes #9203 from davies/unsafe_cache.
2015-10-21 19:20:31 -07:00
Yanbo Liang 40a10d7675 [SPARK-9392][SQL] Dataframe drop should work on unresolved columns
Dataframe drop should work on unresolved columns

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #8821 from yanboliang/spark-9392.
2015-10-21 17:50:33 -07:00
Yin Huai 3afe448d39 [SPARK-9740][SPARK-9592][SPARK-9210][SQL] Change the default behavior of First/Last to RESPECT NULLS.
I am changing the default behavior of `First`/`Last` to respect null values (the SQL standard default behavior).

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

Author: Yin Huai <yhuai@databricks.com>

Closes #8113 from yhuai/firstLast.
2015-10-21 13:43:17 -07:00
Davies Liu f8c6bec657 [SPARK-11197][SQL] run SQL on files directly
This PR introduce a new feature to run SQL directly on files without create a table, for example:

```
select id from json.`path/to/json/files` as j
```

Author: Davies Liu <davies@databricks.com>

Closes #9173 from davies/source.
2015-10-21 13:38:30 -07:00
Wenchen Fan 7c74ebca05 [SPARK-10743][SQL] keep the name of expression if possible when do cast
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8859 from cloud-fan/cast.
2015-10-21 13:22:35 -07:00
Pravin Gadakh 8e82e59834 [SPARK-11037][SQL] using Option instead of Some in JdbcDialects
Using Option instead of Some in getCatalystType method.

Author: Pravin Gadakh <prgadakh@in.ibm.com>

Closes #9195 from pravingadakh/master.
2015-10-21 10:35:09 -07:00
Cheng Lian 89e6db6150 [SPARK-11153][SQL] Disables Parquet filter push-down for string and binary columns
Due to PARQUET-251, `BINARY` columns in existing Parquet files may be written with corrupted statistics information. This information is used by filter push-down optimization. Since Spark 1.5 turns on Parquet filter push-down by default, we may end up with wrong query results. PARQUET-251 has been fixed in parquet-mr 1.8.1, but Spark 1.5 is still using 1.7.0.

This affects all Spark SQL data types that can be mapped to Parquet {{BINARY}}, namely:

- `StringType`

- `BinaryType`

- `DecimalType`

  (But Spark SQL doesn't support pushing down filters involving `DecimalType` columns for now.)

To avoid wrong query results, we should disable filter push-down for columns of `StringType` and `BinaryType` until we upgrade to parquet-mr 1.8.

Author: Cheng Lian <lian@databricks.com>

Closes #9152 from liancheng/spark-11153.workaround-parquet-251.

(cherry picked from commit 0887e5e878)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-10-21 09:02:59 +08:00
Davies Liu 06e6b765d0 [SPARK-11149] [SQL] Improve cache performance for primitive types
This PR improve the performance by:

1) Generate an Iterator that take Iterator[CachedBatch] as input, and call accessors (unroll the loop for columns), avoid the expensive Iterator.flatMap.

2) Use Unsafe.getInt/getLong/getFloat/getDouble instead of ByteBuffer.getInt/getLong/getFloat/getDouble, the later one actually read byte by byte.

3) Remove the unnecessary copy() in Coalesce(), which is not related to memory cache, found during benchmark.

The following benchmark showed that we can speedup the columnar cache of int by 2x.

```
path = '/opt/tpcds/store_sales/'
int_cols = ['ss_sold_date_sk', 'ss_sold_time_sk', 'ss_item_sk','ss_customer_sk']
df = sqlContext.read.parquet(path).select(int_cols).cache()
df.count()

t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```

Author: Davies Liu <davies@databricks.com>

Closes #9145 from davies/byte_buffer.
2015-10-20 14:01:53 -07:00
Cheng Lian 8b877cc4ee [SPARK-11088][SQL] Merges partition values using UnsafeProjection
`DataSourceStrategy.mergeWithPartitionValues` is essentially a projection implemented in a quite inefficient way. This PR optimizes this method with `UnsafeProjection` to avoid unnecessary boxing costs.

Author: Cheng Lian <lian@databricks.com>

Closes #9104 from liancheng/spark-11088.faster-partition-values-merging.
2015-10-19 16:57:20 -07:00
Rishabh Bhardwaj 5966817941 [SPARK-11180][SQL] Support BooleanType in DataFrame.na.fill
Added support for boolean types in fill and replace methods

Author: Rishabh Bhardwaj <rbnext29@gmail.com>

Closes #9166 from rishabhbhardwaj/master.
2015-10-19 14:38:58 -07:00
Wenchen Fan 7893cd95db [SPARK-11119] [SQL] cleanup for unsafe array and map
The purpose of this PR is to keep the unsafe format detail only inside the unsafe class itself, so when we use them(like use unsafe array in unsafe map, use unsafe array and map in columnar cache), we don't need to understand the format before use them.

change list:
* unsafe array's 4-bytes numElements header is now required(was optional), and become a part of unsafe array format.
* w.r.t the previous changing, the `sizeInBytes` of unsafe array now counts the 4-bytes header.
* unsafe map's format was `[numElements] [key array numBytes] [key array content(without numElements header)] [value array content(without numElements header)]` before, which is a little hacky as it makes unsafe array's header optional. I think saving 4 bytes is not a big deal, so the format is now: `[key array numBytes] [unsafe key array] [unsafe value array]`.
* w.r.t the previous changing, the `sizeInBytes` of unsafe map now counts both map's header and array's header.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9131 from cloud-fan/unsafe.
2015-10-19 11:02:26 -07:00
zsxwing 94c8fef296 [SPARK-11126][SQL] Fix a memory leak in SQLListener._stageIdToStageMetrics
SQLListener adds all stage infos to `_stageIdToStageMetrics`, but only removes stage infos belonging to SQL executions. This PR fixed it by ignoring stages that don't belong to SQL executions.

Reported by Terry Hoo in https://www.mail-archive.com/userspark.apache.org/msg38810.html

Author: zsxwing <zsxwing@gmail.com>

Closes #9132 from zsxwing/SPARK-11126.
2015-10-18 13:51:45 -07:00
Koert Kuipers 57f83e36d6 [SPARK-10185] [SQL] Feat sql comma separated paths
Make sure comma-separated paths get processed correcly in ResolvedDataSource for a HadoopFsRelationProvider

Author: Koert Kuipers <koert@tresata.com>

Closes #8416 from koertkuipers/feat-sql-comma-separated-paths.
2015-10-17 14:56:24 -07:00
Pravin Gadakh 3d683a139b [SPARK-10581] [DOCS] Groups are not resolved in scaladoc in sql classes
Groups are not resolved properly in scaladoc in following classes:

sql/core/src/main/scala/org/apache/spark/sql/Column.scala
sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
sql/core/src/main/scala/org/apache/spark/sql/functions.scala

Author: Pravin Gadakh <pravingadakh177@gmail.com>

Closes #9148 from pravingadakh/master.
2015-10-16 13:38:50 -07:00
navis.ryu b9c5e5d4ac [SPARK-11124] JsonParser/Generator should be closed for resource recycle
Some json parsers are not closed. parser in JacksonParser#parseJson, for example.

Author: navis.ryu <navis@apache.org>

Closes #9130 from navis/SPARK-11124.
2015-10-16 11:19:37 -07:00
Josh Rosen eb0b4d6e2d [SPARK-11135] [SQL] Exchange incorrectly skips sorts when existing ordering is non-empty subset of required ordering
In Spark SQL, the Exchange planner tries to avoid unnecessary sorts in cases where the data has already been sorted by a superset of the requested sorting columns. For instance, let's say that a query calls for an operator's input to be sorted by `a.asc` and the input happens to already be sorted by `[a.asc, b.asc]`. In this case, we do not need to re-sort the input. The converse, however, is not true: if the query calls for `[a.asc, b.asc]`, then `a.asc` alone will not satisfy the ordering requirements, requiring an additional sort to be planned by Exchange.

However, the current Exchange code gets this wrong and incorrectly skips sorting when the existing output ordering is a subset of the required ordering. This is simple to fix, however.

This bug was introduced in https://github.com/apache/spark/pull/7458, so it affects 1.5.0+.

This patch fixes the bug and significantly improves the unit test coverage of Exchange's sort-planning logic.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9140 from JoshRosen/SPARK-11135.
2015-10-15 17:36:55 -07:00
Wenchen Fan 6a2359ff1f [SPARK-10412] [SQL] report memory usage for tungsten sql physical operator
https://issues.apache.org/jira/browse/SPARK-10412

some screenshots:
### aggregate:
![screen shot 2015-10-12 at 2 23 11 pm](https://cloud.githubusercontent.com/assets/3182036/10439534/618320a4-70ef-11e5-94d8-62ea7f2d1531.png)

### join
![screen shot 2015-10-12 at 2 23 29 pm](https://cloud.githubusercontent.com/assets/3182036/10439537/6724797c-70ef-11e5-8f75-0cf5cbd42048.png)

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8931 from cloud-fan/viz.
2015-10-15 14:50:58 -07:00
Josh Rosen 4ace4f8a9c [SPARK-11017] [SQL] Support ImperativeAggregates in TungstenAggregate
This patch extends TungstenAggregate to support ImperativeAggregate functions. The existing TungstenAggregate operator only supported DeclarativeAggregate functions, which are defined in terms of Catalyst expressions and can be evaluated via generated projections. ImperativeAggregate functions, on the other hand, are evaluated by calling their `initialize`, `update`, `merge`, and `eval` methods.

The basic strategy here is similar to how SortBasedAggregate evaluates both types of aggregate functions: use a generated projection to evaluate the expression-based declarative aggregates with dummy placeholder expressions inserted in place of the imperative aggregate function output, then invoke the imperative aggregate functions and target them against the aggregation buffer. The bulk of the diff here consists of code that was copied and adapted from SortBasedAggregate, with some key changes to handle TungstenAggregate's sort fallback path.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9038 from JoshRosen/support-interpreted-in-tungsten-agg-final.
2015-10-14 17:27:50 -07:00
Cheng Hao 1baaf2b9bd [SPARK-10829] [SQL] Filter combine partition key and attribute doesn't work in DataSource scan
```scala
withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") {
      withTempPath { dir =>
        val path = s"${dir.getCanonicalPath}/part=1"
        (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(path)

        // If the "part = 1" filter gets pushed down, this query will throw an exception since
        // "part" is not a valid column in the actual Parquet file
        checkAnswer(
          sqlContext.read.parquet(path).filter("a > 0 and (part = 0 or a > 1)"),
          (2 to 3).map(i => Row(i, i.toString, 1)))
      }
    }
```

We expect the result to be:
```
2,1
3,1
```
But got
```
1,1
2,1
3,1
```

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

Closes #8916 from chenghao-intel/partition_filter.
2015-10-14 16:29:32 -07:00
Reynold Xin 2b5e31c7e9 [SPARK-11113] [SQL] Remove DeveloperApi annotation from private classes.
o.a.s.sql.catalyst and o.a.s.sql.execution are supposed to be private.

Author: Reynold Xin <rxin@databricks.com>

Closes #9121 from rxin/SPARK-11113.
2015-10-14 16:27:43 -07:00
Wenchen Fan 56d7da14ab [SPARK-10104] [SQL] Consolidate different forms of table identifiers
Right now, we have QualifiedTableName, TableIdentifier, and Seq[String] to represent table identifiers. We should only have one form and TableIdentifier is the best one because it provides methods to get table name, database name, return unquoted string, and return quoted string.

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8453 from cloud-fan/table-name.
2015-10-14 16:05:37 -07:00
Wenchen Fan 9a430a027f [SPARK-11068] [SQL] [FOLLOW-UP] move execution listener to util
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9119 from cloud-fan/callback.
2015-10-14 15:08:13 -07:00
Huaxin Gao 7e1308d37f [SPARK-8386] [SQL] add write.mode for insertIntoJDBC when the parm overwrite is false
the fix is for jira https://issues.apache.org/jira/browse/SPARK-8386

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #9042 from huaxingao/spark8386.
2015-10-14 12:31:29 -07:00
Yin Huai ce3f9a8065 [SPARK-11091] [SQL] Change spark.sql.canonicalizeView to spark.sql.nativeView.
https://issues.apache.org/jira/browse/SPARK-11091

Author: Yin Huai <yhuai@databricks.com>

Closes #9103 from yhuai/SPARK-11091.
2015-10-13 18:21:24 -07:00
Wenchen Fan 15ff85b316 [SPARK-11068] [SQL] add callback to query execution
With this feature, we can track the query plan, time cost, exception during query execution for spark users.

Author: Wenchen Fan <cloud0fan@163.com>

Closes #9078 from cloud-fan/callback.
2015-10-13 17:59:32 -07:00
Sun Rui 5e3868ba13 [SPARK-10051] [SPARKR] Support collecting data of StructType in DataFrame
Two points in this PR:

1.    Originally thought was that a named R list is assumed to be a struct in SerDe. But this is problematic because some R functions will implicitly generate named lists that are not intended to be a struct when transferred by SerDe. So SerDe clients have to explicitly mark a names list as struct by changing its class from "list" to "struct".

2.    SerDe is in the Spark Core module, and data of StructType is represented as GenricRow which is defined in Spark SQL module. SerDe can't import GenricRow as in maven build  Spark SQL module depends on Spark Core module. So this PR adds a registration hook in SerDe to allow SQLUtils in Spark SQL module to register its functions for serialization and deserialization of StructType.

Author: Sun Rui <rui.sun@intel.com>

Closes #8794 from sun-rui/SPARK-10051.
2015-10-13 10:02:21 -07:00
Davies Liu d0cc79ccd0 [SPARK-11030] [SQL] share the SQLTab across sessions
The SQLTab will be shared by multiple sessions.

If we create multiple independent SQLContexts (not using newSession()), will still see multiple SQLTabs in the Spark UI.

Author: Davies Liu <davies@databricks.com>

Closes #9048 from davies/sqlui.
2015-10-13 09:57:53 -07:00
Davies Liu 6987c06793 [SPARK-11009] [SQL] fix wrong result of Window function in cluster mode
Currently, All windows function could generate wrong result in cluster sometimes.

The root cause is that AttributeReference is called in executor, then id of it may not be unique than others created in driver.

Here is the script that could reproduce the problem (run in local cluster):
```
from pyspark import SparkContext, HiveContext
from pyspark.sql.window import Window
from pyspark.sql.functions import rowNumber

sqlContext = HiveContext(SparkContext())
sqlContext.setConf("spark.sql.shuffle.partitions", "3")
df =  sqlContext.range(1<<20)
df2 = df.select((df.id % 1000).alias("A"), (df.id / 1000).alias('B'))
ws = Window.partitionBy(df2.A).orderBy(df2.B)
df3 = df2.select("client", "date", rowNumber().over(ws).alias("rn")).filter("rn < 0")
assert df3.count() == 0
```

Author: Davies Liu <davies@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #9050 from davies/wrong_window.
2015-10-13 09:43:33 -07:00
Davies Liu c4da5345a0 [SPARK-10990] [SPARK-11018] [SQL] improve unrolling of complex types
This PR improve the unrolling and read of complex types in columnar cache:
1) Using UnsafeProjection to do serialization of complex types, so they will not be serialized three times (two for actualSize)
2) Copy the bytes from UnsafeRow/UnsafeArrayData to ByteBuffer directly, avoiding the immediate byte[]
3) Using the underlying array in ByteBuffer to create UTF8String/UnsafeRow/UnsafeArrayData without copy.

Combine these optimizations,  we can reduce the unrolling time from 25s to 21s (20% less), reduce the scanning time from 3.5s to 2.5s (28% less).

```
df = sqlContext.read.parquet(path)
t = time.time()
df.cache()
df.count()
print 'unrolling', time.time() - t

for i in range(10):
    t = time.time()
    print df.select("*")._jdf.queryExecution().toRdd().count()
    print time.time() - t
```

The schema is
```
root
 |-- a: struct (nullable = true)
 |    |-- b: long (nullable = true)
 |    |-- c: string (nullable = true)
 |-- d: array (nullable = true)
 |    |-- element: long (containsNull = true)
 |-- e: map (nullable = true)
 |    |-- key: long
 |    |-- value: string (valueContainsNull = true)
```

Now the columnar cache depends on that UnsafeProjection support all the data types (including UDT), this PR also fix that.

Author: Davies Liu <davies@databricks.com>

Closes #9016 from davies/complex2.
2015-10-12 21:12:59 -07:00
Yin Huai 8a354bef55 [SPARK-11042] [SQL] Add a mechanism to ban creating multiple root SQLContexts/HiveContexts in a JVM
https://issues.apache.org/jira/browse/SPARK-11042

Author: Yin Huai <yhuai@databricks.com>

Closes #9058 from yhuai/SPARK-11042.
2015-10-12 13:50:34 -07:00