Commit graph

3802 commits

Author SHA1 Message Date
Xiao Li f7cf2096fd [SPARK-20941][SQL] Fix SubqueryExec Reuse
### What changes were proposed in this pull request?
Before this PR, Subquery reuse does not work. Below are three issues:
- Subquery reuse does not work.
- It is sharing the same `SQLConf` (`spark.sql.exchange.reuse`) with the one for Exchange Reuse.
- No test case covers the rule Subquery reuse.

This PR is to fix the above three issues.
- Ignored the physical operator `SubqueryExec` when comparing two plans.
- Added a dedicated conf `spark.sql.subqueries.reuse` for controlling Subquery Reuse
- Added a test case for verifying the behavior

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18169 from gatorsmile/subqueryReuse.
2017-06-01 09:52:18 -07:00
Yuming Wang 6d05c1c1da [SPARK-20910][SQL] Add build-in SQL function - UUID
## What changes were proposed in this pull request?

Add build-int SQL function - UUID.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18136 from wangyum/SPARK-20910.
2017-06-01 16:15:24 +09:00
Shixiong Zhu 2bc3272880 [SPARK-20894][SS] Resolve the checkpoint location in driver and use the resolved path in state store
## What changes were proposed in this pull request?

When the user runs a Structured Streaming query in a cluster, if the driver uses the local file system, StateStore running in executors will throw a file-not-found exception. However, the current error is not obvious.

This PR makes StreamExecution resolve the path in driver and uses the full path including the scheme part (such as `hdfs:/`, `file:/`) in StateStore.

Then if the above error happens, StateStore will throw an error with this full path which starts with `file:/`, and it makes this error obvious: the checkpoint location is on the local file system.

One potential minor issue is that the user cannot use different default file system settings in driver and executors (e.g., use a public HDFS address in driver and a private HDFS address in executors) after this change. However, since the batch query also has this issue (See 4bb6a53ebd/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L402)), it doesn't make things worse.

## How was this patch tested?

The new added test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18149 from zsxwing/SPARK-20894.
2017-05-31 17:24:37 -07:00
gatorsmile de934e6718 [SPARK-19236][SQL][FOLLOW-UP] Added createOrReplaceGlobalTempView method
### What changes were proposed in this pull request?
This PR does the following tasks:
- Added  since
- Added the Python API
- Added test cases

### How was this patch tested?
Added test cases to both Scala and Python

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18147 from gatorsmile/createOrReplaceGlobalTempView.
2017-05-31 11:38:43 -07:00
Liu Shaohui d0f36bcb10 [SPARK-20633][SQL] FileFormatWriter should not wrap FetchFailedException
## What changes were proposed in this pull request?

Explicitly handle the FetchFailedException in FileFormatWriter, so it does not get wrapped.

Note that this is no longer strictly necessary after SPARK-19276, but it improves error messages and also will help avoid others stumbling across this in the future.

## How was this patch tested?

Existing unit tests.

Closes https://github.com/apache/spark/pull/17893

Author: Liu Shaohui <liushaohui@xiaomi.com>

Closes #18145 from squito/SPARK-20633.
2017-05-31 10:53:31 -05:00
Jacek Laskowski beed5e20af [DOCS][MINOR] Scaladoc fixes (aka typo hunting)
## What changes were proposed in this pull request?

Minor changes to scaladoc

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18074 from jaceklaskowski/scaladoc-fixes.
2017-05-31 11:24:37 +01:00
Wenchen Fan 1f5dddffa3 Revert "[SPARK-20392][SQL] Set barrier to prevent re-entering a tree"
This reverts commit 8ce0d8ffb6.
2017-05-30 21:14:55 -07:00
Wenchen Fan 10e526e7e6 [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL UI tab
## What changes were proposed in this pull request?

Currently the `DataFrameWriter` operations have several problems:

1. non-file-format data source writing action doesn't show up in the SQL tab in Spark UI
2. file-format data source writing action shows a scan node in the SQL tab, without saying anything about writing. (streaming also have this issue, but not fixed in this PR)
3. Spark SQL CLI actions don't show up in the SQL tab.

This PR fixes all of them, by refactoring the `ExecuteCommandExec` to make it have children.

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

## How was this patch tested?

existing tests.

Also test the UI manually. For a simple command: `Seq(1 -> "a").toDF("i", "j").write.parquet("/tmp/qwe")`

before this PR:
<img width="266" alt="qq20170523-035840 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326050/24e18ba2-3f6c-11e7-8817-6dd275bf6ac5.png">
after this PR:
<img width="287" alt="qq20170523-035708 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326054/2ad7f460-3f6c-11e7-8053-d68325beb28f.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18064 from cloud-fan/execution.
2017-05-30 20:12:32 -07:00
Tathagata Das fa757ee1d4 [SPARK-20883][SPARK-20376][SS] Refactored StateStore APIs and added conf to choose implementation
## What changes were proposed in this pull request?

A bunch of changes to the StateStore APIs and implementation.
Current state store API has a bunch of problems that causes too many transient objects causing memory pressure.

- `StateStore.get(): Option` forces creation of Some/None objects for every get. Changed this to return the row or null.
- `StateStore.iterator(): (UnsafeRow, UnsafeRow)` forces creation of new tuple for each record returned. Changed this to return a UnsafeRowTuple which can be reused across records.
- `StateStore.updates()` requires the implementation to keep track of updates, while this is used minimally (only by Append mode in streaming aggregations). Removed updates() and updated StateStoreSaveExec accordingly.
- `StateStore.filter(condition)` and `StateStore.remove(condition)` has been merge into a single API `getRange(start, end)` which allows a state store to do optimized range queries (i.e. avoid full scans). Stateful operators have been updated accordingly.
- Removed a lot of unnecessary row copies Each operator copied rows before calling StateStore.put() even if the implementation does not require it to be copied. It is left up to the implementation on whether to copy the row or not.

Additionally,
- Added a name to the StateStoreId so that each operator+partition can use multiple state stores (different names)
- Added a configuration that allows the user to specify which implementation to use.
- Added new metrics to understand the time taken to update keys, remove keys and commit all changes to the state store. These metrics will be visible on the plan diagram in the SQL tab of the UI.
- Refactored unit tests such that they can be reused to test any implementation of StateStore.

## How was this patch tested?
Old and new unit tests

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

Closes #18107 from tdas/SPARK-20376.
2017-05-30 15:33:06 -07:00
Josh Rosen 798a04fd76 HOTFIX: fix Scalastyle break introduced in 4d57981cfb 2017-05-30 12:22:23 -07:00
Arman 4d57981cfb [SPARK-19236][CORE] Added createOrReplaceGlobalTempView method
## What changes were proposed in this pull request?

Added the createOrReplaceGlobalTempView method for dataset

Author: Arman <arman.yazdani.10@gmail.com>

Closes #16598 from arman1371/patch-1.
2017-05-30 11:09:21 -07:00
Liang-Chi Hsieh 35b644bd03 [SPARK-20916][SQL] Improve error message for unaliased subqueries in FROM clause
## What changes were proposed in this pull request?

We changed the parser to reject unaliased subqueries in the FROM clause in SPARK-20690. However, the error message that we now give isn't very helpful:

    scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
    org.apache.spark.sql.catalyst.parser.ParseException:
    mismatched input 'FROM' expecting {<EOF>, 'WHERE', 'GROUP', 'ORDER', 'HAVING', 'LIMIT', 'LATERAL', 'WINDOW', 'UNION', 'EXCEPT', 'MINUS', 'INTERSECT', 'SORT', 'CLUSTER', 'DISTRIBUTE'}(line 1, pos 9)

We should modify the parser to throw a more clear error for such queries:

    scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
    org.apache.spark.sql.catalyst.parser.ParseException:
    The unaliased subqueries in the FROM clause are not supported.(line 1, pos 14)

## How was this patch tested?

Modified existing tests to reflect this change.

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

Closes #18141 from viirya/SPARK-20916.
2017-05-30 06:28:43 -07:00
Yuming Wang d797ed0ef1 [SPARK-20909][SQL] Add build-int SQL function - DAYOFWEEK
## What changes were proposed in this pull request?

Add build-int SQL function - DAYOFWEEK

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18134 from wangyum/SPARK-20909.
2017-05-30 15:40:50 +09:00
Kazuaki Ishizaki c9749068ec [SPARK-20907][TEST] Use testQuietly for test suites that generate long log output
## What changes were proposed in this pull request?

Supress console output by using `testQuietly` in test suites

## How was this patch tested?

Tested by `"SPARK-19372: Filter can be executed w/o generated code due to JVM code size limit"` in `DataFrameSuite`

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

Closes #18135 from kiszk/SPARK-20907.
2017-05-29 12:17:14 -07:00
Kazuaki Ishizaki ef9fd920c3 [SPARK-20750][SQL] Built-in SQL Function Support - REPLACE
## What changes were proposed in this pull request?

This PR adds built-in SQL function `(REPLACE(<string_expression>, <search_string> [, <replacement_string>])`

`REPLACE()` return that string that is replaced all occurrences with given string.

## How was this patch tested?

added new test suites

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

Closes #18047 from kiszk/SPARK-20750.
2017-05-29 11:47:31 -07:00
Tejas Patil f9b59abeae [SPARK-20758][SQL] Add Constant propagation optimization
## What changes were proposed in this pull request?

See class doc of `ConstantPropagation` for the approach used.

## How was this patch tested?

- Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17993 from tejasapatil/SPARK-20758_const_propagation.
2017-05-29 12:21:34 +02:00
Takeshi Yamamuro 24d34281d7 [SPARK-20841][SQL] Support table column aliases in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support table column aliases in FROM clause.

## How was this patch tested?
Added tests in `PlanParserSuite`,  `SQLQueryTestSuite`, and `PlanParserSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18079 from maropu/SPARK-20841.
2017-05-28 13:23:18 -07:00
liuxian 3969a8078e [SPARK-20876][SQL] If the input parameter is float type for ceil or floor,the result is not we expected
## What changes were proposed in this pull request?

spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12345
For this case, the result we expected is `12346`
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12345
For this case, the result we expected is `-12346`

Because in `Ceil` or `Floor`, `inputTypes` has no FloatType, so it is converted to LongType.
## How was this patch tested?

After the modification:
spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12346
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12346

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18103 from 10110346/wip-lx-0525-1.
2017-05-27 16:23:45 -07:00
Wenchen Fan 08ede46b89 [SPARK-20897][SQL] cached self-join should not fail
## What changes were proposed in this pull request?

The failed test case is, we have a `SortMergeJoinExec` for a self-join, which means we have a `ReusedExchange` node in the query plan. It works fine without caching, but throws an exception in `SortMergeJoinExec.outputPartitioning` if we cache it.

The root cause is, `ReusedExchange` doesn't propagate the output partitioning from its child, so in `SortMergeJoinExec.outputPartitioning` we create `PartitioningCollection` with a hash partitioning and an unknown partitioning, and fail.

This bug is mostly fine, because inserting the `ReusedExchange` is the last step to prepare the physical plan, we won't call `SortMergeJoinExec.outputPartitioning` anymore after this.

However, if the dataframe is cached, the physical plan of it becomes `InMemoryTableScanExec`, which contains another physical plan representing the cached query, and it has gone through the entire planning phase and may have `ReusedExchange`. Then the planner call `InMemoryTableScanExec.outputPartitioning`, which then calls `SortMergeJoinExec.outputPartitioning` and trigger this bug.

## How was this patch tested?

a new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18121 from cloud-fan/bug.
2017-05-27 16:16:51 -07:00
setjet c491e2ed90 [SPARK-20873][SQL] Improve the error message for unsupported Column Type
## What changes were proposed in this pull request?
Upon encountering an invalid columntype, the column type object is printed, rather than the type.
This  change improves this by outputting its name.

## How was this patch tested?
Added a simple  unit test to verify the contents of the raised exception

Author: setjet <rubenljanssen@gmail.com>

Closes #18097 from setjet/spark-20873.
2017-05-26 15:07:28 -07:00
Michael Armbrust d935e0a9d9 [SPARK-20844] Remove experimental from Structured Streaming APIs
Now that Structured Streaming has been out for several Spark release and has large production use cases, the `Experimental` label is no longer appropriate.  I've left `InterfaceStability.Evolving` however, as I think we may make a few changes to the pluggable Source & Sink API in Spark 2.3.

Author: Michael Armbrust <michael@databricks.com>

Closes #18065 from marmbrus/streamingGA.
2017-05-26 13:33:23 -07:00
Liang-Chi Hsieh 8ce0d8ffb6 [SPARK-20392][SQL] Set barrier to prevent re-entering a tree
## What changes were proposed in this pull request?

It is reported that there is performance downgrade when applying ML pipeline for dataset with many columns but few rows.

A big part of the performance downgrade comes from some operations (e.g., `select`) on DataFrame/Dataset which re-create new DataFrame/Dataset with a new `LogicalPlan`. The cost can be ignored in the usage of SQL, normally.

However, it's not rare to chain dozens of pipeline stages in ML. When the query plan grows incrementally during running those stages, the total cost spent on re-creation of DataFrame grows too. In particular, the `Analyzer` will go through the big query plan even most part of it is analyzed.

By eliminating part of the cost, the time to run the example code locally is reduced from about 1min to about 30 secs.

In particular, the time applying the pipeline locally is mostly spent on calling transform of the 137 `Bucketizer`s. Before the change, each call of `Bucketizer`'s transform can cost about 0.4 sec. So the total time spent on all `Bucketizer`s' transform is about 50 secs. After the change, each call only costs about 0.1 sec.

<del>We also make `boundEnc` as lazy variable to reduce unnecessary running time.</del>

### Performance improvement

The codes and datasets provided by Barry Becker to re-produce this issue and benchmark can be found on the JIRA.

Before this patch: about 1 min
After this patch: about 20 secs

## How was this patch tested?

Existing tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #17770 from viirya/SPARK-20392.
2017-05-26 13:45:55 +08:00
setjet 2dbe0c5288 [SPARK-20775][SQL] Added scala support from_json
## What changes were proposed in this pull request?

from_json function required to take in a java.util.Hashmap. For other functions, a java wrapper is provided which casts a java hashmap to a scala map. Only a java function is provided in this case, forcing scala users to pass in a java.util.Hashmap.

Added the missing wrapper.

## How was this patch tested?
Added a unit test for passing in a scala map

Author: setjet <rubenljanssen@gmail.com>

Closes #18094 from setjet/spark-20775.
2017-05-26 10:21:39 +08:00
hyukjinkwon e9f983df27 [SPARK-19707][SPARK-18922][TESTS][SQL][CORE] Fix test failures/the invalid path check for sc.addJar on Windows
## What changes were proposed in this pull request?

This PR proposes two things:

- A follow up for SPARK-19707 (Improving the invalid path check for sc.addJar on Windows as well).

```
org.apache.spark.SparkContextSuite:
 - add jar with invalid path *** FAILED *** (32 milliseconds)
   2 was not equal to 1 (SparkContextSuite.scala:309)
   ...
```

- Fix path vs URI related test failures on Windows.

```
org.apache.spark.storage.LocalDirsSuite:
 - SPARK_LOCAL_DIRS override also affects driver *** FAILED *** (0 milliseconds)
   new java.io.File("/NONEXISTENT_PATH").exists() was true (LocalDirsSuite.scala:50)
   ...

 - Utils.getLocalDir() throws an exception if any temporary directory cannot be retrieved *** FAILED *** (15 milliseconds)
   Expected exception java.io.IOException to be thrown, but no exception was thrown. (LocalDirsSuite.scala:64)
   ...
```

```
org.apache.spark.sql.hive.HiveSchemaInferenceSuite:
 - orc: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-dae61ab3-a851-4dd3-bf4e-be97c501f254
   ...

 - parquet: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fa3aff89-a66e-4376-9a37-2a9b87596939
   ...

 - orc: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (141 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fb464e59-b049-481b-9c75-f53295c9fc2c
   ...

 - parquet: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (125 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-9487568e-80a4-42b3-b0a5-d95314c4ccbc
   ...

 - orc: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (156 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-0d2dfa45-1b0f-4958-a8be-1074ed0135a
   ...

 - parquet: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (547 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-6d95d64e-613e-4a59-a0f6-d198c5aa51ee
   ...
```

```
org.apache.spark.sql.execution.command.DDLSuite:
 - create temporary view using *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-3881d9ca-561b-488d-90b9-97587472b853	mp;
   ...

 - insert data to a data source table which has a non-existing location should succeed *** FAILED *** (109 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-4cad3d19-6085-4b75-b407-fe5e9d21df54 did not equal file:///C:/projects/spark/target/tmp/spark-4cad3d19-6085-4b75-b407-fe5e9d21df54 (DDLSuite.scala:1869)
   ...

 - insert into a data source table with a non-existing partition location should succeed *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d did not equal file:///C:/projects/spark/target/tmp/spark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d (DDLSuite.scala:1910)
   ...

 - read data from a data source table which has a non-existing location should succeed *** FAILED *** (93 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-f8c281e2-08c2-4f73-abbf-f3865b702c34 did not equal file:///C:/projects/spark/target/tmp/spark-f8c281e2-08c2-4f73-abbf-f3865b702c34 (DDLSuite.scala:1937)
   ...

 - read data from a data source table with non-existing partition location should succeed *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - create datasource table with a non-existing location *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-387316ae-070c-4e78-9b78-19ebf7b29ec8 did not equal file:///C:/projects/spark/target/tmp/spark-387316ae-070c-4e78-9b78-19ebf7b29ec8 (DDLSuite.scala:1982)
   ...

 - CTAS for external data source table with a non-existing location *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - CTAS for external data source table with a existed location *** FAILED *** (15 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a b *** FAILED *** (125 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a:b *** FAILED *** (143 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a%b *** FAILED *** (109 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a,b *** FAILED *** (109 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - location uri contains a b for datasource table *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-5739cda9-b702-4e14-932c-42e8c4174480a%20b did not equal file:///C:/projects/spark/target/tmp/spark-5739cda9-b702-4e14-932c-42e8c4174480/a%20b (DDLSuite.scala:2084)
   ...

 - location uri contains a:b for datasource table *** FAILED *** (78 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-9bdd227c-840f-4f08-b7c5-4036638f098da:b did not equal file:///C:/projects/spark/target/tmp/spark-9bdd227c-840f-4f08-b7c5-4036638f098d/a:b (DDLSuite.scala:2084)
   ...

 - location uri contains a%b for datasource table *** FAILED *** (78 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-62bb5f1d-fa20-460a-b534-cb2e172a3640a%25b did not equal file:///C:/projects/spark/target/tmp/spark-62bb5f1d-fa20-460a-b534-cb2e172a3640/a%25b (DDLSuite.scala:2084)
   ...

 - location uri contains a b for database *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - location uri contains a:b for database *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - location uri contains a%b for database *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...
```

```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
 - create hive table with a non-existing location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - CTAS for external hive table with a non-existing location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - CTAS for external hive table with a existed location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of parquet table containing a b *** FAILED *** (156 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a:b *** FAILED *** (94 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a%b *** FAILED *** (125 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a,b *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of hive table containing a b *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a:b *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a%b *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a,b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a:b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a%b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...
```

```
org.apache.spark.sql.sources.PathOptionSuite:
 - path option also exist for write path *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc did not equal file:///C:/projects/spark/target/tmp/spark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc (PathOptionSuite.scala:98)
   ...
```

```
org.apache.spark.sql.CachedTableSuite:
 - SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer to this table *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...
```

```
org.apache.spark.sql.execution.DataSourceScanExecRedactionSuite:
 - treeString is redacted *** FAILED *** (250 milliseconds)
   "file:/C:/projects/spark/target/tmp/spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" did not contain "C:\projects\spark\target\tmp\spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" (DataSourceScanExecRedactionSuite.scala:46)
   ...
```

## How was this patch tested?

Tested via AppVeyor for each and checked it passed once each. These should be retested via AppVeyor in this PR.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17987 from HyukjinKwon/windows-20170515.
2017-05-25 17:10:30 +01:00
Liang-Chi Hsieh 6b68d61cf3 [SPARK-20848][SQL][FOLLOW-UP] Shutdown the pool after reading parquet files
## What changes were proposed in this pull request?

This is a follow-up to #18073. Taking a safer approach to shutdown the pool to prevent possible issue. Also using `ThreadUtils.newForkJoinPool` instead to set a better thread name.

## How was this patch tested?

Manually test.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18100 from viirya/SPARK-20848-followup.
2017-05-25 09:55:45 +08:00
liuxian 197f9018a4 [SPARK-20403][SQL] Modify the instructions of some functions
## What changes were proposed in this pull request?
1.    add  instructions of  'cast'  function When using 'show functions'  and 'desc function cast'
       command in spark-sql
2.    Modify the  instructions of functions,such as
     boolean,tinyint,smallint,int,bigint,float,double,decimal,date,timestamp,binary,string

## How was this patch tested?
Before modification:
spark-sql>desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr AS type) - Casts the value `expr` to the target data type `type`.

After modification:
spark-sql> desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`.

spark-sql> desc function cast
Function: cast
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: cast(expr AS type) - Casts the value `expr` to the target data type `type`.

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17698 from 10110346/wip_lx_0418.
2017-05-24 17:32:02 -07:00
Jacek Laskowski 5f8ff2fc9a [SPARK-16202][SQL][DOC] Follow-up to Correct The Description of CreatableRelationProvider's createRelation
## What changes were proposed in this pull request?

Follow-up to SPARK-16202:

1. Remove the duplication of the meaning of `SaveMode` (as one was in fact missing that had proven that the duplication may be incomplete in the future again)

2. Use standard scaladoc tags

/cc gatorsmile rxin yhuai (as they were involved previously)

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18026 from jaceklaskowski/CreatableRelationProvider-SPARK-16202.
2017-05-24 17:24:23 -07:00
Kris Mok c0b3e45e3b [SPARK-20872][SQL] ShuffleExchange.nodeName should handle null coordinator
## What changes were proposed in this pull request?

A one-liner change in `ShuffleExchange.nodeName` to cover the case when `coordinator` is `null`, so that the match expression is exhaustive.

Please refer to [SPARK-20872](https://issues.apache.org/jira/browse/SPARK-20872) for a description of the symptoms.
TL;DR is that inspecting a `ShuffleExchange` (directly or transitively) on the Executor side can hit a case where the `coordinator` field of a `ShuffleExchange` is null, and thus will trigger a `MatchError` in `ShuffleExchange.nodeName()`'s inexhaustive match expression.

Also changed two other match conditions in `ShuffleExchange` on the `coordinator` field to be consistent.

## How was this patch tested?

Manually tested this change with a case where the `coordinator` is null to make sure `ShuffleExchange.nodeName` doesn't throw a `MatchError` any more.

Author: Kris Mok <kris.mok@databricks.com>

Closes #18095 from rednaxelafx/shuffleexchange-nodename.
2017-05-24 17:19:35 -07:00
Reynold Xin a64746677b [SPARK-20867][SQL] Move hints from Statistics into HintInfo class
## What changes were proposed in this pull request?
This is a follow-up to SPARK-20857 to move the broadcast hint from Statistics into a new HintInfo class, so we can be more flexible in adding new hints in the future.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18087 from rxin/SPARK-20867.
2017-05-24 13:57:19 -07:00
Liang-Chi Hsieh f72ad303f0 [SPARK-20848][SQL] Shutdown the pool after reading parquet files
## What changes were proposed in this pull request?

From JIRA: On each call to spark.read.parquet, a new ForkJoinPool is created. One of the threads in the pool is kept in the WAITING state, and never stopped, which leads to unbounded growth in number of threads.

We should shutdown the pool after reading parquet files.

## How was this patch tested?

Added a test to ParquetFileFormatSuite.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18073 from viirya/SPARK-20848.
2017-05-25 00:35:40 +08:00
Kirby Linvill 4816c2ef5e [SPARK-15648][SQL] Add teradataDialect for JDBC connection to Teradata
The contribution is my original work and I license the work to the project under the project’s open source license.

Note: the Teradata JDBC connector limits the row size to 64K. The default string datatype equivalent I used is a 255 character/byte length varchar. This effectively limits the max number of string columns to 250 when using the Teradata jdbc connector.

## What changes were proposed in this pull request?

Added a teradataDialect for JDBC connection to Teradata. The Teradata dialect uses VARCHAR(255) in place of TEXT for string datatypes, and CHAR(1) in place of BIT(1) for boolean datatypes.

## How was this patch tested?

I added two unit tests to double check that the types get set correctly for a teradata jdbc url. I also ran a couple manual tests to make sure the jdbc connector worked with teradata and to make sure that an error was thrown if a row could potentially exceed 64K (this error comes from the teradata jdbc connector, not from the spark code). I did not check how string columns longer than 255 characters are handled.

Author: Kirby Linvill <kirby.linvill@teradata.com>
Author: klinvill <kjlinvill@gmail.com>

Closes #16746 from klinvill/master.
2017-05-23 12:00:58 -07:00
Reynold Xin 0d589ba00b [SPARK-20857][SQL] Generic resolved hint node
## What changes were proposed in this pull request?
This patch renames BroadcastHint to ResolvedHint (and Hint to UnresolvedHint) so the hint framework is more generic and would allow us to introduce other hint types in the future without introducing new hint nodes.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18072 from rxin/SPARK-20857.
2017-05-23 18:44:49 +02:00
gatorsmile f3ed62a381 [SPARK-20831][SQL] Fix INSERT OVERWRITE data source tables with IF NOT EXISTS
### What changes were proposed in this pull request?
Currently, we have a bug when we specify `IF NOT EXISTS` in `INSERT OVERWRITE` data source tables. For example, given a query:
```SQL
INSERT OVERWRITE TABLE $tableName partition (b=2, c=3) IF NOT EXISTS SELECT 9, 10
```
we will get the following error:
```
unresolved operator 'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true;;
'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true
+- Project [cast(9#423 as int) AS a#429, cast(10#424 as int) AS d#430]
   +- Project [9 AS 9#423, 10 AS 10#424]
      +- OneRowRelation$
```

This PR is to fix the issue to follow the behavior of Hive serde tables
> INSERT OVERWRITE will overwrite any existing data in the table or partition unless IF NOT EXISTS is provided for a partition

### How was this patch tested?
Modified an existing test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18050 from gatorsmile/insertPartitionIfNotExists.
2017-05-22 22:24:50 +08:00
Michal Senkyr a2b3b67624 [SPARK-19089][SQL] Add support for nested sequences
## What changes were proposed in this pull request?

Replaced specific sequence encoders with generic sequence encoder to enable nesting of sequences.

Does not add support for nested arrays as that cannot be solved in this way.

## How was this patch tested?

```bash
build/mvn -DskipTests clean package && dev/run-tests
```

Additionally in Spark shell:

```
scala> Seq(Seq(Seq(1))).toDS.collect()
res0: Array[Seq[Seq[Int]]] = Array(List(List(1)))
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #18011 from michalsenkyr/dataset-seq-nested.
2017-05-22 16:49:19 +08:00
Kazuaki Ishizaki 833c8d4152 [SPARK-20770][SQL] Improve ColumnStats
## What changes were proposed in this pull request?

This PR improves the implementation of `ColumnStats` by using the following appoaches.

1. Declare subclasses of `ColumnStats` as `final`
2. Remove unnecessary call of `row.isNullAt(ordinal)`
3. Remove the dependency on `GenericInternalRow`

For 1., this declaration encourages method inlining and other optimizations of JIT compiler
For 2., in `gatherStats()`, while previous code in subclasses of `ColumnStats` always calls `row.isNullAt()` twice, the PR just calls `row.isNullAt()` only once.
For 3., `collectedStatistics()` returns `Array[Any]` instead of `GenericInternalRow`. This removes the dependency of unnecessary package and reduces the number of allocations of `GenericInternalRow`.

In addition to that, in the future, `gatherValueStats()`, which is specialized for each data type, can be effectively called from the generated code without using generic data structure `InternalRow`.

## How was this patch tested?

Tested by existing test suite

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

Closes #18002 from kiszk/SPARK-20770.
2017-05-22 16:23:23 +08:00
caoxuewen 3c9eef35a8 [SPARK-20786][SQL] Improve ceil and floor handle the value which is not expected
## What changes were proposed in this pull request?

spark-sql>SELECT ceil(1234567890123456);
1234567890123456

spark-sql>SELECT ceil(12345678901234567);
12345678901234568

spark-sql>SELECT ceil(123456789012345678);
123456789012345680

when the length of the getText is greater than 16. long to double will be precision loss.

but mysql handle the value is ok.

mysql> SELECT ceil(1234567890123456);
+------------------------+
| ceil(1234567890123456) |
+------------------------+
|       1234567890123456 |
+------------------------+
1 row in set (0.00 sec)

mysql> SELECT ceil(12345678901234567);
+-------------------------+
| ceil(12345678901234567) |
+-------------------------+
|       12345678901234567 |
+-------------------------+
1 row in set (0.00 sec)

mysql> SELECT ceil(123456789012345678);
+--------------------------+
| ceil(123456789012345678) |
+--------------------------+
|       123456789012345678 |
+--------------------------+
1 row in set (0.00 sec)

## How was this patch tested?

Supplement the unit test.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #18016 from heary-cao/ceil_long.
2017-05-21 22:39:07 -07:00
Tathagata Das 9d6661c829 [SPARK-20792][SS] Support same timeout operations in mapGroupsWithState function in batch queries as in streaming queries
## What changes were proposed in this pull request?

Currently, in the batch queries, timeout is disabled (i.e. GroupStateTimeout.NoTimeout) which means any GroupState.setTimeout*** operation would throw UnsupportedOperationException. This makes it weird when converting a streaming query into a batch query by changing the input DF from streaming to a batch DF. If the timeout was enabled and used, then the batch query will start throwing UnsupportedOperationException.

This PR creates the dummy state in batch queries with the provided timeoutConf so that it behaves in the same way. The code has been refactored to make it obvious when the state is being created for a batch query or a streaming query.

## How was this patch tested?
Additional tests

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

Closes #18024 from tdas/SPARK-20792.
2017-05-21 13:07:25 -07:00
Yuming Wang bff021dfaf [SPARK-20751][SQL] Add built-in SQL Function - COT
## What changes were proposed in this pull request?

Add built-in SQL Function - COT.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17999 from wangyum/SPARK-20751.
2017-05-19 09:40:22 -07:00
tpoterba 3f2cd51ee0 [SPARK-20773][SQL] ParquetWriteSupport.writeFields is quadratic in number of fields
Fix quadratic List indexing in ParquetWriteSupport.

I noticed this function while profiling some code with today. It showed up as a significant factor in a table with twenty columns; with hundreds of columns, it could dominate any other function call.

## What changes were proposed in this pull request?

The writeFields method iterates from 0 until number of fields, indexing into rootFieldWriters for each element. rootFieldWriters is a List, so indexing is a linear operation. The complexity of the writeFields method is thus quadratic in the number of fields.

Solution: explicitly convert rootFieldWriters to Array (implicitly converted to WrappedArray) for constant-time indexing.

## How was this patch tested?

This is a one-line change for performance reasons.

Author: tpoterba <tpoterba@broadinstitute.org>
Author: Tim Poterba <tpoterba@gmail.com>

Closes #18005 from tpoterba/tpoterba-patch-1.
2017-05-19 14:17:12 +02:00
Ala Luszczak ce8edb8bf4 [SPARK-20798] GenerateUnsafeProjection should check if a value is null before calling the getter
## What changes were proposed in this pull request?

GenerateUnsafeProjection.writeStructToBuffer() did not honor the assumption that the caller must make sure that a value is not null before using the getter. This could lead to various errors. This change fixes that behavior.

Example of code generated before:
```scala
/* 059 */         final UTF8String fieldName = value.getUTF8String(0);
/* 060 */         if (value.isNullAt(0)) {
/* 061 */           rowWriter1.setNullAt(0);
/* 062 */         } else {
/* 063 */           rowWriter1.write(0, fieldName);
/* 064 */         }
```

Example of code generated now:
```scala
/* 060 */         boolean isNull1 = value.isNullAt(0);
/* 061 */         UTF8String value1 = isNull1 ? null : value.getUTF8String(0);
/* 062 */         if (isNull1) {
/* 063 */           rowWriter1.setNullAt(0);
/* 064 */         } else {
/* 065 */           rowWriter1.write(0, value1);
/* 066 */         }
```

## How was this patch tested?

Adds GenerateUnsafeProjectionSuite.

Author: Ala Luszczak <ala@databricks.com>

Closes #18030 from ala/fix-generate-unsafe-projection.
2017-05-19 13:18:48 +02:00
hyukjinkwon 8fb3d5c6da [SPARK-20364][SQL] Disable Parquet predicate pushdown for fields having dots in the names
## What changes were proposed in this pull request?

This is an alternative workaround by simply avoiding the predicate pushdown for columns having dots in the names. This is an approach different with https://github.com/apache/spark/pull/17680.

The downside of this PR is, literally it does not push down filters on the column having dots in Parquet files at all (both no record level and no rowgroup level) whereas the downside of the approach in that PR, it does not use the Parquet's API properly but in a hacky way to support this case.

I assume we prefer a safe way here by using the Parquet API properly but this does close that PR as we are basically just avoiding here.

This way looks a simple workaround and probably it is fine given the problem looks arguably rather corner cases (although it might end up with reading whole row groups under the hood but either looks not the best).

Currently, if there are dots in the column name, predicate pushdown seems being failed in Parquet.

**With dots**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("col.dots").write.parquet(path)
spark.read.parquet(path).where("`col.dots` IS NOT NULL").show()
```

```
+--------+
|col.dots|
+--------+
+--------+
```

**Without dots**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("coldots").write.parquet(path)
spark.read.parquet(path).where("`coldots` IS NOT NULL").show()
```

```
+-------+
|coldots|
+-------+
|      1|
+-------+
```

**After**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("col.dots").write.parquet(path)
spark.read.parquet(path).where("`col.dots` IS NOT NULL").show()
```

```
+--------+
|col.dots|
+--------+
|       1|
+--------+
```

## How was this patch tested?

Unit tests added in `ParquetFilterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18000 from HyukjinKwon/SPARK-20364-workaround.
2017-05-18 10:52:23 -07:00
Liang-Chi Hsieh 7463a88be6 [SPARK-20690][SQL] Subqueries in FROM should have alias names
## What changes were proposed in this pull request?

We add missing attributes into Filter in Analyzer. But we shouldn't do it through subqueries like this:

    select 1 from  (select 1 from onerow t1 LIMIT 1) where  t1.c1=1

This query works in current codebase. However, the outside where clause shouldn't be able to refer `t1.c1` attribute.

The root cause is we allow subqueries in FROM have no alias names previously, it is confusing and isn't supported by various databases such as MySQL, Postgres, Oracle. We shouldn't support it too.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #17935 from viirya/SPARK-20690.
2017-05-17 12:57:35 +08:00
Kazuaki Ishizaki 6f62e9d9b9 [SPARK-19372][SQL] Fix throwing a Java exception at df.fliter() due to 64KB bytecode size limit
## What changes were proposed in this pull request?

When an expression for `df.filter()` has many nodes (e.g. 400), the size of Java bytecode for the generated Java code is more than 64KB. It produces an Java exception. As a result, the execution fails.
This PR continues to execute by calling `Expression.eval()` disabling code generation if an exception has been caught.

## How was this patch tested?

Add a test suite into `DataFrameSuite`

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

Closes #17087 from kiszk/SPARK-19372.
2017-05-16 14:47:21 -07:00
Takuya UESHIN c8c878a416 [SPARK-20588][SQL] Cache TimeZone instances.
## What changes were proposed in this pull request?

Because the method `TimeZone.getTimeZone(String ID)` is synchronized on the TimeZone class, concurrent call of this method will become a bottleneck.
This especially happens when casting from string value containing timezone info to timestamp value, which uses `DateTimeUtils.stringToTimestamp()` and gets TimeZone instance on the site.

This pr makes a cache of the generated TimeZone instances to avoid the synchronization.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17933 from ueshin/issues/SPARK-20588.
2017-05-15 16:52:22 -07:00
Dongjoon Hyun bbd163d589 [SPARK-20735][SQL][TEST] Enable cross join in TPCDSQueryBenchmark
## What changes were proposed in this pull request?

Since [SPARK-17298](https://issues.apache.org/jira/browse/SPARK-17298), some queries (q28, q61, q77, q88, q90) in the test suites fail with a message "_Use the CROSS JOIN syntax to allow cartesian products between these relations_".

This benchmark is used as a reference model for Spark TPC-DS, so this PR aims to enable the correct configuration in `TPCDSQueryBenchmark.scala`.

## How was this patch tested?

Manual. (Run TPCDSQueryBenchmark)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17977 from dongjoon-hyun/SPARK-20735.
2017-05-15 11:24:30 -07:00
Tathagata Das 499ba2cb47 [SPARK-20717][SS] Minor tweaks to the MapGroupsWithState behavior
## What changes were proposed in this pull request?

Timeout and state data are two independent entities and should be settable independently. Therefore, in the same call of the user-defined function, one should be able to set the timeout before initializing the state and also after removing the state. Whether timeouts can be set or not, should not depend on the current state, and vice versa.

However, a limitation of the current implementation is that state cannot be null while timeout is set. This is checked lazily after the function call has completed.

## How was this patch tested?
- Updated existing unit tests that test the behavior of GroupState.setTimeout*** wrt to the current state
- Added new tests that verify the disallowed cases where state is undefined but timeout is set.

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

Closes #17957 from tdas/SPARK-20717.
2017-05-15 10:48:10 -07:00
Tejas Patil d2416925c4 [SPARK-17729][SQL] Enable creating hive bucketed tables
## What changes were proposed in this pull request?

Hive allows inserting data to bucketed table without guaranteeing bucketed and sorted-ness based on these two configs : `hive.enforce.bucketing` and `hive.enforce.sorting`.

What does this PR achieve ?
- Spark will disallow users from writing outputs to hive bucketed tables by default (given that output won't adhere with Hive's semantics).
- IF user still wants to write to hive bucketed table, the only resort is to use `hive.enforce.bucketing=false` and `hive.enforce.sorting=false` which means user does NOT care about bucketing guarantees.

Changes done in this PR:
- Extract table's bucketing information in `HiveClientImpl`
- While writing table info to metastore, `HiveClientImpl` now populates the bucketing information in the hive `Table` object
- `InsertIntoHiveTable` allows inserts to bucketed table only if both `hive.enforce.bucketing` and `hive.enforce.sorting` are `false`

Ability to create bucketed tables will enable adding test cases to Spark while I add more changes related to hive bucketing support. Design doc for hive hive bucketing support : https://docs.google.com/document/d/1a8IDh23RAkrkg9YYAeO51F4aGO8-xAlupKwdshve2fc/edit#

## How was this patch tested?
- Added test for creating bucketed and sorted table.
- Added test to ensure that INSERTs fail if strict bucket / sort is enforced
- Added test to ensure that INSERTs can go through if strict bucket / sort is NOT enforced
- Added test to validate that bucketing information shows up in output of DESC FORMATTED
- Added test to ensure that `SHOW CREATE TABLE` works for hive bucketed tables

Author: Tejas Patil <tejasp@fb.com>

Closes #17644 from tejasapatil/SPARK-17729_create_bucketed_table.
2017-05-16 01:47:23 +08:00
Tathagata Das 271175e2bd [SPARK-20716][SS] StateStore.abort() should not throw exceptions
## What changes were proposed in this pull request?

StateStore.abort() should do a best effort attempt to clean up temporary resources. It should not throw errors, especially because its called in a TaskCompletionListener, because this error could hide previous real errors in the task.

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

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

Closes #17958 from tdas/SPARK-20716.
2017-05-15 10:46:38 -07:00
Takeshi Yamamuro b0888d1ac3 [SPARK-20730][SQL] Add an optimizer rule to combine nested Concat
## What changes were proposed in this pull request?
This pr added a new Optimizer rule to combine nested Concat. The master supports a pipeline operator '||' to concatenate strings in #17711 (This pr is follow-up). Since the parser currently generates nested Concat expressions, the optimizer needs to combine the nested expressions.

## How was this patch tested?
Added tests in `CombineConcatSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17970 from maropu/SPARK-20730.
2017-05-15 16:24:55 +08:00
Wenchen Fan 1283c3d11a [SPARK-20725][SQL] partial aggregate should behave correctly for sameResult
## What changes were proposed in this pull request?

For aggregate function with `PartialMerge` or `Final` mode, the input is aggregate buffers instead of the actual children expressions. So the actual children expressions won't affect the result, we should normalize the expr id for them.

## How was this patch tested?

a new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17964 from cloud-fan/tmp.
2017-05-13 12:09:06 -07:00
hyukjinkwon 3f98375d8a [SPARK-18772][SQL] Avoid unnecessary conversion try for special floats in JSON
## What changes were proposed in this pull request?

This PR is based on  https://github.com/apache/spark/pull/16199 and extracts the valid change from https://github.com/apache/spark/pull/9759 to resolve SPARK-18772

This avoids additional conversion try with `toFloat` and `toDouble`.

For avoiding additional conversions, please refer the codes below:

**Before**

```scala
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._

scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:30:41 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)
java.lang.NumberFormatException: For input string: "nan"
...
```

**After**

```scala
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._

scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:44:30 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.RuntimeException: Cannot parse nan as DoubleType.
...
```

## How was this patch tested?

Unit tests added in `JsonSuite`.

Closes #16199

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Nathan Howell <nhowell@godaddy.com>

Closes #17956 from HyukjinKwon/SPARK-18772.
2017-05-13 20:56:04 +08:00
Xiao Li b84ff7eb62 [SPARK-20719][SQL] Support LIMIT ALL
### What changes were proposed in this pull request?
`LIMIT ALL` is the same as omitting the `LIMIT` clause. It is supported by both PrestgreSQL and Presto. This PR is to support it by adding it in the parser.

### How was this patch tested?
Added a test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17960 from gatorsmile/LimitAll.
2017-05-12 15:26:10 -07:00
Tathagata Das 0d3a63193c [SPARK-20714][SS] Fix match error when watermark is set with timeout = no timeout / processing timeout
## What changes were proposed in this pull request?

When watermark is set, and timeout conf is NoTimeout or ProcessingTimeTimeout (both do not need the watermark), the query fails at runtime with the following exception.
```
MatchException: Some(org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate1a9b798e) (of class scala.Some)
    org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:120)
    org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:116)
    org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:70)
    org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:65)
    org.apache.spark.sql.execution.streaming.state.StateStoreRDD.compute(StateStoreRDD.scala:64)
```

The match did not correctly handle cases where watermark was defined by the timeout was different from EventTimeTimeout.

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

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

Closes #17954 from tdas/SPARK-20714.
2017-05-12 10:49:50 -07:00
Takeshi Yamamuro b526f70c16 [SPARK-19951][SQL] Add string concatenate operator || to Spark SQL
## What changes were proposed in this pull request?
This pr added code to support `||` for string concatenation. This string operation is supported in PostgreSQL and MySQL.

## How was this patch tested?
Added tests in `SparkSqlParserSuite`

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17711 from maropu/SPARK-19951.
2017-05-12 09:55:51 -07:00
Takeshi Yamamuro 92ea7fd7b6 [SPARK-20710][SQL] Support aliases in CUBE/ROLLUP/GROUPING SETS
## What changes were proposed in this pull request?
This pr added  `Analyzer` code for supporting aliases in CUBE/ROLLUP/GROUPING SETS (This is follow-up of #17191).

## How was this patch tested?
Added tests in `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17948 from maropu/SPARK-20710.
2017-05-12 20:48:30 +08:00
wangzhenhua 54b4f2ad43 [SPARK-20718][SQL][FOLLOWUP] Fix canonicalization for HiveTableScanExec
## What changes were proposed in this pull request?

Fix canonicalization for different filter orders in `HiveTableScanExec`.

## How was this patch tested?

Added a new test case.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17962 from wzhfy/canonicalizeHiveTableScanExec.
2017-05-12 20:43:22 +08:00
Sean Owen fc8a2b6ee6 [SPARK-20554][BUILD] Remove usage of scala.language.reflectiveCalls
## What changes were proposed in this pull request?

Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17949 from srowen/SPARK-20554.
2017-05-12 09:55:04 +01:00
hyukjinkwon 720708ccdd [SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`).

- Support single argument for `to_timestamp` similarly with APIs in other languages.

  For example, the one below works

  ```
  import org.apache.spark.sql.functions._
  Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show()
  ```

  prints

  ```
  +----------------------------------------+
  |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')|
  +----------------------------------------+
  |                     2016-12-31 00:12:00|
  +----------------------------------------+
  ```

  whereas this does not work in SQL.

  **Before**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7
  ```

  **After**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  2016-12-31 00:12:00
  ```

- Related document improvement for SQL function descriptions and other API descriptions accordingly.

  **Before**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
  Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00.0
  ```

  **After**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage:
      to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to
        a date. Returns null with invalid input. By default, it follows casting rules to a date if
        the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_date('2009-07-30 04:17:52');
         2009-07-30
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
   Usage:
      to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to
        a timestamp. Returns null with invalid input. By default, it follows casting rules to
        a timestamp if the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31 00:12:00');
         2016-12-31 00:12:00
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00
  ```

## How was this patch tested?

Added tests in `datetime.sql`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 16:42:58 +08:00
wangzhenhua c8da535600 [SPARK-20718][SQL] FileSourceScanExec with different filter orders should be the same after canonicalization
## What changes were proposed in this pull request?

Since `constraints` in `QueryPlan` is a set, the order of filters can differ. Usually this is ok because of canonicalization. However, in `FileSourceScanExec`, its data filters and partition filters are sequences, and their orders are not canonicalized. So `def sameResult` returns different results for different orders of data/partition filters. This leads to, e.g. different decision for `ReuseExchange`, and thus results in unstable performance.

## How was this patch tested?

Added a new test for `FileSourceScanExec.sameResult`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17959 from wzhfy/canonicalizeFileSourceScanExec.
2017-05-12 13:42:48 +08:00
liuxian 2b36eb696f [SPARK-20665][SQL] Bround" and "Round" function return NULL
## What changes were proposed in this pull request?
   spark-sql>select bround(12.3, 2);
   spark-sql>NULL
For this case,  the expected result is 12.3, but it is null.
So ,when the second parameter is bigger than "decimal.scala", the result is not we expected.
"round" function  has the same problem. This PR can solve the problem for both of them.

## How was this patch tested?
unit test cases in MathExpressionsSuite and MathFunctionsSuite

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17906 from 10110346/wip_lx_0509.
2017-05-12 11:38:50 +08:00
Liang-Chi Hsieh 609ba5f2b9 [SPARK-20399][SQL] Add a config to fallback string literal parsing consistent with old sql parser behavior
## What changes were proposed in this pull request?

The new SQL parser is introduced into Spark 2.0. All string literals are unescaped in parser. Seems it bring an issue regarding the regex pattern string.

The following codes can reproduce it:

    val data = Seq("\u0020\u0021\u0023", "abc")
    val df = data.toDF()

    // 1st usage: works in 1.6
    // Let parser parse pattern string
    val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'")
    // 2nd usage: works in 1.6, 2.x
    // Call Column.rlike so the pattern string is a literal which doesn't go through parser
    val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$"))

    // In 2.x, we need add backslashes to make regex pattern parsed correctly
    val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'")

Follow the discussion in #17736, this patch adds a config to fallback to 1.6 string literal parsing and mitigate migration issue.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #17887 from viirya/add-config-fallback-string-parsing.
2017-05-12 11:15:10 +08:00
Takeshi Yamamuro 04901dd03a [SPARK-20431][SQL] Specify a schema by using a DDL-formatted string
## What changes were proposed in this pull request?
This pr supported a DDL-formatted string in `DataFrameReader.schema`.
This fix could make users easily define a schema without importing  `o.a.spark.sql.types._`.

## How was this patch tested?
Added tests in `DataFrameReaderWriterSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17719 from maropu/SPARK-20431.
2017-05-11 11:06:29 -07:00
Takeshi Yamamuro 3aa4e464a8 [SPARK-20416][SQL] Print UDF names in EXPLAIN
## What changes were proposed in this pull request?
This pr added `withName` in `UserDefinedFunction` for printing UDF names in EXPLAIN

## How was this patch tested?
Added tests in `UDFSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17712 from maropu/SPARK-20416.
2017-05-11 09:49:05 -07:00
Takeshi Yamamuro 8c67aa7f00 [SPARK-20311][SQL] Support aliases for table value functions
## What changes were proposed in this pull request?
This pr added parsing rules to support aliases in table value functions.
The previous pr (#17666) has been reverted because of the regression. This new pr fixed the regression and add tests in `SQLQueryTestSuite`.

## How was this patch tested?
Added tests in `PlanParserSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17928 from maropu/SPARK-20311-3.
2017-05-11 18:09:31 +08:00
Wenchen Fan b4c99f4369 [SPARK-20569][SQL] RuntimeReplaceable functions should not take extra parameters
## What changes were proposed in this pull request?

`RuntimeReplaceable` always has a constructor with the expression to replace with, and this constructor should not be the function builder.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17876 from cloud-fan/minor.
2017-05-11 00:41:15 -07:00
Robert Kruszewski 65accb813a [SPARK-17029] make toJSON not go through rdd form but operate on dataset always
## What changes were proposed in this pull request?

Don't convert toRdd when doing toJSON
## How was this patch tested?

Existing unit tests

Author: Robert Kruszewski <robertk@palantir.com>

Closes #14615 from robert3005/robertk/correct-tojson.
2017-05-11 15:26:48 +08:00
Ala Luszczak 5c2c4dcce5 [SPARK-19447] Remove remaining references to generated rows metric
## What changes were proposed in this pull request?

b486ffc86d left behind references to "number of generated rows" metrics, that should have been removed.

## How was this patch tested?

Existing unit tests.

Author: Ala Luszczak <ala@databricks.com>

Closes #17939 from ala/SPARK-19447-fix.
2017-05-10 08:41:04 -07:00
Wenchen Fan 789bdbe3d0 [SPARK-20688][SQL] correctly check analysis for scalar sub-queries
## What changes were proposed in this pull request?

In `CheckAnalysis`, we should call `checkAnalysis` for `ScalarSubquery` at the beginning, as later we will call `plan.output` which is invalid if `plan` is not resolved.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17930 from cloud-fan/tmp.
2017-05-10 19:30:00 +08:00
NICHOLAS T. MARION b512233a45 [SPARK-20393][WEBU UI] Strengthen Spark to prevent XSS vulnerabilities
## What changes were proposed in this pull request?

Add stripXSS and stripXSSMap to Spark Core's UIUtils. Calling these functions at any point that getParameter is called against a HttpServletRequest.

## How was this patch tested?

Unit tests, IBM Security AppScan Standard no longer showing vulnerabilities, manual verification of WebUI pages.

Author: NICHOLAS T. MARION <nmarion@us.ibm.com>

Closes #17686 from n-marion/xss-fix.
2017-05-10 10:59:57 +01:00
Takuya UESHIN 0ef16bd4b0 [SPARK-20668][SQL] Modify ScalaUDF to handle nullability.
## What changes were proposed in this pull request?

When registering Scala UDF, we can know if the udf will return nullable value or not. `ScalaUDF` and related classes should handle the nullability.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17911 from ueshin/issues/SPARK-20668.
2017-05-09 23:48:25 -07:00
Josh Rosen a90c5cd822 [SPARK-20686][SQL] PropagateEmptyRelation incorrectly handles aggregate without grouping
## What changes were proposed in this pull request?

The query

```
SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1
```

should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows.

This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead:

An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows.

If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation.

The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be).

This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation.

## How was this patch tested?

- Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file.
- Updated unit tests in `PropagateEmptyRelationSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17929 from JoshRosen/fix-PropagateEmptyRelation.
2017-05-10 14:36:36 +08:00
hyukjinkwon 3d2131ab4d [SPARK-20590][SQL] Use Spark internal datasource if multiples are found for the same shorten name
## What changes were proposed in this pull request?

One of the common usability problems around reading data in spark (particularly CSV) is that there can often be a conflict between different readers in the classpath.

As an example, if someone launches a 2.x spark shell with the spark-csv package in the classpath, Spark currently fails in an extremely unfriendly way (see databricks/spark-csv#367):

```bash
./bin/spark-shell --packages com.databricks:spark-csv_2.11:1.5.0
scala> val df = spark.read.csv("/foo/bar.csv")
java.lang.RuntimeException: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat, com.databricks.spark.csv.DefaultSource15), please specify the fully qualified class name.
  at scala.sys.package$.error(package.scala:27)
  at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:574)
  at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:85)
  at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:85)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:295)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
  at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:533)
  at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:412)
  ... 48 elided
```

This PR proposes a simple way of fixing this error by picking up the internal datasource if there is single (the datasource that has "org.apache.spark" prefix).

```scala
scala> spark.range(1).write.format("csv").mode("overwrite").save("/tmp/abc")
17/05/10 09:47:44 WARN DataSource: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
```

```scala
scala> spark.range(1).write.format("Csv").mode("overwrite").save("/tmp/abc")
17/05/10 09:47:52 WARN DataSource: Multiple sources found for Csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
```

## How was this patch tested?

Manually tested as below:

```bash
./bin/spark-shell --packages com.databricks:spark-csv_2.11:1.5.0
```

```scala
spark.sparkContext.setLogLevel("WARN")
```

**positive cases**:

```scala
scala> spark.range(1).write.format("csv").mode("overwrite").save("/tmp/abc")
17/05/10 09:47:44 WARN DataSource: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
```

```scala
scala> spark.range(1).write.format("Csv").mode("overwrite").save("/tmp/abc")
17/05/10 09:47:52 WARN DataSource: Multiple sources found for Csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
```

(newlines were inserted for readability).

```scala
scala> spark.range(1).write.format("com.databricks.spark.csv").mode("overwrite").save("/tmp/abc")
```

```scala
scala> spark.range(1).write.format("org.apache.spark.sql.execution.datasources.csv.CSVFileFormat").mode("overwrite").save("/tmp/abc")
```

**negative cases**:

```scala
scala> spark.range(1).write.format("com.databricks.spark.csv.CsvRelation").save("/tmp/abc")
java.lang.InstantiationException: com.databricks.spark.csv.CsvRelation
...
```

```scala
scala> spark.range(1).write.format("com.databricks.spark.csv.CsvRelatio").save("/tmp/abc")
java.lang.ClassNotFoundException: Failed to find data source: com.databricks.spark.csv.CsvRelatio. Please find packages at http://spark.apache.org/third-party-projects.html
...
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17916 from HyukjinKwon/datasource-detect.
2017-05-10 13:44:47 +08:00
Yuming Wang 771abeb46f [SPARK-17685][SQL] Make SortMergeJoinExec's currentVars is null when calling createJoinKey
## What changes were proposed in this pull request?

The following SQL query cause `IndexOutOfBoundsException` issue when `LIMIT > 1310720`:
```sql
CREATE TABLE tab1(int int, int2 int, str string);
CREATE TABLE tab2(int int, int2 int, str string);
INSERT INTO tab1 values(1,1,'str');
INSERT INTO tab1 values(2,2,'str');
INSERT INTO tab2 values(1,1,'str');
INSERT INTO tab2 values(2,3,'str');

SELECT
  count(*)
FROM
  (
    SELECT t1.int, t2.int2
    FROM (SELECT * FROM tab1 LIMIT 1310721) t1
    INNER JOIN (SELECT * FROM tab2 LIMIT 1310721) t2
    ON (t1.int = t2.int AND t1.int2 = t2.int2)
  ) t;
```

This pull request fix this issue.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17920 from wangyum/SPARK-17685.
2017-05-09 19:45:00 -07:00
uncleGen c0189abc7c [SPARK-20373][SQL][SS] Batch queries with 'Dataset/DataFrame.withWatermark()` does not execute
## What changes were proposed in this pull request?

Any Dataset/DataFrame batch query with the operation `withWatermark` does not execute because the batch planner does not have any rule to explicitly handle the EventTimeWatermark logical plan.
The right solution is to simply remove the plan node, as the watermark should not affect any batch query in any way.

Changes:
- In this PR, we add a new rule `EliminateEventTimeWatermark` to check if we need to ignore the event time watermark. We will ignore watermark in any batch query.

Depends upon:
- [SPARK-20672](https://issues.apache.org/jira/browse/SPARK-20672). We can not add this rule into analyzer directly, because streaming query will be copied to `triggerLogicalPlan ` in every trigger, and the rule will be applied to `triggerLogicalPlan` mistakenly.

Others:
- A typo fix in example.

## How was this patch tested?

add new unit test.

Author: uncleGen <hustyugm@gmail.com>

Closes #17896 from uncleGen/SPARK-20373.
2017-05-09 15:08:09 -07:00
Reynold Xin ac1ab6b9db Revert "[SPARK-12297][SQL] Hive compatibility for Parquet Timestamps"
This reverts commit 22691556e5.

See JIRA ticket for more information.
2017-05-09 11:35:59 -07:00
Sean Owen 25ee816e09 [SPARK-19876][BUILD] Move Trigger.java to java source hierarchy
## What changes were proposed in this pull request?

Simply moves `Trigger.java` to `src/main/java` from `src/main/scala`
See https://github.com/apache/spark/pull/17219

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17921 from srowen/SPARK-19876.2.
2017-05-09 10:22:23 -07:00
Reynold Xin d099f414d2 [SPARK-20674][SQL] Support registering UserDefinedFunction as named UDF
## What changes were proposed in this pull request?
For some reason we don't have an API to register UserDefinedFunction as named UDF. It is a no brainer to add one, in addition to the existing register functions we have.

## How was this patch tested?
Added a test case in UDFSuite for the new API.

Author: Reynold Xin <rxin@databricks.com>

Closes #17915 from rxin/SPARK-20674.
2017-05-09 09:24:28 -07:00
Xiao Li 0d00c768a8 [SPARK-20667][SQL][TESTS] Cleanup the cataloged metadata after completing the package of sql/core and sql/hive
## What changes were proposed in this pull request?

So far, we do not drop all the cataloged objects after each package. Sometimes, we might hit strange test case errors because the previous test suite did not drop the cataloged/temporary objects (tables/functions/database). At least, we can first clean up the environment when completing the package of `sql/core` and `sql/hive`.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17908 from gatorsmile/reset.
2017-05-09 20:10:50 +08:00
sujith71955 42cc6d13ed [SPARK-20380][SQL] Unable to set/unset table comment property using ALTER TABLE SET/UNSET TBLPROPERTIES ddl
### What changes were proposed in this pull request?
Table comment was not getting  set/unset using **ALTER TABLE  SET/UNSET TBLPROPERTIES** query
eg: ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment)
 when user alter the table properties  and adds/updates table comment,table comment which is a field  of **CatalogTable**  instance is not getting updated and  old table comment if exists was shown to user, inorder  to handle this issue, update the comment field value in **CatalogTable** with the newly added/modified comment along with other table level properties when user executes **ALTER TABLE  SET TBLPROPERTIES** query.

This pr has also taken care of unsetting the table comment when user executes query  **ALTER TABLE  UNSET TBLPROPERTIES** inorder to unset or remove table comment.
eg: ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment')

### How was this patch tested?
Added test cases  as part of **SQLQueryTestSuite** for verifying  table comment using desc formatted table query after adding/modifying table comment as part of **AlterTableSetPropertiesCommand** and unsetting the table comment using **AlterTableUnsetPropertiesCommand**.

Author: sujith71955 <sujithchacko.2010@gmail.com>

Closes #17649 from sujith71955/alter_table_comment.
2017-05-07 23:15:00 -07:00
Imran Rashid 22691556e5 [SPARK-12297][SQL] Hive compatibility for Parquet Timestamps
## What changes were proposed in this pull request?

This change allows timestamps in parquet-based hive table to behave as a "floating time", without a timezone, as timestamps are for other file formats.  If the storage timezone is the same as the session timezone, this conversion is a no-op.  When data is read from a hive table, the table property is *always* respected.  This allows spark to not change behavior when reading old data, but read newly written data correctly (whatever the source of the data is).

Spark inherited the original behavior from Hive, but Hive is also updating behavior to use the same  scheme in HIVE-12767 / HIVE-16231.

The default for Spark remains unchanged; created tables do not include the new table property.

This will only apply to hive tables; nothing is added to parquet metadata to indicate the timezone, so data that is read or written directly from parquet files will never have any conversions applied.

## How was this patch tested?

Added a unit test which creates tables, reads and writes data, under a variety of permutations (different storage timezones, different session timezones, vectorized reading on and off).

Author: Imran Rashid <irashid@cloudera.com>

Closes #16781 from squito/SPARK-12297.
2017-05-08 12:16:00 +09:00
Jacek Laskowski 500436b436 [MINOR][SQL][DOCS] Improve unix_timestamp's scaladoc (and typo hunting)
## What changes were proposed in this pull request?

* Docs are consistent (across different `unix_timestamp` variants and their internal expressions)
* typo hunting

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17801 from jaceklaskowski/unix_timestamp.
2017-05-07 13:56:13 -07:00
Xiao Li cafca54c0e [SPARK-20557][SQL] Support JDBC data type Time with Time Zone
### What changes were proposed in this pull request?

This PR is to support JDBC data type TIME WITH TIME ZONE. It can be converted to TIMESTAMP

In addition, before this PR, for unsupported data types, we simply output the type number instead of the type name.

```
java.sql.SQLException: Unsupported type 2014
```
After this PR, the message is like
```
java.sql.SQLException: Unsupported type TIMESTAMP_WITH_TIMEZONE
```

- Also upgrade the H2 version to `1.4.195` which has the type fix for "TIMESTAMP WITH TIMEZONE". However, it is not fully supported. Thus, we capture the exception, but we still need it to partially test the support of "TIMESTAMP WITH TIMEZONE", because Docker tests are not regularly run.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17835 from gatorsmile/h2.
2017-05-06 22:21:19 -07:00
Jannik Arndt b31648c081 [SPARK-20557][SQL] Support for db column type TIMESTAMP WITH TIME ZONE
## What changes were proposed in this pull request?

SparkSQL can now read from a database table with column type [TIMESTAMP WITH TIME ZONE](https://docs.oracle.com/javase/8/docs/api/java/sql/Types.html#TIMESTAMP_WITH_TIMEZONE).

## How was this patch tested?

Tested against Oracle database.

JoshRosen, you seem to know the class, would you look at this? Thanks!

Author: Jannik Arndt <jannik@jannikarndt.de>

Closes #17832 from JannikArndt/spark-20557-timestamp-with-timezone.
2017-05-05 11:42:55 -07:00
Yucai 41439fd52d [SPARK-20381][SQL] Add SQL metrics of numOutputRows for ObjectHashAggregateExec
## What changes were proposed in this pull request?

ObjectHashAggregateExec is missing numOutputRows, add this metrics for it.

## How was this patch tested?

Added unit tests for the new metrics.

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

Closes #17678 from yucai/objectAgg_numOutputRows.
2017-05-05 09:51:57 -07:00
madhu 9064f1b044 [SPARK-20495][SQL][CORE] Add StorageLevel to cacheTable API
## What changes were proposed in this pull request?
Currently cacheTable API only supports MEMORY_AND_DISK. This PR adds additional API to take different storage levels.
## How was this patch tested?
unit tests

Author: madhu <phatak.dev@gmail.com>

Closes #17802 from phatak-dev/cacheTableAPI.
2017-05-05 22:44:03 +08:00
Yuming Wang 37cdf077cd [SPARK-19660][SQL] Replace the deprecated property name fs.default.name to fs.defaultFS that newly introduced
## What changes were proposed in this pull request?

Replace the deprecated property name `fs.default.name` to `fs.defaultFS` that newly introduced.

## How was this patch tested?

Existing tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17856 from wangyum/SPARK-19660.
2017-05-05 11:31:59 +01:00
Dongjoon Hyun bfc8c79c8d [SPARK-20566][SQL] ColumnVector should support appendFloats for array
## What changes were proposed in this pull request?

This PR aims to add a missing `appendFloats` API for array into **ColumnVector** class. For double type, there is `appendDoubles` for array [here](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java#L818-L824).

## How was this patch tested?

Pass the Jenkins with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17836 from dongjoon-hyun/SPARK-20566.
2017-05-04 21:04:15 +08:00
hyukjinkwon 13eb37c860 [MINOR][SQL] Fix the test title from =!= to <=>, remove a duplicated test and add a test for =!=
## What changes were proposed in this pull request?

This PR proposes three things as below:

- This test looks not testing `<=>` and identical with the test above, `===`. So, it removes the test.

  ```diff
  -   test("<=>") {
  -     checkAnswer(
  -      testData2.filter($"a" === 1),
  -      testData2.collect().toSeq.filter(r => r.getInt(0) == 1))
  -
  -    checkAnswer(
  -      testData2.filter($"a" === $"b"),
  -      testData2.collect().toSeq.filter(r => r.getInt(0) == r.getInt(1)))
  -   }
  ```

- Replace the test title from `=!=` to `<=>`. It looks the test actually testing `<=>`.

  ```diff
  +  private lazy val nullData = Seq(
  +    (Some(1), Some(1)), (Some(1), Some(2)), (Some(1), None), (None, None)).toDF("a", "b")
  +
    ...
  -  test("=!=") {
  +  test("<=>") {
  -    val nullData = spark.createDataFrame(sparkContext.parallelize(
  -      Row(1, 1) ::
  -      Row(1, 2) ::
  -      Row(1, null) ::
  -      Row(null, null) :: Nil),
  -      StructType(Seq(StructField("a", IntegerType), StructField("b", IntegerType))))
  -
         checkAnswer(
           nullData.filter($"b" <=> 1),
    ...
  ```

- Add the tests for `=!=` which looks not existing.

  ```diff
  +  test("=!=") {
  +    checkAnswer(
  +      nullData.filter($"b" =!= 1),
  +      Row(1, 2) :: Nil)
  +
  +    checkAnswer(nullData.filter($"b" =!= null), Nil)
  +
  +    checkAnswer(
  +      nullData.filter($"a" =!= $"b"),
  +      Row(1, 2) :: Nil)
  +  }
  ```

## How was this patch tested?

Manually running the tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17842 from HyukjinKwon/minor-test-fix.
2017-05-03 13:08:25 -07:00
Liwei Lin 6b9e49d12f [SPARK-19965][SS] DataFrame batch reader may fail to infer partitions when reading FileStreamSink's output
## The Problem

Right now DataFrame batch reader may fail to infer partitions when reading FileStreamSink's output:

```
[info] - partitioned writing and batch reading with 'basePath' *** FAILED *** (3 seconds, 928 milliseconds)
[info]   java.lang.AssertionError: assertion failed: Conflicting directory structures detected. Suspicious paths:
[info] 	***/stream.output-65e3fa45-595a-4d29-b3df-4c001e321637
[info] 	***/stream.output-65e3fa45-595a-4d29-b3df-4c001e321637/_spark_metadata
[info]
[info] If provided paths are partition directories, please set "basePath" in the options of the data source to specify the root directory of the table. If there are multiple root directories, please load them separately and then union them.
[info]   at scala.Predef$.assert(Predef.scala:170)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningUtils$.parsePartitions(PartitioningUtils.scala:133)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningUtils$.parsePartitions(PartitioningUtils.scala:98)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning(PartitioningAwareFileIndex.scala:156)
[info]   at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.partitionSpec(InMemoryFileIndex.scala:54)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.partitionSchema(PartitioningAwareFileIndex.scala:55)
[info]   at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:133)
[info]   at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:361)
[info]   at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:160)
[info]   at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:536)
[info]   at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:520)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply$mcV$sp(FileStreamSinkSuite.scala:292)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply(FileStreamSinkSuite.scala:268)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply(FileStreamSinkSuite.scala:268)
```

## What changes were proposed in this pull request?

This patch alters `InMemoryFileIndex` to filter out these `basePath`s whose ancestor is the streaming metadata dir (`_spark_metadata`). E.g., the following and other similar dir or files will be filtered out:
- (introduced by globbing `basePath/*`)
   - `basePath/_spark_metadata`
- (introduced by globbing `basePath/*/*`)
   - `basePath/_spark_metadata/0`
   - `basePath/_spark_metadata/1`
   - ...

## How was this patch tested?

Added unit tests

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17346 from lw-lin/filter-metadata.
2017-05-03 11:10:24 -07:00
Reynold Xin 527fc5d0c9 [SPARK-20576][SQL] Support generic hint function in Dataset/DataFrame
## What changes were proposed in this pull request?
We allow users to specify hints (currently only "broadcast" is supported) in SQL and DataFrame. However, while SQL has a standard hint format (/*+ ... */), DataFrame doesn't have one and sometimes users are confused that they can't find how to apply a broadcast hint. This ticket adds a generic hint function on DataFrame that allows using the same hint on DataFrames as well as SQL.

As an example, after this patch, the following will apply a broadcast hint on a DataFrame using the new hint function:

```
df1.join(df2.hint("broadcast"))
```

## How was this patch tested?
Added a test case in DataFrameJoinSuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #17839 from rxin/SPARK-20576.
2017-05-03 09:22:25 -07:00
Liwei Lin 27f543b15f [SPARK-20441][SPARK-20432][SS] Within the same streaming query, one StreamingRelation should only be transformed to one StreamingExecutionRelation
## What changes were proposed in this pull request?

Within the same streaming query, when one `StreamingRelation` is referred multiple times – e.g. `df.union(df)` – we should transform it only to one `StreamingExecutionRelation`, instead of two or more different `StreamingExecutionRelation`s (each of which would have a separate set of source, source logs, ...).

## How was this patch tested?

Added two test cases, each of which would fail without this patch.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17735 from lw-lin/SPARK-20441.
2017-05-03 08:55:02 -07:00
Sean Owen 16fab6b0ef [SPARK-20523][BUILD] Clean up build warnings for 2.2.0 release
## What changes were proposed in this pull request?

Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17803 from srowen/SPARK-20523.
2017-05-03 10:18:35 +01:00
Michael Armbrust 6235132a8c [SPARK-20567] Lazily bind in GenerateExec
It is not valid to eagerly bind with the child's output as this causes failures when we attempt to canonicalize the plan (replacing the attribute references with dummies).

Author: Michael Armbrust <michael@databricks.com>

Closes #17838 from marmbrus/fixBindExplode.
2017-05-02 22:44:27 -07:00
Xiao Li b1e639ab09 [SPARK-19235][SQL][TEST][FOLLOW-UP] Enable Test Cases in DDLSuite with Hive Metastore
### What changes were proposed in this pull request?
This is a follow-up of enabling test cases in DDLSuite with Hive Metastore. It consists of the following remaining tasks:
- Run all the `alter table` and `drop table` DDL tests against data source tables when using Hive metastore.
- Do not run any `alter table` and `drop table` DDL test against Hive serde tables when using InMemoryCatalog.
- Reenable `alter table: set serde partition` and `alter table: set serde` tests for Hive serde tables.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17524 from gatorsmile/cleanupDDLSuite.
2017-05-02 16:49:24 +08:00
Kazuaki Ishizaki afb21bf22a [SPARK-20537][CORE] Fixing OffHeapColumnVector reallocation
## What changes were proposed in this pull request?

As #17773 revealed `OnHeapColumnVector` may copy a part of the original storage.

`OffHeapColumnVector` reallocation also copies to the new storage data up to 'elementsAppended'. This variable is only updated when using the `ColumnVector.appendX` API, while `ColumnVector.putX` is more commonly used.
This PR copies the new storage data up to the previously-allocated size in`OffHeapColumnVector`.

## How was this patch tested?

Existing test suites

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

Closes #17811 from kiszk/SPARK-20537.
2017-05-02 13:56:41 +08:00
Sean Owen af726cd611 [SPARK-20459][SQL] JdbcUtils throws IllegalStateException: Cause already initialized after getting SQLException
## What changes were proposed in this pull request?

Avoid failing to initCause on JDBC exception with cause initialized to null

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17800 from srowen/SPARK-20459.
2017-05-01 17:01:05 -07:00
Kunal Khamar 6fc6cf88d8 [SPARK-20464][SS] Add a job group and description for streaming queries and fix cancellation of running jobs using the job group
## What changes were proposed in this pull request?

Job group: adding a job group is required to properly cancel running jobs related to a query.
Description: the new description makes it easier to group the batches of a query by sorting by name in the Spark Jobs UI.

## How was this patch tested?

- Unit tests
- UI screenshot

  - Order by job id:
![screen shot 2017-04-27 at 5 10 09 pm](https://cloud.githubusercontent.com/assets/7865120/25509468/15452274-2b6e-11e7-87ba-d929816688cf.png)

  - Order by description:
![screen shot 2017-04-27 at 5 10 22 pm](https://cloud.githubusercontent.com/assets/7865120/25509474/1c298512-2b6e-11e7-99b8-fef1ef7665c1.png)

  - Order by job id (no query name):
![screen shot 2017-04-27 at 5 21 33 pm](https://cloud.githubusercontent.com/assets/7865120/25509482/28c96dc8-2b6e-11e7-8df0-9d3cdbb05e36.png)

  - Order by description (no query name):
![screen shot 2017-04-27 at 5 21 44 pm](https://cloud.githubusercontent.com/assets/7865120/25509489/37674742-2b6e-11e7-9357-b5c38ec16ac4.png)

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17765 from kunalkhamar/sc-6696.
2017-05-01 11:37:30 -07:00
Herman van Hovell 6b44c4d63a [SPARK-20534][SQL] Make outer generate exec return empty rows
## What changes were proposed in this pull request?
Generate exec does not produce `null` values if the generator for the input row is empty and the generate operates in outer mode without join. This is caused by the fact that the `join=false` code path is different from the `join=true` code path, and that the `join=false` code path did deal with outer properly. This PR addresses this issue.

## How was this patch tested?
Updated `outer*` tests in `GeneratorFunctionSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17810 from hvanhovell/SPARK-20534.
2017-05-01 09:46:35 -07:00
hyukjinkwon 1ee494d086 [SPARK-20492][SQL] Do not print empty parentheses for invalid primitive types in parser
## What changes were proposed in this pull request?

Currently, when the type string is invalid, it looks printing empty parentheses. This PR proposes a small improvement in an error message by removing it in the parse as below:

```scala
spark.range(1).select($"col".cast("aa"))
```

**Before**

```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa() is not supported.(line 1, pos 0)

== SQL ==
aa
^^^
```

**After**

```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa is not supported.(line 1, pos 0)

== SQL ==
aa
^^^
```

## How was this patch tested?

Unit tests in `DataTypeParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17784 from HyukjinKwon/SPARK-20492.
2017-04-30 08:24:10 -07:00
hyukjinkwon d228cd0b02 [SPARK-20442][PYTHON][DOCS] Fill up documentations for functions in Column API in PySpark
## What changes were proposed in this pull request?

This PR proposes to fill up the documentation with examples for `bitwiseOR`, `bitwiseAND`, `bitwiseXOR`. `contains`, `asc` and `desc` in `Column` API.

Also, this PR fixes minor typos in the documentation and matches some of the contents between Scala doc and Python doc.

Lastly, this PR suggests to use `spark` rather than `sc` in doc tests in `Column` for Python documentation.

## How was this patch tested?

Doc tests were added and manually tested with the commands below:

`./python/run-tests.py --module pyspark-sql`
`./python/run-tests.py --module pyspark-sql --python-executable python3`
`./dev/lint-python`

Output was checked via `make html` under `./python/docs`. The snapshots will be left on the codes with comments.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17737 from HyukjinKwon/SPARK-20442.
2017-04-29 13:46:40 -07:00