Commit graph

8661 commits

Author SHA1 Message Date
HyukjinKwon 882f54b0a3 [SPARK-29870][SQL][FOLLOW-UP] Keep CalendarInterval's toString
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/26418. This PR removed `CalendarInterval`'s `toString` with an unfinished changes.

### Why are the changes needed?

1. Ideally we should make each PR isolated and separate targeting one issue without touching unrelated codes.

2. There are some other places where the string formats were exposed to users. For example:

    ```scala
    scala> sql("select interval 1 days as a").selectExpr("to_csv(struct(a))").show()
    ```
    ```
    +--------------------------+
    |to_csv(named_struct(a, a))|
    +--------------------------+
    |      "CalendarInterval...|
    +--------------------------+
    ```

3.  Such fixes:

    ```diff
     private def writeMapData(
        map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = {
      val keyArray = map.keyArray()
    + val keyString = mapType.keyType match {
    +   case CalendarIntervalType =>
    +    (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i))
    +   case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString
    + }
    ```

    can cause performance regression due to type dispatch for each map.

### Does this PR introduce any user-facing change?

Yes, see 2. case above.

### How was this patch tested?

Manually tested.

Closes #26572 from HyukjinKwon/SPARK-29783.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-19 09:11:41 +09:00
HyukjinKwon 8469614c05 [SPARK-25694][SQL][FOLLOW-UP] Move 'spark.sql.defaultUrlStreamHandlerFactory.enabled' into StaticSQLConf.scala
### What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/26530 and proposes to move the configuration `spark.sql.defaultUrlStreamHandlerFactory.enabled` to `StaticSQLConf.scala` for consistency.

### Why are the changes needed?

To put the similar configurations together and for readability.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Manually tested as described in https://github.com/apache/spark/pull/26530.

Closes #26570 from HyukjinKwon/SPARK-25694.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-19 09:08:20 +09:00
Kent Yao ea010a2bc2 [SPARK-29873][SQL][TEST][FOLLOWUP] set operations should not escape when regen golden file with --SET --import both specified
### What changes were proposed in this pull request?

When regenerating golden files, the set operations via `--SET` will not be done, but those with --import should be exceptions because we need the set command.

### Why are the changes needed?

fix test tool.
### Does this PR introduce any user-facing change?

### How was this patch tested?

add ut, but I'm not sure we need these tests for tests itself.
cc maropu cloud-fan

Closes #26557 from yaooqinn/SPARK-29873.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 01:32:13 +08:00
Kent Yao ae6b711b26 [SPARK-29941][SQL] Add ansi type aliases for char and decimal
### What changes were proposed in this pull request?

Checked with SQL Standard and PostgreSQL

> CHAR is equivalent to CHARACTER. DEC is equivalent to DECIMAL. INT is equivalent to INTEGER. VARCHAR is equivalent to CHARACTER VARYING. ...

```sql
postgres=# select dec '1.0';
numeric
---------
1.0
(1 row)

postgres=# select CHARACTER '. second';
  bpchar
----------
 . second
(1 row)

postgres=# select CHAR '. second';
  bpchar
----------
 . second
(1 row)
```

### Why are the changes needed?

For better ansi support
### Does this PR introduce any user-facing change?

yes, we add character as char and dec as decimal

### How was this patch tested?

add ut

Closes #26574 from yaooqinn/SPARK-29941.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 23:30:31 +08:00
fuwhu c32e228689 [SPARK-29859][SQL] ALTER DATABASE (SET LOCATION) should look up catalog like v2 commands
### What changes were proposed in this pull request?
Add AlterNamespaceSetLocationStatement, AlterNamespaceSetLocation, AlterNamespaceSetLocationExec to make ALTER DATABASE (SET LOCATION) look up catalog like v2 commands.
And also refine the code of AlterNamespaceSetProperties, AlterNamespaceSetPropertiesExec, DescribeNamespace, DescribeNamespaceExec to use SupportsNamespaces instead of CatalogPlugin for catalog parameter.

### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.

### Does this PR introduce any user-facing change?
Yes, add "ALTER NAMESPACE ... SET LOCATION" whose function is same as "ALTER DATABASE ... SET LOCATION" and "ALTER SCHEMA ... SET LOCATION".

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

Closes #26562 from fuwhu/SPARK-29859.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 20:40:23 +08:00
Kent Yao 50f6d930da [SPARK-29870][SQL] Unify the logic of multi-units interval string to CalendarInterval
### What changes were proposed in this pull request?

We now have two different implementation for multi-units interval strings to CalendarInterval type values.

One is used to covert interval string literals to CalendarInterval. This approach will re-delegate the interval string to spark parser which handles the string as a `singleInterval` -> `multiUnitsInterval` -> eventually call `IntervalUtils.fromUnitStrings`

The other is used in `Cast`, which eventually calls `IntervalUtils.stringToInterval`. This approach is ~10 times faster than the other.

We should unify these two for better performance and simple logic. this pr uses the 2nd approach.

### Why are the changes needed?

We should unify these two for better performance and simple logic.

### Does this PR introduce any user-facing change?

no

### How was this patch tested?

we shall not fail on existing uts

Closes #26491 from yaooqinn/SPARK-29870.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 15:50:06 +08:00
Kent Yao 5cebe587c7 [SPARK-29783][SQL] Support SQL Standard/ISO_8601 output style for interval type
### What changes were proposed in this pull request?

Add 3 interval output types which are named as `SQL_STANDARD`, `ISO_8601`, `MULTI_UNITS`. And we add a new conf `spark.sql.dialect.intervalOutputStyle` for this. The `MULTI_UNITS` style displays the interval values in the former behavior and it is the default. The newly added `SQL_STANDARD`, `ISO_8601` styles can be found in the following table.

Style | conf | Year-Month Interval | Day-Time Interval | Mixed Interval
-- | -- | -- | -- | --
Format With Time Unit Designators | MULTI_UNITS | 1 year 2 mons | 1 days 2 hours 3 minutes 4.123456 seconds | interval 1 days 2 hours 3 minutes 4.123456 seconds
SQL STANDARD  | SQL_STANDARD | 1-2 | 3 4:05:06 | -1-2 3 -4:05:06
ISO8601 Basic Format| ISO_8601| P1Y2M| P3DT4H5M6S|P-1Y-2M3D-4H-5M-6S

### Why are the changes needed?

for ANSI SQL support
### Does this PR introduce any user-facing change?

yes,interval out now has 3 output styles
### How was this patch tested?

add new unit tests

cc cloud-fan maropu MaxGekk HyukjinKwon thanks.

Closes #26418 from yaooqinn/SPARK-29783.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 15:42:22 +08:00
gschiavon 73912379d0 [SPARK-29020][SQL] Improving array_sort behaviour
### What changes were proposed in this pull request?
I've noticed that there are two functions to sort arrays sort_array and array_sort.

sort_array is from 1.5.0 and it has the possibility of ordering both ascending and descending

array_sort is from 2.4.0 and it only has the possibility of ordering in ascending.

Basically I just added the possibility of ordering either ascending or descending using array_sort.

I think it would be good to have unified behaviours and not having to user sort_array when you want to order in descending order.
Imagine that you are new to spark, I'd like to be able to sort array using the newest spark functions.

### Why are the changes needed?
Basically to be able to sort the array in descending order using *array_sort* instead of using *sort_array* from 1.5.0

### Does this PR introduce any user-facing change?
Yes, now you are able to sort the array in descending order. Note that it has the same behaviour with nulls than sort_array

### How was this patch tested?
Test's added

This is the link to the [jira](https://issues.apache.org/jira/browse/SPARK-29020)

Closes #25728 from Gschiavon/improving-array-sort.

Lead-authored-by: gschiavon <german.schiavon@lifullconnect.com>
Co-authored-by: Takuya UESHIN <ueshin@databricks.com>
Co-authored-by: gschiavon <Gschiavon@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-18 16:07:05 +09:00
Zhou Jiang ee3bd6d768 [SPARK-25694][SQL] Add a config for URL.setURLStreamHandlerFactory
### What changes were proposed in this pull request?

Add a property `spark.fsUrlStreamHandlerFactory.enabled` to allow users turn off the default registration of `org.apache.hadoop.fs.FsUrlStreamHandlerFactory`

### Why are the changes needed?

This [SPARK-25694](https://issues.apache.org/jira/browse/SPARK-25694) is a long-standing issue. Originally, [[SPARK-12868][SQL] Allow adding jars from hdfs](https://github.com/apache/spark/pull/17342 ) added this for better Hive support. However, this have a side-effect when the users use Apache Spark without `-Phive`. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users.

### Does this PR introduce any user-facing change?

Yes. This provides a new user-configurable property.
By default, the behavior is unchanged.

### How was this patch tested?

Manual testing.

**BEFORE**
```
$ build/sbt package
$ bin/spark-shell
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
java.lang.Error: factory already defined
  at java.net.URL.setURLStreamHandlerFactory(URL.java:1134)
  ... 47 elided
```

**AFTER**
```
$ build/sbt package
$ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
```

Closes #26530 from jiangzho/master.

Lead-authored-by: Zhou Jiang <zhou_jiang@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: zhou-jiang <zhou_jiang@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-11-18 05:44:00 +00:00
xy_xin d83cacfcf5 [SPARK-29907][SQL] Move DELETE/UPDATE/MERGE relative rules to dmlStatementNoWith to support cte
### What changes were proposed in this pull request?

SPARK-27444 introduced `dmlStatementNoWith` so that any dml that needs cte support can leverage it. It be better if we move DELETE/UPDATE/MERGE rules to `dmlStatementNoWith`.

### Why are the changes needed?
Wit this change, we can support syntax like "With t AS (SELECT) DELETE FROM xxx", and so as UPDATE/MERGE.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?

New cases added.

Closes #26536 from xianyinxin/SPARK-29907.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 11:48:56 +08:00
Maxim Gekk 5eb8973f87 [SPARK-29930][SQL] Remove SQL configs declared to be removed in Spark 3.0
### What changes were proposed in this pull request?
In the PR, I propose to remove the following SQL configs:
1. `spark.sql.fromJsonForceNullableSchema`
2. `spark.sql.legacy.compareDateTimestampInTimestamp`
3. `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation`

that are declared to be removed in Spark 3.0

### Why are the changes needed?
To make code cleaner and improve maintainability.

### Does this PR introduce any user-facing change?
Yes

### How was this patch tested?
By `TypeCoercionSuite`, `JsonExpressionsSuite` and `DDLSuite`.

Closes #26559 from MaxGekk/remove-sql-configs.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-17 10:14:04 -08:00
Pavithra Ramachandran a9959be2bc [SPARK-29456][WEBUI] Improve tooltip for Session Statistics Table column in JDBC/ODBC Server Tab
What changes were proposed in this pull request?
Some of the columns of JDBC/ODBC tab  Session info in Web UI are hard to understand.

Add tool tip for Start time, finish time , Duration and Total Execution

![Screenshot from 2019-10-16 12-33-17](https://user-images.githubusercontent.com/51401130/66901981-76d68980-f01d-11e9-9686-e20346a38c25.png)

Why are the changes needed?
To improve the understanding of the WebUI

Does this PR introduce any user-facing change?
No

How was this patch tested?
manual test

Closes #26138 from PavithraRamachandran/JDBC_tooltip.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-17 07:04:40 -06:00
fuwhu 388a737b98 [SPARK-29858][SQL] ALTER DATABASE (SET DBPROPERTIES) should look up catalog like v2 commands
### What changes were proposed in this pull request?
Add AlterNamespaceSetPropertiesStatement, AlterNamespaceSetProperties and AlterNamespaceSetPropertiesExec to make ALTER DATABASE (SET DBPROPERTIES) command look up catalog like v2 commands.

### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.

### Does this PR introduce any user-facing change?
Yes, add "ALTER NAMESPACE ... SET (DBPROPERTIES | PROPERTIES) ..." whose function is same as "ALTER DATABASE ... SET DBPROPERTIES ..." and "ALTER SCHEMA ... SET DBPROPERTIES ...".

### How was this patch tested?
New unit test

Closes #26551 from fuwhu/SPARK-29858.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-16 19:50:02 -08:00
Maxim Gekk e88267cb5a [SPARK-29928][SQL][TESTS] Check parsing timestamps up to microsecond precision by JSON/CSV datasource
### What changes were proposed in this pull request?
In the PR, I propose to add tests from the commit 9c7e8be1dc for Spark 2.4 that check parsing of timestamp strings for various seconds fractions.

### Why are the changes needed?
To make sure that current behavior is the same as in Spark 2.4

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`.

Closes #26558 from MaxGekk/parse-timestamp-micros-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-16 18:01:25 -08:00
Yuanjian Li 40ea4a11d7 [SPARK-29807][SQL] Rename "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled"
### What changes were proposed in this pull request?
Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled"

### Why are the changes needed?
The relation between "spark.sql.ansi.enabled" and "spark.sql.dialect" is confusing, since the "PostgreSQL" dialect should contain the features of "spark.sql.ansi.enabled".

To make things clearer, we can rename the "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled", thus the option "spark.sql.dialect.spark.ansi.enabled" is only for Spark dialect.

For the casting and arithmetic operations, runtime exceptions should be thrown if "spark.sql.dialect" is "spark" and "spark.sql.dialect.spark.ansi.enabled" is true or "spark.sql.dialect" is PostgresSQL.

### Does this PR introduce any user-facing change?
Yes, the config name changed.

### How was this patch tested?
Existing UT.

Closes #26444 from xuanyuanking/SPARK-29807.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-16 17:46:39 +08:00
Dongjoon Hyun f77c10de38 [SPARK-29923][SQL][TESTS] Set io.netty.tryReflectionSetAccessible for Arrow on JDK9+
### What changes were proposed in this pull request?

This PR aims to add `io.netty.tryReflectionSetAccessible=true` to the testing configuration for JDK11 because this is an officially documented requirement of Apache Arrow.

Apache Arrow community documented this requirement at `0.15.0` ([ARROW-6206](https://github.com/apache/arrow/pull/5078)).
> #### For java 9 or later, should set "-Dio.netty.tryReflectionSetAccessible=true".
> This fixes `java.lang.UnsupportedOperationException: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available`. thrown by netty.

### Why are the changes needed?

After ARROW-3191, Arrow Java library requires the property `io.netty.tryReflectionSetAccessible` to be set to true for JDK >= 9. After https://github.com/apache/spark/pull/26133, JDK11 Jenkins job seem to fail.

- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/676/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/677/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/678/

```scala
Previous exception in task:
sun.misc.Unsafe or java.nio.DirectByteBuffer.<init>(long, int) not available&#010;
io.netty.util.internal.PlatformDependent.directBuffer(PlatformDependent.java:473)&#010;
io.netty.buffer.NettyArrowBuf.getDirectBuffer(NettyArrowBuf.java:243)&#010;
io.netty.buffer.NettyArrowBuf.nioBuffer(NettyArrowBuf.java:233)&#010;
io.netty.buffer.ArrowBuf.nioBuffer(ArrowBuf.java:245)&#010;
org.apache.arrow.vector.ipc.message.ArrowRecordBatch.computeBodyLength(ArrowRecordBatch.java:222)&#010;
```

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Pass the Jenkins with JDK11.

Closes #26552 from dongjoon-hyun/SPARK-ARROW-JDK11.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 23:58:15 -08:00
Takeshi Yamamuro 6d6b233791 [SPARK-29343][SQL][FOLLOW-UP] Remove floating-point Sum/Average/CentralMomentAgg from order-insensitive aggregates
### What changes were proposed in this pull request?

This pr is to remove floating-point `Sum/Average/CentralMomentAgg` from order-insensitive aggregates in `EliminateSorts`.

This pr comes from the gatorsmile suggestion: https://github.com/apache/spark/pull/26011#discussion_r344583899

### Why are the changes needed?

Bug fix.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Added tests in `SubquerySuite`.

Closes #26534 from maropu/SPARK-29343-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 18:54:02 -08:00
fuwhu 16e7195299 [SPARK-29834][SQL] DESC DATABASE should look up catalog like v2 commands
### What changes were proposed in this pull request?
Add DescribeNamespaceStatement, DescribeNamespace and DescribeNamespaceExec
to make "DESC DATABASE" look up catalog like v2 commands.

### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.

### Does this PR introduce any user-facing change?
Yes, add "DESC NAMESPACE" whose function is same as "DESC DATABASE" and "DESC SCHEMA".

### How was this patch tested?
New unit test

Closes #26513 from fuwhu/SPARK-29834.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 18:50:42 -08:00
HyukjinKwon 7720781695 [SPARK-29127][SQL][PYTHON] Add a clue for Python related version information in integrated UDF tests
### What changes were proposed in this pull request?

This PR proposes to show Python, pandas and PyArrow versions in integrated UDF tests as a clue so when the test cases fail, it show the related version information.

I think we don't really need this kind of version information in the test case name for now since I intend that integrated SQL test cases do not target to test different combinations of Python, Pandas and PyArrow.

### Why are the changes needed?

To make debug easier.

### Does this PR introduce any user-facing change?

It will change test name to include related Python, pandas and PyArrow versions.

### How was this patch tested?

Manually tested:

```
[info] - udf/postgreSQL/udf-case.sql - Scala UDF *** FAILED *** (8 seconds, 229 milliseconds)
[info]   udf/postgreSQL/udf-case.sql - Scala UDF
...
[info] - udf/postgreSQL/udf-case.sql - Regular Python UDF *** FAILED *** (6 seconds, 298 milliseconds)
[info]   udf/postgreSQL/udf-case.sql - Regular Python UDF
[info]   Python: 3.7
...
[info] - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF *** FAILED *** (6 seconds, 376 milliseconds)
[info]   udf/postgreSQL/udf-case.sql - Scalar Pandas UDF
[info]   Python: 3.7 Pandas: 0.25.3 PyArrow: 0.14.0
```

Closes #26538 from HyukjinKwon/investigate-flaky-test.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 18:37:33 -08:00
Pablo Langa 848bdfa218 [SPARK-29829][SQL] SHOW TABLE EXTENDED should do multi-catalog resolution
### What changes were proposed in this pull request?

Add ShowTableStatement and make SHOW TABLE EXTENDED go through the same catalog/table resolution framework of v2 commands.

We don’t have this methods in the catalog to implement an V2 command

- catalog.getPartition
- catalog.getTempViewOrPermanentTableMetadata

### Why are the changes needed?

It's important to make all the commands have the same table resolution behavior, to avoid confusing

```sql
USE my_catalog
DESC t // success and describe the table t from my_catalog
SHOW TABLE EXTENDED FROM LIKE 't' // report table not found as there is no table t in the session catalog
```

### Does this PR introduce any user-facing change?

Yes. When running SHOW TABLE EXTENDED Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.

### How was this patch tested?

Unit tests.

Closes #26540 from planga82/feature/SPARK-29481_ShowTableExtended.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 14:25:33 -08:00
Takeshi Yamamuro ee4784bf26 [SPARK-26499][SQL][FOLLOW-UP] Replace update with setByte for ByteType in JdbcUtils.makeGetter
### What changes were proposed in this pull request?

This is a follow-up pr to fix the code coming from #23400; it replaces `update` with `setByte` for ByteType in `JdbcUtils.makeGetter`.

### Why are the changes needed?

For better code.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #26532 from maropu/SPARK-26499-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-15 08:12:41 -06:00
Yuming Wang 4f10e54ba3 [SPARK-29655][SQL] Read bucketed tables obeys spark.sql.shuffle.partitions
### What changes were proposed in this pull request?

In order to avoid frequently changing the value of `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions`, we usually set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` much larger than `spark.sql.shuffle.partitions` after enabling adaptive execution, which causes some bucket map join lose efficacy and add more `ShuffleExchange`.

How to reproduce:
```scala
val bucketedTableName = "bucketed_table"
spark.range(10000).write.bucketBy(500, "id").sortBy("id").mode(org.apache.spark.sql.SaveMode.Overwrite).saveAsTable(bucketedTableName)
val bucketedTable = spark.table(bucketedTableName)
val df = spark.range(8)

spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1)
// Spark 2.4. spark.sql.adaptive.enabled=false
// We set spark.sql.shuffle.partitions <= 500 every time based on our data in this case.
spark.conf.set("spark.sql.shuffle.partitions", 500)
bucketedTable.join(df, "id").explain()
// Since 3.0. We enabled adaptive execution and set spark.sql.adaptive.shuffle.maxNumPostShufflePartitions to a larger values to fit more cases.
spark.conf.set("spark.sql.adaptive.enabled", true)
spark.conf.set("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions", 1000)
bucketedTable.join(df, "id").explain()
```

```
scala> bucketedTable.join(df, "id").explain()
== Physical Plan ==
*(4) Project [id#5L]
+- *(4) SortMergeJoin [id#5L], [id#7L], Inner
   :- *(1) Sort [id#5L ASC NULLS FIRST], false, 0
   :  +- *(1) Project [id#5L]
   :     +- *(1) Filter isnotnull(id#5L)
   :        +- *(1) ColumnarToRow
   :           +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 500 out of 500
   +- *(3) Sort [id#7L ASC NULLS FIRST], false, 0
      +- Exchange hashpartitioning(id#7L, 500), true, [id=#49]
         +- *(2) Range (0, 8, step=1, splits=16)
```
vs
```
scala> bucketedTable.join(df, "id").explain()
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- Project [id#5L]
   +- SortMergeJoin [id#5L], [id#7L], Inner
      :- Sort [id#5L ASC NULLS FIRST], false, 0
      :  +- Exchange hashpartitioning(id#5L, 1000), true, [id=#93]
      :     +- Project [id#5L]
      :        +- Filter isnotnull(id#5L)
      :           +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 500 out of 500
      +- Sort [id#7L ASC NULLS FIRST], false, 0
         +- Exchange hashpartitioning(id#7L, 1000), true, [id=#92]
            +- Range (0, 8, step=1, splits=16)
```

This PR makes read bucketed tables always obeys `spark.sql.shuffle.partitions` even enabling adaptive execution and set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` to avoid add more `ShuffleExchange`.

### Why are the changes needed?
Do not degrade performance after enabling adaptive execution.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Unit test.

Closes #26409 from wangyum/SPARK-29655.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-15 15:49:24 +08:00
Kent Yao 0c68578fa9 [SPARK-29888][SQL] new interval string parser shall handle numeric with only fractional part
### What changes were proposed in this pull request?

Current string to interval cast logic does not support i.e. cast('.111 second' as interval) which will fail in SIGN state and return null, actually, it is 00:00:00.111.
```scala
-- !query 63
select interval '.111 seconds'
-- !query 63 schema
struct<0.111 seconds:interval>
-- !query 63 output
0.111 seconds

-- !query 64
select cast('.111 seconds' as interval)
-- !query 64 schema
struct<CAST(.111 seconds AS INTERVAL):interval>
-- !query 64 output
NULL
````
### Why are the changes needed?

bug fix.
### Does this PR introduce any user-facing change?

no
### How was this patch tested?

add ut

Closes #26514 from yaooqinn/SPARK-29888.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-15 13:33:30 +08:00
Bryan Cutler 65a189c7a1 [SPARK-29376][SQL][PYTHON] Upgrade Apache Arrow to version 0.15.1
### What changes were proposed in this pull request?

Upgrade Apache Arrow to version 0.15.1. This includes Java artifacts and increases the minimum required version of PyArrow also.

Version 0.12.0 to 0.15.1 includes the following selected fixes/improvements relevant to Spark users:

* ARROW-6898 - [Java] Fix potential memory leak in ArrowWriter and several test classes
* ARROW-6874 - [Python] Memory leak in Table.to_pandas() when conversion to object dtype
* ARROW-5579 - [Java] shade flatbuffer dependency
* ARROW-5843 - [Java] Improve the readability and performance of BitVectorHelper#getNullCount
* ARROW-5881 - [Java] Provide functionalities to efficiently determine if a validity buffer has completely 1 bits/0 bits
* ARROW-5893 - [C++] Remove arrow::Column class from C++ library
* ARROW-5970 - [Java] Provide pointer to Arrow buffer
* ARROW-6070 - [Java] Avoid creating new schema before IPC sending
* ARROW-6279 - [Python] Add Table.slice method or allow slices in \_\_getitem\_\_
* ARROW-6313 - [Format] Tracking for ensuring flatbuffer serialized values are aligned in stream/files.
* ARROW-6557 - [Python] Always return pandas.Series from Array/ChunkedArray.to_pandas, propagate field names to Series from RecordBatch, Table
* ARROW-2015 - [Java] Use Java Time and Date APIs instead of JodaTime
* ARROW-1261 - [Java] Add container type for Map logical type
* ARROW-1207 - [C++] Implement Map logical type

Changelog can be seen at https://arrow.apache.org/release/0.15.0.html

### Why are the changes needed?

Upgrade to get bug fixes, improvements, and maintain compatibility with future versions of PyArrow.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Existing tests, manually tested with Python 3.7, 3.8

Closes #26133 from BryanCutler/arrow-upgrade-015-SPARK-29376.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-15 13:27:30 +09:00
Wenchen Fan bb8b04d4a2 [SPARK-29889][SQL][TEST] unify the interval tests
### What changes were proposed in this pull request?

move interval tests to `interval.sql`, and import it to `ansi/interval.sql`

### Why are the changes needed?

improve test coverage

### Does this PR introduce any user-facing change?

no

### How was this patch tested?

N/A

Closes #26515 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-15 10:38:51 +08:00
HyukjinKwon 17321782de [SPARK-26923][R][SQL][FOLLOW-UP] Show stderr in the exception whenever possible in RRunner
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/23977 I made a mistake related to this line: 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL112)

Previously,

1. the reader iterator for R worker read some initial data eagerly during RDD materialization. So it read the data before actual execution. For some reasons, in this case, it showed standard error from R worker.

2. After that, when error happens during actual execution, stderr wasn't shown: 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL260)

After my change 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL112), it now ignores 1. case and only does 2. of previous code path, because 1. does not happen anymore as I avoided to such eager execution (which is consistent with PySpark code path).

This PR proposes to do only 1.  before/after execution always because It is pretty much possible R worker was failed during actual execution and it's best to show the stderr from R worker whenever possible.

### Why are the changes needed?

It currently swallows standard error from R worker which makes debugging harder.

### Does this PR introduce any user-facing change?

Yes,

```R
df <- createDataFrame(list(list(n=1)))
collect(dapply(df, function(x) {
  stop("asdkjasdjkbadskjbsdajbk")
  x
}, structType("a double")))
```

**Before:**

```
Error in handleErrors(returnStatus, conn) :
  org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 13.0 failed 1 times, most recent failure: Lost task 0.0 in stage 13.0 (TID 13, 192.168.35.193, executor driver): org.apache.spark.SparkException: R worker exited unexpectedly (cranshed)
	at org.apache.spark.api.r.RRunner$$anon$1.read(RRunner.scala:130)
	at org.apache.spark.api.r.BaseRRunner$ReaderIterator.hasNext(BaseRRunner.scala:118)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:726)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:337)
	at org.apache.spark.
```

**After:**

```
Error in handleErrors(returnStatus, conn) :
  org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, 192.168.35.193, executor driver): org.apache.spark.SparkException: R unexpectedly exited.
R worker produced errors: Error in computeFunc(inputData) : asdkjasdjkbadskjbsdajbk

	at org.apache.spark.api.r.BaseRRunner$ReaderIterator$$anonfun$1.applyOrElse(BaseRRunner.scala:144)
	at org.apache.spark.api.r.BaseRRunner$ReaderIterator$$anonfun$1.applyOrElse(BaseRRunner.scala:137)
	at scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38)
	at org.apache.spark.api.r.RRunner$$anon$1.read(RRunner.scala:128)
	at org.apache.spark.api.r.BaseRRunner$ReaderIterator.hasNext(BaseRRunner.scala:113)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegen
```

### How was this patch tested?

Manually tested and unittest was added.

Closes #26517 from HyukjinKwon/SPARK-26923-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-15 11:13:36 +09:00
Terry Kim e46e487b08 [SPARK-29682][SQL] Resolve conflicting attributes in Expand correctly
### What changes were proposed in this pull request?

This PR addresses issues where conflicting attributes in `Expand` are not correctly handled.

### Why are the changes needed?

```Scala
val numsDF = Seq(1, 2, 3, 4, 5, 6).toDF("nums")
val cubeDF = numsDF.cube("nums").agg(max(lit(0)).as("agcol"))
cubeDF.join(cubeDF, "nums").show
```
fails with the following exception:
```
org.apache.spark.sql.AnalysisException:
Failure when resolving conflicting references in Join:
'Join Inner
:- Aggregate [nums#38, spark_grouping_id#36], [nums#38, max(0) AS agcol#35]
:  +- Expand [List(nums#3, nums#37, 0), List(nums#3, null, 1)], [nums#3, nums#38, spark_grouping_id#36]
:     +- Project [nums#3, nums#3 AS nums#37]
:        +- Project [value#1 AS nums#3]
:           +- LocalRelation [value#1]
+- Aggregate [nums#38, spark_grouping_id#36], [nums#38, max(0) AS agcol#58]
   +- Expand [List(nums#3, nums#37, 0), List(nums#3, null, 1)], [nums#3, nums#38, spark_grouping_id#36]
                                                                         ^^^^^^^
      +- Project [nums#3, nums#3 AS nums#37]
         +- Project [value#1 AS nums#3]
            +- LocalRelation [value#1]

Conflicting attributes: nums#38
```
As you can see from the above plan, `num#38`, the output of `Expand` on the right side of `Join`, should have been handled to produce new attribute. Since the conflict is not resolved in `Expand`, the failure is happening upstream at `Aggregate`. This PR addresses handling conflicting attributes in `Expand`.

### Does this PR introduce any user-facing change?

Yes, the previous example now shows the following output:
```
+----+-----+-----+
|nums|agcol|agcol|
+----+-----+-----+
|   1|    0|    0|
|   6|    0|    0|
|   4|    0|    0|
|   2|    0|    0|
|   5|    0|    0|
|   3|    0|    0|
+----+-----+-----+
```
### How was this patch tested?

Added new unit test.

Closes #26441 from imback82/spark-29682.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-14 14:47:14 +08:00
Takeshi Yamamuro b5a02d37e6 [SPARK-29873][SQL][TESTS] Support --import directive to load queries from another test case in SQLQueryTestSuite
### What changes were proposed in this pull request?

This pr is to support `--import` directive to load queries from another test case in SQLQueryTestSuite.

This fix comes from the cloud-fan suggestion in https://github.com/apache/spark/pull/26479#discussion_r345086978

### Why are the changes needed?

This functionality might reduce duplicate test code in `SQLQueryTestSuite`.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Run `SQLQueryTestSuite`.

Closes #26497 from maropu/ImportTests.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-14 14:38:27 +08:00
wuyi fe1f456b20 [SPARK-29837][SQL] PostgreSQL dialect: cast to boolean
### What changes were proposed in this pull request?

Make SparkSQL's `cast to boolean` behavior be consistent with PostgreSQL when
spark.sql.dialect is configured as PostgreSQL.

### Why are the changes needed?

SparkSQL and PostgreSQL have a lot different cast behavior between types by default. We should make SparkSQL's cast behavior be consistent with PostgreSQL when `spark.sql.dialect` is configured as PostgreSQL.

### Does this PR introduce any user-facing change?

Yes. If user switches to PostgreSQL dialect now, they will

* get an exception if they input a invalid string, e.g "erut", while they get `null` before;

* get an exception if they input `TimestampType`, `DateType`, `LongType`, `ShortType`, `ByteType`, `DecimalType`, `DoubleType`, `FloatType` values,  while they get `true` or `false` result before.

And here're evidences for those unsupported types from PostgreSQL:

timestamp:
```
postgres=# select cast(cast('2019-11-11' as timestamp) as boolean);
ERROR:  cannot cast type timestamp without time zone to boolean
```

date:
```
postgres=# select cast(cast('2019-11-11' as date) as boolean);
ERROR:  cannot cast type date to boolean
```

bigint:
```
postgres=# select cast(cast('20191111' as bigint) as boolean);
ERROR:  cannot cast type bigint to boolean
```

smallint:
```
postgres=# select cast(cast(2019 as smallint) as boolean);
ERROR:  cannot cast type smallint to boolean
```

bytea:
```
postgres=# select cast(cast('2019' as bytea) as boolean);
ERROR:  cannot cast type bytea to boolean
```

decimal:
```
postgres=# select cast(cast('2019' as decimal) as boolean);
ERROR:  cannot cast type numeric to boolean
```

float:
```
postgres=# select cast(cast('2019' as float) as boolean);
ERROR:  cannot cast type double precision to boolean
```

### How was this patch tested?

Added and tested manually.

Closes #26463 from Ngone51/dev-postgre-cast2bool.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-14 11:55:01 +08:00
Liang-Chi Hsieh 39596b913b [SPARK-29649][SQL] Stop task set if FileAlreadyExistsException was thrown when writing to output file
### What changes were proposed in this pull request?

We already know task attempts that do not clean up output files in staging directory can cause job failure (SPARK-27194). There was proposals trying to fix it by changing output filename, or deleting existing output files. These proposals are not reliable completely.

The difficulty is, as previous failed task attempt wrote the output file, at next task attempt the output file is still under same staging directory, even the output file name is different.

If the job will go to fail eventually, there is no point to re-run the task until max attempts are reached. For the jobs running a lot of time, re-running the task can waste a lot of time.

This patch proposes to let Spark detect such file already exist exception and stop the task set early.

### Why are the changes needed?

For now, if FileAlreadyExistsException is thrown during data writing job in SQL, the job will continue re-running task attempts until max failure number is reached. It is no point for re-running tasks as task attempts will also fail because they can not write to the existing file too. We should stop the task set early.

### Does this PR introduce any user-facing change?

Yes. If FileAlreadyExistsException is thrown during data writing job in SQL, no more task attempts are re-tried and the task set will be stoped early.

### How was this patch tested?

Unit test.

Closes #26312 from viirya/stop-taskset-if-outputfile-exists.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-13 18:01:38 -08:00
shivsood 32d44b1d0e [SPARK-29644][SQL] Corrected ShortType and ByteType mapping to SmallInt and TinyInt in JDBCUtils
### What changes were proposed in this pull request?
Corrected ShortType and ByteType mapping to SmallInt and TinyInt, corrected setter methods to set ShortType and ByteType  as setShort() and setByte(). Changes in JDBCUtils.scala
Fixed Unit test cases to where applicable and added new E2E test cases in to test table read/write using ShortType and ByteType.

#### Problems

- In master in JDBCUtils.scala line number 547 and 551 have a problem where ShortType and ByteType are set as Integers rather than set as Short and Byte respectively.
```
case ShortType =>
(stmt: PreparedStatement, row: Row, pos: Int) =>
stmt.setInt(pos + 1, row.getShort(pos))
The issue was pointed out by maropu

case ByteType =>
(stmt: PreparedStatement, row: Row, pos: Int) =>
 stmt.setInt(pos + 1, row.getByte(pos))
```

- Also at line JDBCUtils.scala 247 TinyInt is interpreted wrongly as IntergetType in getCatalystType()

``` case java.sql.Types.TINYINT       => IntegerType ```

- At line 172 ShortType was wrongly interpreted as IntegerType
``` case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) ```

- All thru out tests, ShortType and ByteType were being interpreted as IntegerTypes.

### Why are the changes needed?
A given type should be set using the right type.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Corrected Unit test cases where applicable. Validated in CI/CD
Added a test case in MsSqlServerIntegrationSuite.scala, PostgresIntegrationSuite.scala , MySQLIntegrationSuite.scala to write/read tables from dataframe with cols as shorttype and bytetype. Validated by manual as follows.
```
./build/mvn install -DskipTests
./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12
```

Closes #26301 from shivsood/shorttype_fix_maropu.

Authored-by: shivsood <shivsood@microsoft.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-13 17:56:13 -08:00
Wesley Hoffman 39b502af17 [SPARK-29778][SQL] pass writer options to saveAsTable in append mode
### What changes were proposed in this pull request?

`saveAsTable` had an oversight where write options were not considered in the append save mode.

### Why are the changes needed?

Address the bug so that write options can be considered during appends.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Unit test added that looks in the logic plan of `AppendData` for the existing write options.

Closes #26474 from SpaceRangerWes/master.

Authored-by: Wesley Hoffman <wesleyhoffman109@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-13 14:10:30 -08:00
Burak Yavuz 363af16c72 [SPARK-29568][SS] Stop existing running streams when a new stream is launched
### What changes were proposed in this pull request?

This PR adds a SQL Conf: `spark.sql.streaming.stopActiveRunOnRestart`. When this conf is `true` (by default it is), an already running stream will be stopped, if a new copy gets launched on the same checkpoint location.

### Why are the changes needed?

In multi-tenant environments where you have multiple SparkSessions, you can accidentally start multiple copies of the same stream (i.e. streams using the same checkpoint location). This will cause all new instantiations of the new stream to fail. However, sometimes you may want to turn off the old stream, as the old stream may have turned into a zombie (you no longer have access to the query handle or SparkSession).

It would be nice to have a SQL flag that allows the stopping of the old stream for such zombie cases.

### Does this PR introduce any user-facing change?

Yes. Now by default, if you launch a new copy of an already running stream on a multi-tenant cluster, the existing stream will be stopped.

### How was this patch tested?

Unit tests in StreamingQueryManagerSuite

Closes #26225 from brkyvz/stopStream.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-11-13 08:59:46 -08:00
Wenchen Fan 4dcbdcd265 [SPARK-29863][SQL] Rename EveryAgg/AnyAgg to BoolAnd/BoolOr
### What changes were proposed in this pull request?

rename EveryAgg/AnyAgg to BoolAnd/BoolOr

### Why are the changes needed?

Under ansi mode, `every`, `any` and `some` are reserved keywords and can't be used as function names. `EveryAgg`/`AnyAgg` has several aliases and I think it's better to not pick  reserved keywords  as the primary name.

### Does this PR introduce any user-facing change?

no

### How was this patch tested?

existing tests

Closes #26486 from cloud-fan/naming.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-13 21:42:42 +08:00
Wenchen Fan 942753a44b [SPARK-29753][SQL] refine the default catalog config
### What changes were proposed in this pull request?

rename the config to address the comment: https://github.com/apache/spark/pull/24594#discussion_r285431212

improve the config description, provide a default value to simplify the code.

### Why are the changes needed?

make the config more understandable.

### Does this PR introduce any user-facing change?

no

### How was this patch tested?

existing tests

Closes #26395 from cloud-fan/config.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-13 21:27:36 +08:00
xy_xin d7bdc6aa17 [SPARK-29835][SQL] Remove the unnecessary conversion from Statement to LogicalPlan for DELETE/UPDATE
### What changes were proposed in this pull request?

The current parse and analyze flow for DELETE is: 1, the SQL string will be firstly parsed to `DeleteFromStatement`; 2, the `DeleteFromStatement` be converted to `DeleteFromTable`. However, the SQL string can be parsed to `DeleteFromTable` directly, where a `DeleteFromStatement` seems to be redundant.

It is the same for UPDATE.

This pr removes the unnecessary `DeleteFromStatement` and `UpdateTableStatement`.

### Why are the changes needed?

This makes the codes for DELETE and UPDATE cleaner, and keep align with MERGE INTO.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Existed tests and new tests.

Closes #26464 from xianyinxin/SPARK-29835.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-13 20:53:12 +08:00
Terry Kim b5a2ed6a37 [SPARK-29851][SQL] V2 catalog: Change default behavior of dropping namespace to cascade
### What changes were proposed in this pull request?

Currently, `SupportsNamespaces.dropNamespace` drops a namespace only if it is empty. Thus, to implement a cascading drop, one needs to iterate all objects (tables, view, etc.) within the namespace (including its sub-namespaces recursively) and drop them one by one. This can have a negative impact on the performance when there are large number of objects.

Instead, this PR proposes to change the default behavior of dropping a namespace to cascading such that implementing cascading/non-cascading drop is simpler without performance penalties.

### Why are the changes needed?

The new behavior makes implementing cascading/non-cascading drop simple without performance penalties.

### Does this PR introduce any user-facing change?

Yes. The default behavior of `SupportsNamespaces.dropNamespace` is now cascading.

### How was this patch tested?

Added new unit tests.

Closes #26476 from imback82/drop_ns_cascade.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-13 17:06:27 +08:00
Kent Yao f926809a1f [SPARK-29390][SQL] Add the justify_days(), justify_hours() and justif_interval() functions
### What changes were proposed in this pull request?

Add 3 interval functions justify_days, justify_hours, justif_interval to support justify interval values

### Why are the changes needed?

For feature parity with postgres

add three interval functions to justify interval values.

justify_days(interval) | interval | Adjust interval so 30-day time periods are represented as months | justify_days(interval '35 days') | 1 mon 5 days
-- | -- | -- | -- | --
justify_hours(interval) | interval | Adjust interval so 24-hour time periods are represented as days | justify_hours(interval '27 hours') | 1 day 03:00:00
justify_interval(interval) | interval | Adjust interval using justify_days and justify_hours, with additional sign adjustments | justify_interval(interval '1 mon -1 hour') | 29 days 23:00:00

### Does this PR introduce any user-facing change?

yes. new interval functions are added

### How was this patch tested?

add ut

Closes #26465 from yaooqinn/SPARK-29390.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-11-13 15:04:39 +09:00
HyukjinKwon 80fbc382a6 Revert "[SPARK-29462] The data type of "array()" should be array<null>"
This reverts commit 0dcd739534.
2019-11-13 13:12:20 +09:00
angerszhu eb79af8dae [SPARK-29145][SQL][FOLLOW-UP] Move tests from SubquerySuite to subquery/in-subquery/in-joins.sql
### What changes were proposed in this pull request?
Follow comment of https://github.com/apache/spark/pull/25854#discussion_r342383272

### Why are the changes needed?
NO

### Does this PR introduce any user-facing change?
NO

### How was this patch tested?
ADD TEST CASE

Closes #26406 from AngersZhuuuu/SPARK-29145-FOLLOWUP.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-12 17:34:03 -08:00
Ankitraj 45e212e161 [SPARK-29570][WEBUI] Improve tooltip for Executor Tab for Shuffle Write,Blacklisted,Logs,Threaddump columns
### What changes were proposed in this pull request?
All tooltips message will display in centre.

### Why are the changes needed?
Some time tooltips will hide the data of column and tooltips display position will be inconsistent in UI.

### Does this PR introduce any user-facing change?
yes.

![Screenshot 2019-10-26 at 3 08 51 AM](https://user-images.githubusercontent.com/8948111/67606124-04dd0d80-f79e-11e9-865a-b7e9bffc9890.png)

### How was this patch tested?
Manual test.

Closes #26263 from 07ARB/SPARK-29570.

Lead-authored-by: Ankitraj <8948111+07ARB@users.noreply.github.com>
Co-authored-by: 07ARB <ankitrajboudh@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-12 18:49:54 -06:00
Wenchen Fan 030e5d987e [SPARK-29789][SQL] should not parse the bucket column name when creating v2 tables
### What changes were proposed in this pull request?

When creating v2 expressions, we have public java APIs, as well as interval scala APIs. All of these APIs take a string column name and parse it to `NamedReference`.

This is convenient for end-users, but not for interval development. For example, the query plan already contains the parsed partition/bucket column names, and it's tricky if we need to quote the names before creating v2 expressions.

This PR proposes to change the interval scala APIs to take `NamedReference` directly, with a new method to create `NamedReference` with the exact name parts. The public java APIs are not changed.

### Why are the changes needed?

fix a bug, and make it easier to create v2 expressions correctly in the future.

### Does this PR introduce any user-facing change?

yes, now v2 CREATE TABLE works as expected.

### How was this patch tested?

a new test

Closes #26425 from cloud-fan/extract.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Ryan Blue <blue@apache.org>
2019-11-12 12:25:45 -08:00
Wenchen Fan 414cade011 [SPARK-29850][SQL] sort-merge-join an empty table should not memory leak
### What changes were proposed in this pull request?

When whole stage codegen `HashAggregateExec`, create the hash map when we begin to process inputs.

### Why are the changes needed?

Sort-merge join completes directly if the left side table is empty. If there is an aggregate in the right side, the aggregate will not be triggered at all, but its hash map is created during codegen and can't be released.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

a new test

Closes #26471 from cloud-fan/memory.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-13 01:00:30 +08:00
Kent Yao d99398e9f5 [SPARK-29855][SQL] typed literals with negative sign with proper result or exception
### What changes were proposed in this pull request?

```sql
-- !query 83
select -integer '7'
-- !query 83 schema
struct<7:int>
-- !query 83 output
7

-- !query 86
select -date '1999-01-01'
-- !query 86 schema
struct<DATE '1999-01-01':date>
-- !query 86 output
1999-01-01

-- !query 87
select -timestamp '1999-01-01'
-- !query 87 schema
struct<TIMESTAMP('1999-01-01 00:00:00'):timestamp>
-- !query 87 output
1999-01-01 00:00:00
```
the integer should be -7 and the date and timestamp results are confusing which should throw exceptions

### Why are the changes needed?

bug fix
### Does this PR introduce any user-facing change?

NO
### How was this patch tested?

ADD UTs

Closes #26479 from yaooqinn/SPARK-29855.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-12 23:53:07 +09:00
Pablo Langa 37e387a22d [SPARK-29519][SQL] SHOW TBLPROPERTIES should do multi-catalog resolution
### What changes were proposed in this pull request?

Add ShowTablePropertiesStatement and make SHOW TBLPROPERTIES go through the same catalog/table resolution framework of v2 commands.

### Why are the changes needed?

It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.

USE my_catalog
DESC t // success and describe the table t from my_catalog
SHOW TBLPROPERTIES t // report table not found as there is no table t in the session catalog

### Does this PR introduce any user-facing change?

yes. When running SHOW TBLPROPERTIES Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.

### How was this patch tested?

Unit tests.

Closes #26176 from planga82/feature/SPARK-29519_SHOW_TBLPROPERTIES_datasourceV2.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-12 13:31:28 +08:00
Jungtaek Lim (HeartSaVioR) c941362cb9 [SPARK-26154][SS] Streaming left/right outer join should not return outer nulls for already matched rows
### What changes were proposed in this pull request?

This patch fixes the edge case of streaming left/right outer join described below:

Suppose query is provided as

`select * from A join B on A.id = B.id AND (A.ts <= B.ts AND B.ts <= A.ts + interval 5 seconds)`

and there're two rows for L1 (from A) and R1 (from B) which ensures L1.id = R1.id and L1.ts = R1.ts.
(we can simply imagine it from self-join)

Then Spark processes L1 and R1 as below:

- row L1 and row R1 are joined at batch 1
- row R1 is evicted at batch 2 due to join and watermark condition, whereas row L1 is not evicted
- row L1 is evicted at batch 3 due to join and watermark condition

When determining outer rows to match with null, Spark applies some assumption commented in codebase, as below:

```
Checking whether the current row matches a key in the right side state, and that key
has any value which satisfies the filter function when joined. If it doesn't,
we know we can join with null, since there was never (including this batch) a match
within the watermark period. If it does, there must have been a match at some point, so
we know we can't join with null.
```

But as explained the edge-case earlier, the assumption is not correct. As we don't have any good assumption to optimize which doesn't have edge-case, we have to track whether such row is matched with others before, and match with null row only when the row is not matched.

To track the matching of row, the patch adds a new state to streaming join state manager, and mark whether the row is matched to others or not. We leverage the information when dealing with eviction of rows which would be candidates to match with null rows.

This approach introduces new state format which is not compatible with old state format - queries with old state format will be still running but they will still have the issue and be required to discard checkpoint and rerun to take this patch in effect.

### Why are the changes needed?

This patch fixes a correctness issue.

### Does this PR introduce any user-facing change?

No for compatibility viewpoint, but we'll encourage end users to discard the old checkpoint and rerun the query if they run stream-stream outer join query with old checkpoint, which might be "yes" for the question.

### How was this patch tested?

Added UT which fails on current Spark and passes with this patch. Also passed existing streaming join UTs.

Closes #26108 from HeartSaVioR/SPARK-26154-shorten-alternative.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-11 15:47:17 -08:00
Marcelo Vanzin 9753a8e330 [SPARK-29766][SQL] Do metrics aggregation asynchronously in SQL listener
This unblocks the event handling thread, which should help avoid dropped
events when large queries are running.

Existing unit tests should already cover this code.

Closes #26405 from vanzin/SPARK-29766.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-11 14:20:34 -08:00
DB Tsai a6a2748585 [SPARK-29805][SQL] Enable nested schema pruning and nested pruning on expressions by default
### What changes were proposed in this pull request?
Enable nested schema pruning and nested pruning on expressions by default. We have been using those features in production in Apple for couple months with great success. For some jobs, we reduce the data reading by more than 8x and 21x faster in wall clock time.

### Why are the changes needed?
Better performance.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Existing tests.

Closes #26443 from dbtsai/enableNestedSchemaPrunning.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-11-11 19:11:05 +00:00
Takeshi Yamamuro cceb2d6f11 [SPARK-29825][SQL][TESTS] Add join-related configs in inner-join.sql and postgreSQL/join.sql
### What changes were proposed in this pull request?

For better test coverage, this pr is to add join-related configs in `inner-join.sql` and `postgreSQL/join.sql`. These join related configs were just copied from ones in the other join-related tests in `SQLQueryTestSuite` (e.g., https://github.com/apache/spark/blob/master/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql#L2-L4).

### Why are the changes needed?

Better test coverage.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #26459 from maropu/AddJoinConds.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-11 10:21:33 -08:00
Kent Yao d06a9cc4bd [SPARK-29822][SQL] Fix cast error when there are white spaces between signs and values
### What changes were proposed in this pull request?

With the latest string to literal optimization https://github.com/apache/spark/pull/26256, some interval strings can not be cast when there are some spaces between signs and unit values. After state `PARSE_SIGN`, it directly goes to  `PARSE_UNIT_VALUE` when takes a space character as the end. So when there are some white spaces come before the real unit value, it fails to parse, we should add a new state like `TRIM_VALUE` to trim all these spaces.

How to re-produce, which aim the revisions since  https://github.com/apache/spark/pull/26256 is merged

```sql
select cast(v as interval) from values ('+     1 second') t(v);
select cast(v as interval) from values ('-     1 second') t(v);
```

### Why are the changes needed?

bug fix
### Does this PR introduce any user-facing change?

no
### How was this patch tested?

1. ut
2. new benchmark test

Closes #26449 from yaooqinn/SPARK-29605.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-11 21:53:33 +08:00