Commit graph

1875 commits

Author SHA1 Message Date
Nong Li a74d743cc7 [SPARK-12640][SQL] Add simple benchmarking utility class and add Parquet scan benchmarks.
[SPARK-12640][SQL] Add simple benchmarking utility class and add Parquet scan benchmarks.

We've run benchmarks ad hoc to measure the scanner performance. We will continue to invest in this
and it makes sense to get these benchmarks into code. This adds a simple benchmarking utility to do
this.

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

Closes #10589 from nongli/spark-12640.
2016-01-06 19:20:43 -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
Marcelo Vanzin b3ba1be3b7 [SPARK-3873][TESTS] Import ordering fixes.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10582 from vanzin/SPARK-3873-tests.
2016-01-05 19:07:39 -08:00
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
thomastechs c82924d564 [SPARK-12533][SQL] hiveContext.table() throws the wrong exception
Avoiding the the No such table exception and throwing analysis exception as per the bug: SPARK-12533

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

Closes #10529 from thomastechs/topic-branch.
2016-01-03 11:09:30 -08:00
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
Herman van Hovell f76ee109d8 [SPARK-8641][SPARK-12455][SQL] Native Spark Window functions - Follow-up (docs & tests)
This PR is a follow-up for PR https://github.com/apache/spark/pull/9819. It adds documentation for the window functions and a couple of NULL tests.

The documentation was largely based on the documentation in (the source of)  Hive and Presto:
* https://prestodb.io/docs/current/functions/window.html
* https://cwiki.apache.org/confluence/display/Hive/LanguageManual+WindowingAndAnalytics

I am not sure if we need to add the licenses of these two projects to the licenses directory. They are both under the ASL. srowen any thoughts?

cc yhuai

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

Closes #10402 from hvanhovell/SPARK-8641-docs.
2015-12-30 16:51:07 -08:00
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
gatorsmile 4f75f785df [SPARK-12564][SQL] Improve missing column AnalysisException
```
org.apache.spark.sql.AnalysisException: cannot resolve 'value' given input columns text;
```

lets put a `:` after `columns` and put the columns in `[]` so that they match the toString of DataFrame.

Author: gatorsmile <gatorsmile@gmail.com>

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

Author: Reynold Xin <rxin@databricks.com>

Closes #10504 from rxin/SPARK-12549.
2015-12-29 16:58:23 -08:00
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
Wenchen Fan 8543997f2d [HOT-FIX] bypass hive test when parse logical plan to json
https://github.com/apache/spark/pull/10311 introduces some rare, non-deterministic flakiness for hive udf tests, see https://github.com/apache/spark/pull/10311#issuecomment-166548851

I can't reproduce it locally, and may need more time to investigate, a quick solution is: bypass hive tests for json serialization.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10430 from cloud-fan/hot-fix.
2015-12-28 11:45:44 -08:00
Cheng Lian 8e23d8db7f [SPARK-12218] Fixes ORC conjunction predicate push down
This PR is a follow-up of PR #10362.

Two major changes:

1.  The fix introduced in #10362 is OK for Parquet, but may disable ORC PPD in many cases

    PR #10362 stops converting an `AND` predicate if any branch is inconvertible.  On the other hand, `OrcFilters` combines all filters into a single big conjunction first and then tries to convert it into ORC `SearchArgument`.  This means, if any filter is inconvertible, no filters can be pushed down.  This PR fixes this issue by finding out all convertible filters first before doing the actual conversion.

    The reason behind the current implementation is mostly due to the limitation of ORC `SearchArgument` builder, which is documented in this PR in detail.

1.  Copied the `AND` predicate fix for ORC from #10362 to avoid merge conflict.

Same as #10362, this PR targets master (2.0.0-SNAPSHOT), branch-1.6, and branch-1.5.

Author: Cheng Lian <lian@databricks.com>

Closes #10377 from liancheng/spark-12218.fix-orc-conjunction-ppd.
2015-12-28 08:48:44 -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
pierre-borckmans 43b2a63900 [SPARK-12477][SQL] - Tungsten projection fails for null values in array fields
Accessing null elements in an array field fails when tungsten is enabled.
It works in Spark 1.3.1, and in Spark > 1.5 with Tungsten disabled.

This PR solves this by checking if the accessed element in the array field is null, in the generated code.

Example:
```
// Array of String
case class AS( as: Seq[String] )
val dfAS = sc.parallelize( Seq( AS ( Seq("a",null,"b") ) ) ).toDF
dfAS.registerTempTable("T_AS")
for (i <- 0 to 2) { println(i + " = " + sqlContext.sql(s"select as[$i] from T_AS").collect.mkString(","))}
```

With Tungsten disabled:
```
0 = [a]
1 = [null]
2 = [b]
```

With Tungsten enabled:
```
0 = [a]
15/12/22 09:32:50 ERROR Executor: Exception in task 7.0 in stage 1.0 (TID 15)
java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.UnsafeRowWriters$UTF8StringWriter.getSize(UnsafeRowWriters.java:90)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.execution.TungstenProject$$anonfun$3$$anonfun$apply$3.apply(basicOperators.scala:90)
	at org.apache.spark.sql.execution.TungstenProject$$anonfun$3$$anonfun$apply$3.apply(basicOperators.scala:88)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
```

Author: pierre-borckmans <pierre.borckmans@realimpactanalytics.com>

Closes #10429 from pierre-borckmans/SPARK-12477_Tungsten-Projection-Null-Element-In-Array.
2015-12-22 23:00:42 -08:00
Liang-Chi Hsieh 50301c0a28 [SPARK-11164][SQL] Add InSet pushdown filter back for Parquet
When the filter is ```"b in ('1', '2')"```, the filter is not pushed down to Parquet. Thanks!

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

Closes #10278 from gatorsmile/parquetFilterNot.
2015-12-23 14:08:29 +08:00
Cheng Lian 86761e10e1 [SPARK-12478][SQL] Bugfix: Dataset fields of product types can't be null
When creating extractors for product types (i.e. case classes and tuples), a null check is missing, thus we always assume input product values are non-null.

This PR adds a null check in the extractor expression for product types. The null check is stripped off for top level product fields, which are mapped to the outermost `Row`s, since they can't be null.

Thanks cloud-fan for helping investigating this issue!

Author: Cheng Lian <lian@databricks.com>

Closes #10431 from liancheng/spark-12478.top-level-null-field.
2015-12-23 10:21:00 +08:00
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
Davies Liu 29cecd4a42 [SPARK-12388] change default compression to lz4
According the benchmark [1], LZ4-java could be 80% (or 30%) faster than Snappy.

After changing the compressor to LZ4, I saw 20% improvement on end-to-end time for a TPCDS query (Q4).

[1] https://github.com/ning/jvm-compressor-benchmark/wiki

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #10342 from davies/lz4.
2015-12-21 14:21:43 -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
Reynold Xin f496031bd2 Bump master version to 2.0.0-SNAPSHOT.
Author: Reynold Xin <rxin@databricks.com>

Closes #10387 from rxin/version-bump.
2015-12-19 15:13:05 -08:00
Kousuke Saruta 6eba655259 [SPARK-12404][SQL] Ensure objects passed to StaticInvoke is Serializable
Now `StaticInvoke` receives `Any` as a object and `StaticInvoke` can be serialized but sometimes the object passed is not serializable.

For example, following code raises Exception because `RowEncoder#extractorsFor` invoked indirectly makes `StaticInvoke`.

```
case class TimestampContainer(timestamp: java.sql.Timestamp)
val rdd = sc.parallelize(1 to 2).map(_ => TimestampContainer(System.currentTimeMillis))
val df = rdd.toDF
val ds = df.as[TimestampContainer]
val rdd2 = ds.rdd                                 <----------------- invokes extractorsFor indirectory
```

I'll add test cases.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Author: Michael Armbrust <michael@databricks.com>

Closes #10357 from sarutak/SPARK-12404.
2015-12-18 14:05:06 -08:00
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
Shixiong Zhu 0370abdfd6 [MINOR] Hide the error logs for 'SQLListenerMemoryLeakSuite'
Hide the error logs for 'SQLListenerMemoryLeakSuite' to avoid noises. Most of changes are space changes.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10363 from zsxwing/hide-log.
2015-12-17 18:18:12 -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
Reynold Xin 554d840a9a Style fix for the previous 3 JDBC filter push down commits. 2015-12-15 22:32:51 -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
hyukjinkwon 28112657ea [SPARK-12236][SQL] JDBC filter tests all pass if filters are not really pushed down
https://issues.apache.org/jira/browse/SPARK-12236
Currently JDBC filters are not tested properly. All the tests pass even if the filters are not pushed down due to Spark-side filtering.

In this PR,
Firstly, I corrected the tests to properly check the pushed down filters by removing Spark-side filtering.
Also, `!=` was being tested which is actually not pushed down. So I removed them.
Lastly, I moved the `stripSparkFilter()` function to `SQLTestUtils` as this functions would be shared for all tests for pushed down filters. This function would be also shared with ORC datasource as the filters for that are also not being tested properly.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #10221 from HyukjinKwon/SPARK-12236.
2015-12-15 17:02:14 -08:00
Nong Li 86ea64dd14 [SPARK-12271][SQL] Improve error message when Dataset.as[ ] has incompatible schemas.
Author: Nong Li <nong@databricks.com>

Closes #10260 from nongli/spark-11271.
2015-12-15 16:55:58 -08:00
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
Davies Liu c119a34d1e [SPARK-12258] [SQL] passing null into ScalaUDF (follow-up)
This is a follow-up PR for #10259

Author: Davies Liu <davies@databricks.com>

Closes #10266 from davies/null_udf2.
2015-12-11 11:15:53 -08:00
Davies Liu b1b4ee7f35 [SPARK-12258][SQL] passing null into ScalaUDF
Check nullability and passing them into ScalaUDF.

Closes #10249

Author: Davies Liu <davies@databricks.com>

Closes #10259 from davies/udf_null.
2015-12-10 17:22:18 -08:00
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
Mark Grover 2166c2a750 [SPARK-11796] Fix httpclient and httpcore depedency issues related to docker-client
This commit fixes dependency issues which prevented the Docker-based JDBC integration tests from running in the Maven build.

Author: Mark Grover <mgrover@cloudera.com>

Closes #9876 from markgrover/master_docker.
2015-12-09 18:39:36 -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
hyukjinkwon f6883bb7af [SPARK-11676][SQL] Parquet filter tests all pass if filters are not really pushed down
Currently Parquet predicate tests all pass even if filters are not pushed down or this is disabled.

In this PR, For checking evaluating filters, Simply it makes the expression from `expression.Filter` and then try to create filters just like Spark does.

For checking the results, this manually accesses to the child rdd (of `expression.Filter`) and produces the results which should be filtered properly, and then compares it to expected values.

Now, if filters are not pushed down or this is disabled, this throws exceptions.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9659 from HyukjinKwon/SPARK-11676.
2015-12-09 15:15:30 +08:00
Michael Armbrust 3959489423 [SPARK-12069][SQL] Update documentation with Datasets
Author: Michael Armbrust <michael@databricks.com>

Closes #10060 from marmbrus/docs.
2015-12-08 15:58:35 -08:00
Andrew Ray 4bcb894948 [SPARK-12205][SQL] Pivot fails Analysis when aggregate is UnresolvedFunction
Delays application of ResolvePivot until all aggregates are resolved to prevent problems with UnresolvedFunction and adds unit test

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

Closes #10202 from aray/sql-pivot-unresolved-function.
2015-12-08 10:52:17 -08:00
gatorsmile 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
gatorsmile c0b13d5565 [SPARK-12195][SQL] Adding BigDecimal, Date and Timestamp into Encoder
This PR is to add three more data types into Encoder, including `BigDecimal`, `Date` and `Timestamp`.

marmbrus cloud-fan rxin Could you take a quick look at these three types? Not sure if it can be merged to 1.6. Thank you very much!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10188 from gatorsmile/dataTypesinEncoder.
2015-12-08 10:15:58 -08:00
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 fd95eeaf49 [SPARK-11954][SQL] Encoder for JavaBeans
create java version of `constructorFor` and `extractorFor` in `JavaTypeInference`

Author: Wenchen Fan <wenchen@databricks.com>

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

Closes #9937 from cloud-fan/pojo.
2015-12-01 10:35:12 -08:00
Wenchen Fan 9df24624af [SPARK-11856][SQL] add type cast if the real type is different but compatible with encoder schema
When we build the `fromRowExpression` for an encoder, we set up a lot of "unresolved" stuff and lost the required data type, which may lead to runtime error if the real type doesn't match the encoder's schema.
For example, we build an encoder for `case class Data(a: Int, b: String)` and the real type is `[a: int, b: long]`, then we will hit runtime error and say that we can't construct class `Data` with int and long, because we lost the information that `b` should be a string.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9840 from cloud-fan/err-msg.
2015-12-01 10:24:53 -08:00
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
Liang-Chi Hsieh c87531b765 [SPARK-11949][SQL] Set field nullable property for GroupingSets to get correct results for null values
JIRA: https://issues.apache.org/jira/browse/SPARK-11949

The result of cube plan uses incorrect schema. The schema of cube result should set nullable property to true because the grouping expressions will have null values.

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

Closes #10038 from viirya/fix-cube.
2015-12-01 07:44:22 -08:00
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
gatorsmile 149cd692ee [SPARK-12028] [SQL] get_json_object returns an incorrect result when the value is null literals
When calling `get_json_object` for the following two cases, both results are `"null"`:

```scala
    val tuple: Seq[(String, String)] = ("5", """{"f1": null}""") :: Nil
    val df: DataFrame = tuple.toDF("key", "jstring")
    val res = df.select(functions.get_json_object($"jstring", "$.f1")).collect()
```
```scala
    val tuple2: Seq[(String, String)] = ("5", """{"f1": "null"}""") :: Nil
    val df2: DataFrame = tuple2.toDF("key", "jstring")
    val res3 = df2.select(functions.get_json_object($"jstring", "$.f1")).collect()
```

Fixed the problem and also added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10018 from gatorsmile/get_json_object.
2015-11-27 22:44:08 -08:00
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
Wenchen Fan 1a5baaa651 [SPARK-11894][SQL] fix isNull for GetInternalRowField
We should use `InternalRow.isNullAt` to check if the field is null before calling `InternalRow.getXXX`

Thanks gatorsmile who discovered this bug.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9904 from cloud-fan/null.
2015-11-23 10:13:59 -08:00
Xiu Guo 94ce65dfcb [SPARK-11628][SQL] support column datatype of char(x) to recognize HiveChar
Can someone review my code to make sure I'm not missing anything? Thanks!

Author: Xiu Guo <xguo27@gmail.com>
Author: Xiu Guo <guoxi@us.ibm.com>

Closes #9612 from xguo27/SPARK-11628.
2015-11-23 08:53:40 -08:00
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 47815878ad [HOTFIX] Fix Java Dataset Tests 2015-11-20 16:03:14 -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
Nong Li 9ed4ad4265 [SPARK-11724][SQL] Change casting between int and timestamp to consistently treat int in seconds.
Hive has since changed this behavior as well. https://issues.apache.org/jira/browse/HIVE-3454

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

Closes #9685 from nongli/spark-11724.
2015-11-20 14:19:34 -08:00
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
Andrew Ray 37cff1b1a7 [SPARK-11275][SQL] Incorrect results when using rollup/cube
Fixes bug with grouping sets (including cube/rollup) where aggregates that included grouping expressions would return the wrong (null) result.

Also simplifies the analyzer rule a bit and leaves column pruning to the optimizer.

Added multiple unit tests to DataFrameAggregateSuite and verified it passes hive compatibility suite:
```
build/sbt -Phive -Dspark.hive.whitelist='groupby.*_grouping.*' 'test-only org.apache.spark.sql.hive.execution.HiveCompatibilitySuite'
```

This is an alternative to pr https://github.com/apache/spark/pull/9419 but I think its better as it simplifies the analyzer rule instead of adding another special case to it.

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

Closes #9815 from aray/groupingset-agg-fix.
2015-11-19 15:11:30 -08:00
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
Reynold Xin e61367b9f9 [SPARK-11833][SQL] Add Java tests for Kryo/Java Dataset encoders
Also added some nicer error messages for incompatible types (private types and primitive types) for Kryo/Java encoder.

Author: Reynold Xin <rxin@databricks.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

Closes #9602 from marmbrus/dataset-replClasses.
2015-11-18 16:48:09 -08:00
Reynold Xin 5df08949f5 [SPARK-11810][SQL] Java-based encoder for opaque types in Datasets.
This patch refactors the existing Kryo encoder expressions and adds support for Java serialization.

Author: Reynold Xin <rxin@databricks.com>

Closes #9802 from rxin/SPARK-11810.
2015-11-18 15:42:07 -08:00
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
JihongMa 09ad9533d5 [SPARK-11720][SQL][ML] Handle edge cases when count = 0 or 1 for Stats function
return Double.NaN for mean/average when count == 0 for all numeric types that is converted to Double, Decimal type continue to return null.

Author: JihongMa <linlin200605@gmail.com>

Closes #9705 from JihongMA/SPARK-11720.
2015-11-18 13:03:37 -08:00
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 33b8373334 [SPARK-11725][SQL] correctly handle null inputs for UDF
If user use primitive parameters in UDF, there is no way for him to do the null-check for primitive inputs, so we are assuming the primitive input is null-propagatable for this case and return null if the input is null.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9770 from cloud-fan/udf.
2015-11-18 10:23:12 -08:00
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
mayuanwen e8833dd12c [SPARK-11679][SQL] Invoking method " apply(fields: java.util.List[StructField])" in "StructType" gets ClassCastException
In the previous method, fields.toArray will cast java.util.List[StructField] into Array[Object] which can not cast into Array[StructField], thus when invoking this method will throw "java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to [Lorg.apache.spark.sql.types.StructField;"
I directly cast java.util.List[StructField] into Array[StructField]  in this patch.

Author: mayuanwen <mayuanwen@qiyi.com>

Closes #9649 from jackieMaKing/Spark-11679.
2015-11-17 11:15:46 -08:00
Kevin Yu e01865af0d [SPARK-11447][SQL] change NullType to StringType during binaryComparison between NullType and StringType
During executing PromoteStrings rule, if one side of binaryComparison is StringType and the other side is not StringType, the current code will promote(cast) the StringType to DoubleType, and if the StringType doesn't contain the numbers, it will get null value. So if it is doing <=> (NULL-safe equal) with Null, it will not filter anything, caused the problem reported by this jira.

I proposal to the changes through this PR, can you review my code changes ?

This problem only happen for <=>, other operators works fine.

scala> val filteredDF = df.filter(df("column") > (new Column(Literal(null))))
filteredDF: org.apache.spark.sql.DataFrame = [column: string]

scala> filteredDF.show
+------+
|column|
+------+
+------+

scala> val filteredDF = df.filter(df("column") === (new Column(Literal(null))))
filteredDF: org.apache.spark.sql.DataFrame = [column: string]

scala> filteredDF.show
+------+
|column|
+------+
+------+

scala> df.registerTempTable("DF")

scala> sqlContext.sql("select * from DF where 'column' = NULL")
res27: org.apache.spark.sql.DataFrame = [column: string]

scala> res27.show
+------+
|column|
+------+
+------+

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

Closes #9720 from kevinyu98/working_on_spark-11447.
2015-11-16 22:54:29 -08:00
hyukjinkwon 75d2020731 [SPARK-11694][FOLLOW-UP] Clean up imports, use a common function for metadata and add a test for FIXED_LEN_BYTE_ARRAY
As discussed https://github.com/apache/spark/pull/9660 https://github.com/apache/spark/pull/9060, I cleaned up unused imports, added a test for fixed-length byte array and used a common function for writing metadata for Parquet.

For the test for fixed-length byte array, I have tested and checked the encoding types with [parquet-tools](https://github.com/Parquet/parquet-mr/tree/master/parquet-tools).

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9754 from HyukjinKwon/SPARK-11694-followup.
2015-11-17 14:35:00 +08:00
Reynold Xin fbad920dbf [SPARK-11768][SPARK-9196][SQL] Support now function in SQL (alias for current_timestamp).
This patch adds an alias for current_timestamp (now function).

Also fixes SPARK-9196 to re-enable the test case for current_timestamp.

Author: Reynold Xin <rxin@databricks.com>

Closes #9753 from rxin/SPARK-11768.
2015-11-16 20:47:46 -08:00
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
Bartlomiej Alberski 31296628ac [SPARK-11553][SQL] Primitive Row accessors should not convert null to default value
Invocation of getters for type extending AnyVal returns default value (if field value is null) instead of throwing NPE. Please check comments for SPARK-11553 issue for more details.

Author: Bartlomiej Alberski <bartlomiej.alberski@allegrogroup.com>

Closes #9642 from alberskib/bugfix/SPARK-11553.
2015-11-16 15:14:38 -08:00
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
Yin Huai 3e2e1873b2 [SPARK-11738] [SQL] Making ArrayType orderable
https://issues.apache.org/jira/browse/SPARK-11738

Author: Yin Huai <yhuai@databricks.com>

Closes #9718 from yhuai/makingArrayOrderable.
2015-11-15 13:59:59 -08:00
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
Yin Huai d83c2f9f0b [SPARK-11736][SQL] Add monotonically_increasing_id to function registry.
https://issues.apache.org/jira/browse/SPARK-11736

Author: Yin Huai <yhuai@databricks.com>

Closes #9703 from yhuai/MonotonicallyIncreasingID.
2015-11-14 21:04:18 -08:00
hyukjinkwon 139c15b624 [SPARK-11694][SQL] Parquet logical types are not being tested properly
All the physical types are properly tested at `ParquetIOSuite` but logical type mapping is not being tested.

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

Closes #9660 from HyukjinKwon/SPARK-11694.
2015-11-14 18:36:01 +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
Daoyuan Wang 39b1e36fbc [SPARK-11396] [SQL] add native implementation of datetime function to_unix_timestamp
`to_unix_timestamp` is the deterministic version of `unix_timestamp`, as it accepts at least one parameters.

Since the behavior here is quite similar to `unix_timestamp`, I think the dataframe API is not necessary here.

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

Closes #9347 from adrian-wang/to_unix_timestamp.
2015-11-11 20:36:21 -08:00
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
Josh Rosen 529a1d3380 [SPARK-6152] Use shaded ASM5 to support closure cleaning of Java 8 compiled classes
This patch modifies Spark's closure cleaner (and a few other places) to use ASM 5, which is necessary in order to support cleaning of closures that were compiled by Java 8.

In order to avoid ASM dependency conflicts, Spark excludes ASM from all of its dependencies and uses a shaded version of ASM 4 that comes from `reflectasm` (see [SPARK-782](https://issues.apache.org/jira/browse/SPARK-782) and #232). This patch updates Spark to use a shaded version of ASM 5.0.4 that was published by the Apache XBean project; the POM used to create the shaded artifact can be found at https://github.com/apache/geronimo-xbean/blob/xbean-4.4/xbean-asm5-shaded/pom.xml.

http://movingfulcrum.tumblr.com/post/80826553604/asm-framework-50-the-missing-migration-guide was a useful resource while upgrading the code to use the new ASM5 opcodes.

I also added a new regression tests in the `java8-tests` subproject; the existing tests were insufficient to catch this bug, which only affected Scala 2.11 user code which was compiled targeting Java 8.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9512 from JoshRosen/SPARK-6152.
2015-11-11 11:16:39 -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
Davies Liu 521b3cae11 [SPARK-11598] [SQL] enable tests for ShuffledHashOuterJoin
Author: Davies Liu <davies@databricks.com>

Closes #9573 from davies/join_condition.
2015-11-09 23:28:32 -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
hyukjinkwon 9565c246ea [SPARK-9557][SQL] Refactor ParquetFilterSuite and remove old ParquetFilters code
Actually this was resolved by https://github.com/apache/spark/pull/8275.

But I found the JIRA issue for this is not marked as resolved since the PR above was made for another issue but the PR above resolved both.

I commented that this is resolved by the PR above; however, I opened this PR as I would like to just add
a little bit of corrections.

In the previous PR, I refactored the test by not reducing just collecting filters; however, this would not test  properly `And` filter (which is not given to the tests). I unintentionally changed this from the original way (before being refactored).

In this PR, I just followed the original way to collect filters by reducing.

I would like to close this if this PR is inappropriate and somebody would like this deal with it in the separate PR related with this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9554 from HyukjinKwon/SPARK-9557.
2015-11-09 15:20:50 -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
Herman van Hovell 30c8ba71a7 [SPARK-11451][SQL] Support single distinct count on multiple columns.
This PR adds support for multiple column in a single count distinct aggregate to the new aggregation path.

cc yhuai

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

Closes #9409 from hvanhovell/SPARK-11451.
2015-11-08 11:06:10 -08:00
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
Imran Rashid 49f1a82037 [SPARK-10116][CORE] XORShiftRandom.hashSeed is random in high bits
https://issues.apache.org/jira/browse/SPARK-10116

This is really trivial, just happened to notice it -- if `XORShiftRandom.hashSeed` is really supposed to have random bits throughout (as the comment implies), it needs to do something for the conversion to `long`.

mengxr mkolod

Author: Imran Rashid <irashid@cloudera.com>

Closes #8314 from squito/SPARK-10116.
2015-11-06 20:06:24 +00:00
Yin Huai 8211aab079 [SPARK-9858][SQL] Add an ExchangeCoordinator to estimate the number of post-shuffle partitions for aggregates and joins (follow-up)
https://issues.apache.org/jira/browse/SPARK-9858

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

Author: Yin Huai <yhuai@databricks.com>

Closes #9453 from yhuai/numReducer-followUp.
2015-11-06 11:13:51 -08:00
Cheng Lian c048929c6a [SPARK-10978][SQL][FOLLOW-UP] More comprehensive tests for PR #9399
This PR adds test cases that test various column pruning and filter push-down cases.

Author: Cheng Lian <lian@databricks.com>

Closes #9468 from liancheng/spark-10978.follow-up.
2015-11-06 11:11:36 -08:00
Liang-Chi Hsieh 574141a298 [SPARK-9162] [SQL] Implement code generation for ScalaUDF
JIRA: https://issues.apache.org/jira/browse/SPARK-9162

Currently ScalaUDF extends CodegenFallback and doesn't provide code generation implementation. This path implements code generation for ScalaUDF.

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

Closes #9270 from viirya/scalaudf-codegen.
2015-11-06 10:52:04 -08:00
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