Commit graph

1725 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