Commit graph

8731 commits

Author SHA1 Message Date
Maxim Gekk e6b157cf70 [SPARK-29978][SQL][TESTS] Check json_tuple does not truncate results
### What changes were proposed in this pull request?
I propose to add a test from the commit a936522113 for 2.4. I extended the test by a few more lengths of requested field to cover more code branches in Jackson Core. In particular, [the optimization](5eb8973f87/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala (L473-L476)) calls Jackson's method 42b8b56684/src/main/java/com/fasterxml/jackson/core/json/UTF8JsonGenerator.java (L742-L746) where the internal buffer size is **8000**. In this way:
- 2000 to check 2000+2000+2000 < 8000
- 2800 from the 2.4 commit. It covers the specific case: 42b8b56684/src/main/java/com/fasterxml/jackson/core/json/UTF8JsonGenerator.java (L746)
- 8000-1, 8000, 8000+1 are sizes around the size of the internal buffer
- 65535 to test an outstanding large field.

### Why are the changes needed?
To be sure that the current implementation and future versions of Spark don't have the bug fixed in 2.4.

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

### How was this patch tested?
By running `JsonFunctionsSuite`.

Closes #26613 from MaxGekk/json_tuple-test.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-21 09:59:31 +09:00
LantaoJin 06e203b856 [SPARK-29911][SQL] Uncache cached tables when session closed
### What changes were proposed in this pull request?
The local temporary view is session-scoped. Its lifetime is the lifetime of the session that created it.  But now cache data is cross-session. Its lifetime is the lifetime of the Spark application. That's will cause the memory leak if cache a local temporary view in memory when the session closed.
In this PR, we uncache the cached data of local temporary view when session closed. This PR doesn't impact the cached data of global temp view and persisted view.

How to reproduce:
1. create a local temporary view v1
2. cache it in memory
3. close session without drop table v1.

The application will hold the memory forever. In a long running thrift server scenario. It's worse.
```shell
0: jdbc:hive2://localhost:10000> CACHE TABLE testCacheTable AS SELECT 1;
CACHE TABLE testCacheTable AS SELECT 1;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (1.498 seconds)
0: jdbc:hive2://localhost:10000> !close
!close
Closing: 0: jdbc:hive2://localhost:10000
0: jdbc:hive2://localhost:10000 (closed)> !connect 'jdbc:hive2://localhost:10000'
!connect 'jdbc:hive2://localhost:10000'
Connecting to jdbc:hive2://localhost:10000
Enter username for jdbc:hive2://localhost:10000:
lajin
Enter password for jdbc:hive2://localhost:10000:
***
Connected to: Spark SQL (version 3.0.0-SNAPSHOT)
Driver: Hive JDBC (version 1.2.1.spark2)
Transaction isolation: TRANSACTION_REPEATABLE_READ
1: jdbc:hive2://localhost:10000> select * from testCacheTable;
select * from testCacheTable;
Error: Error running query: org.apache.spark.sql.AnalysisException: Table or view not found: testCacheTable; line 1 pos 14;
'Project [*]
+- 'UnresolvedRelation [testCacheTable] (state=,code=0)
```
<img width="1047" alt="Screen Shot 2019-11-15 at 2 03 49 PM" src="https://user-images.githubusercontent.com/1853780/68923527-7ca8c180-07b9-11ea-9cc7-74f276c46840.png">

### Why are the changes needed?
Resolve memory leak for thrift server

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

### How was this patch tested?
Manual test in UI storage tab
And add an UT

Closes #26543 from LantaoJin/SPARK-29911.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-20 18:19:30 -06:00
Sean Owen 1febd373ea [MINOR][TESTS] Replace JVM assert with JUnit Assert in tests
### What changes were proposed in this pull request?

Use JUnit assertions in tests uniformly, not JVM assert() statements.

### Why are the changes needed?

assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything.

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

No. The assertion logic should be identical.

### How was this patch tested?

Existing tests.

Closes #26581 from srowen/assertToJUnit.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-20 14:04:15 -06:00
Yuanjian Li 23b3c4fafd [SPARK-29951][SQL] Make the behavior of Postgre dialect independent of ansi mode config
### What changes were proposed in this pull request?
Fix the inconsistent behavior of build-in function SQL LEFT/RIGHT.

### Why are the changes needed?
As the comment in https://github.com/apache/spark/pull/26497#discussion_r345708065, Postgre dialect should not be affected by the ANSI mode config.
During reran the existing tests, only the LEFT/RIGHT build-in SQL function broke the assumption. We fix this by following https://www.postgresql.org/docs/12/sql-keywords-appendix.html: `LEFT/RIGHT reserved (can be function or type)`

### Does this PR introduce any user-facing change?
Yes, the Postgre dialect will not be affected by the ANSI mode config.

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

Closes #26584 from xuanyuanking/SPARK-29951.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-21 00:56:48 +08:00
Takeshi Yamamuro 6eeb131941 [SPARK-28885][SQL][FOLLOW-UP] Re-enable the ported PgSQL regression tests of SQLQueryTestSuite
### What changes were proposed in this pull request?

SPARK-28885(#26107) has supported the ANSI store assignment rules and stopped running some ported PgSQL regression tests that violate the rules. To re-activate these tests, this pr is to modify them for passing tests with the rules.

### Why are the changes needed?

To make the test coverage better.

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

No.

### How was this patch tested?

Existing tests.

Closes #26492 from maropu/SPARK-28885-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-20 08:32:13 -08:00
Luca Canali b5df40bd87 [SPARK-29894][SQL][WEBUI] Add Codegen Stage Id to Spark plan graphs in Web UI SQL Tab
### What changes were proposed in this pull request?
The Web UI SQL Tab provides information on the executed SQL using plan graphs and by reporting SQL execution plans. Both sources provide useful information. Physical execution plans report Codegen Stage Ids. This PR adds Codegen Stage Ids to the plan graphs.

### Why are the changes needed?
It is useful to have Codegen Stage Id information also reported in plan graphs, this allows to more easily match physical plans and graphs with metrics when troubleshooting SQL execution.
Example snippet to show the proposed change:

![](https://issues.apache.org/jira/secure/attachment/12985837/snippet__plan_graph_with_Codegen_Stage_Id_Annotated.png)

Example of the current state:
![](https://issues.apache.org/jira/secure/attachment/12985838/snippet_plan_graph_before_patch.png)

Physical plan:
![](https://issues.apache.org/jira/secure/attachment/12985932/Physical_plan_Annotated.png)

### Does this PR introduce any user-facing change?
This PR adds Codegen Stage Id information to SQL plan graphs in the Web UI/SQL Tab.

### How was this patch tested?
Added a test + manually tested

Closes #26519 from LucaCanali/addCodegenStageIdtoWEBUIGraphs.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-20 23:20:33 +08:00
Takeshi Yamamuro 0032d85153 [SPARK-29968][SQL] Remove the Predicate code from SparkPlan
### What changes were proposed in this pull request?

This is to refactor Predicate code; it mainly removed `newPredicate` from `SparkPlan`.
Modifications are listed below;
 - Move `Predicate` from `o.a.s.sqlcatalyst.expressions.codegen.GeneratePredicate.scala` to `o.a.s.sqlcatalyst.expressions.predicates.scala`
 - To resolve the name conflict,  rename `o.a.s.sqlcatalyst.expressions.codegen.Predicate` to `o.a.s.sqlcatalyst.expressions.BasePredicate`
 - Extend `CodeGeneratorWithInterpretedFallback ` for `BasePredicate`

This comes from the cloud-fan suggestion: https://github.com/apache/spark/pull/26420#discussion_r348005497

### Why are the changes needed?

For better code/test coverage.

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

No.

### How was this patch tested?

Existing tests.

Closes #26604 from maropu/RefactorPredicate.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-20 21:13:51 +08:00
Nikita Konda 5a70af7a6c [SPARK-29029][SQL] Use AttributeMap in PhysicalOperation.collectProjectsAndFilters
### What changes were proposed in this pull request?

This PR fixes the issue of substituting aliases while collecting filters in  `PhysicalOperation.collectProjectsAndFilters`. When the `AttributeReference` in alias map differs from the `AttributeReference` in filter condition only in qualifier, it does not substitute alias and throws exception saying `key videoid#47L not found` in the following scenario.

```
[1] Project [userid#0]
+- [2] Filter (isnotnull(videoid#47L) && NOT (videoid#47L = 30))
   +- [3] Project [factorial(videoid#1) AS videoid#47L, userid#0]
      +- [4] Filter (isnotnull(avebitrate#2) && (avebitrate#2 < 10))
         +- [5] Relation[userid#0,videoid#1,avebitrate#2]
```

### Why are the changes needed?

We need to use `AttributeMap` where the key is `AttributeReference`'s `ExprId` instead of `Map[Attribute, Expression]` while collecting and substituting aliases in `PhysicalOperation.collectProjectsAndFilters`.

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

### How was this patch tested?
New unit tests were added in `TestPhysicalOperation` which reproduces the bug

Closes #25761 from nikitagkonda/SPARK-29029-use-attributemap-for-aliasmap-in-physicaloperation.

Authored-by: Nikita Konda <nikita.konda@workday.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-19 20:01:42 -08:00
Wenchen Fan 9e58b10c8e [SPARK-29945][SQL] do not handle negative sign specially in the parser
### What changes were proposed in this pull request?

Remove the special handling of the negative sign in the parser (interval literal and type constructor)

### Why are the changes needed?

The negative sign is an operator (UnaryMinus). We don't need to handle it specially, which is kind of doing constant folding at parser side.

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

The error message becomes a little different. Now it reports type mismatch for the `-` operator.

### How was this patch tested?

existing tests

Closes #26578 from cloud-fan/interval.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-11-20 11:08:04 +09:00
Maxim Gekk 40b8a08b8b [SPARK-29963][SQL][TESTS] Check formatting 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 47cb1f359a for Spark 2.4 that check formatting 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 #26601 from MaxGekk/format-timestamp-micros-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-20 10:34:25 +09:00
Wenchen Fan 3d2a6f464f [SPARK-29906][SQL] AQE should not introduce extra shuffle for outermost limit
### What changes were proposed in this pull request?

`AdaptiveSparkPlanExec` should forward `executeCollect` and `executeTake` to the underlying physical plan.

### Why are the changes needed?

some physical plan has optimization in `executeCollect` and `executeTake`. For example, `CollectLimitExec` won't do shuffle for outermost limit.

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

no

### How was this patch tested?

a new test

This closes #26560

Closes #26576 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-11-19 10:39:38 -08:00
Jobit Mathew 6fb8b86065 [SPARK-29913][SQL] Improve Exception in postgreCastToBoolean
### What changes were proposed in this pull request?
Exception improvement.

### Why are the changes needed?
After selecting pgSQL dialect, queries which are failing because of wrong syntax will give long exception stack trace. For example,
`explain select cast ("abc" as boolean);`

Current output:

> ERROR SparkSQLDriver: Failed in [explain select cast ("abc" as boolean)]
> java.lang.IllegalArgumentException: invalid input syntax for type boolean: abc
> 	at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$2(PostgreCastToBoolean.scala:51)
> 	at org.apache.spark.sql.catalyst.expressions.CastBase.buildCast(Cast.scala:277)
> 	at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$1(PostgreCastToBoolean.scala:44)
> 	at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeEval(Cast.scala:773)
> 	at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:460)
> 	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52)
> 	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:286)
> 	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:286)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:291)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
> 	at org.apache.spark.sql.catalyst.plans.QueryPlan.
>       .
>       .
>       .

### Does this PR introduce any user-facing change?
Yes. After this PR, output for above query will be:

> == Physical Plan ==
> org.apache.spark.sql.AnalysisException: invalid input syntax for type boolean: abc;
>
> Time taken: 0.044 seconds, Fetched 1 row(s)
> 19/11/15 15:38:57 INFO SparkSQLCLIDriver: Time taken: 0.044 seconds, Fetched 1 row(s)

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

Closes #26546 from jobitmathew/pgsqlexception.

Authored-by: Jobit Mathew <jobit.mathew@huawei.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 21:30:38 +08:00
Kent Yao 79ed4ae2db [SPARK-29926][SQL] Fix weird interval string whose value is only a dangling decimal point
### What changes were proposed in this pull request?

Currently, we support to parse '1. second' to 1s or even '. second' to 0s.

```sql
-- !query 118
select interval '1. seconds'
-- !query 118 schema
struct<1 seconds:interval>
-- !query 118 output
1 seconds

-- !query 119
select interval '. seconds'
-- !query 119 schema
struct<0 seconds:interval>
-- !query 119 output
0 seconds
```

```sql
postgres=# select interval '1. second';
ERROR:  invalid input syntax for type interval: "1. second"
LINE 1: select interval '1. second';

postgres=# select interval '. second';
ERROR:  invalid input syntax for type interval: ". second"
LINE 1: select interval '. second';
```
We fix this by fixing the new interval parser's VALUE_FRACTIONAL_PART state

With further digging, we found that 1. is valid in python, r, scala, and presto and so on... so this PR
ONLY forbid the invalid interval value in the form of  '. seconds'.

### Why are the changes needed?

bug fix

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

yes, now we treat '. second' .... as invalid intervals
### How was this patch tested?

add ut

Closes #26573 from yaooqinn/SPARK-29926.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 21:01:26 +08:00
jiake a8d98833b8 [SPARK-29893] improve the local shuffle reader performance by changing the reading task number from 1 to multi
### What changes were proposed in this pull request?
This PR update the local reader task number from 1 to multi `partitionStartIndices.length`.

### Why are the changes needed?
Improve the performance of local shuffle reader.

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

### How was this patch tested?
Existing UTs

Closes #26516 from JkSelf/improveLocalShuffleReader.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 19:18:08 +08:00
wangguangxin.cn ffc9753037 [SPARK-29918][SQL] RecordBinaryComparator should check endianness when compared by long
### What changes were proposed in this pull request?
This PR try to make sure the comparison results of  `compared by 8 bytes at a time` and `compared by bytes wise` in RecordBinaryComparator is *consistent*, by reverse long bytes if it is little-endian and using Long.compareUnsigned.

### Why are the changes needed?
If the architecture supports unaligned or the offset is 8 bytes aligned, `RecordBinaryComparator` compare 8 bytes at a time by reading 8 bytes as a long.  Related code is
```
    if (Platform.unaligned() || (((leftOff + i) % 8 == 0) && ((rightOff + i) % 8 == 0))) {
      while (i <= leftLen - 8) {
        final long v1 = Platform.getLong(leftObj, leftOff + i);
        final long v2 = Platform.getLong(rightObj, rightOff + i);
        if (v1 != v2) {
          return v1 > v2 ? 1 : -1;
        }
        i += 8;
      }
    }
```

Otherwise, it will compare bytes by bytes.  Related code is
```
    while (i < leftLen) {
      final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff;
      final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff;
      if (v1 != v2) {
        return v1 > v2 ? 1 : -1;
      }
      i += 1;
    }
```

However, on little-endian machine,  the result of *compared by a long value* and *compared bytes by bytes* maybe different.

For two same records, its offsets may vary in the first run and second run, which will lead to compare them using long comparison or byte-by-byte comparison, the result maybe different.

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

### How was this patch tested?
Add new test cases in RecordBinaryComparatorSuite

Closes #26548 from WangGuangxin/binary_comparator.

Authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 16:10:22 +08:00
Wenchen Fan 16134d6d0f [SPARK-29948][SQL] make the default alias consistent between date, timestamp and interval
### What changes were proposed in this pull request?

Update `Literal.sql` to make date, timestamp and interval consistent. They should all use the `TYPE 'value'` format.

### Why are the changes needed?

Make the default alias consistent. For example, without this patch we will see
```
scala> sql("select interval '1 day', date '2000-10-10'").show
+------+-----------------+
|1 days|DATE '2000-10-10'|
+------+-----------------+
|1 days|       2000-10-10|
+------+-----------------+
```

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

no

### How was this patch tested?

existing tests

Closes #26579 from cloud-fan/sql.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 15:37:35 +08:00
LantaoJin 5ac37a8265 [SPARK-29869][SQL] improve error message in HiveMetastoreCatalog#convertToLogicalRelation
### What changes were proposed in this pull request?
In our production, HiveMetastoreCatalog#convertToLogicalRelation throws AssertError occasionally:
```sql
scala> spark.table("hive_table").show
java.lang.AssertionError: assertion failed
  at scala.Predef$.assert(Predef.scala:208)
  at org.apache.spark.sql.hive.HiveMetastoreCatalog.convertToLogicalRelation(HiveMetastoreCatalog.scala:261)
  at org.apache.spark.sql.hive.HiveMetastoreCatalog.convert(HiveMetastoreCatalog.scala:137)
  at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:220)
  at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:207)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$2(AnalysisHelper.scala:108)
  at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$1(AnalysisHelper.scala:108)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown(AnalysisHelper.scala:106)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown$(AnalysisHelper.scala:104)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$4(AnalysisHelper.scala:113)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$1(AnalysisHelper.scala:113)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown(AnalysisHelper.scala:106)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown$(AnalysisHelper.scala:104)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators(AnalysisHelper.scala:73)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators$(AnalysisHelper.scala:72)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:29)
  at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:207)
  at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:191)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:130)
  at scala.collection.IndexedSeqOptimized.foldLeft(IndexedSeqOptimized.scala:60)
  at scala.collection.IndexedSeqOptimized.foldLeft$(IndexedSeqOptimized.scala:68)
  at scala.collection.mutable.ArrayBuffer.foldLeft(ArrayBuffer.scala:49)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:127)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:119)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:119)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:168)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:162)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:122)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:98)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:98)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:146)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:201)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:145)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:66)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:63)
  at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:63)
  at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:55)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:86)
  at org.apache.spark.sql.SparkSession.table(SparkSession.scala:585)
  at org.apache.spark.sql.SparkSession.table(SparkSession.scala:581)
  ... 47 elided
````
Most of cases occurred in reading a table which created by an old Spark version.
After recreated the table, the issue will be gone.

After deep dive, the root cause is this external table is a non-partitioned table but the `LOCATION` set to a partitioned path {{/tablename/dt=yyyymmdd}}. The partitionSpec is inferred.

### Why are the changes needed?
Above error message is very confused. We need more details about assert failure information.

This issue caused by `PartitioningAwareFileIndex#inferPartitioning()`. For non-HiveMetastore Spark, it's useful. But for Hive table, it shouldn't infer partition if Hive tell us it's a non partitioned table. (new added)

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

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

Closes #26499 from LantaoJin/SPARK-29869.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 15:22:08 +08:00
Terry Kim 3d45779b68 [SPARK-29728][SQL] Datasource V2: Support ALTER TABLE RENAME TO
### What changes were proposed in this pull request?

This PR adds `ALTER TABLE a.b.c RENAME TO x.y.x` support for V2 catalogs.

### Why are the changes needed?

The current implementation doesn't support this command V2 catalogs.

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

Yes, now the renaming table works for v2 catalogs:
```
scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show
+---------+---------+
|namespace|tableName|
+---------+---------+
|  ns1.ns2|      old|
+---------+---------+

scala> spark.sql("ALTER TABLE testcat.ns1.ns2.old RENAME TO testcat.ns1.ns2.new").show

scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show
+---------+---------+
|namespace|tableName|
+---------+---------+
|  ns1.ns2|      new|
+---------+---------+
```
### How was this patch tested?

Added unit tests.

Closes #26539 from imback82/rename_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 12:03:29 +08:00
shivsood a834dba120 Revert "[SPARK-29644][SQL] Corrected ShortType and ByteType mapping to SmallInt and TinyInt in JDBCUtils
This reverts commit f7e53865 i.e PR #26301 from master

Closes #26583 from shivsood/revert_29644_master.

Authored-by: shivsood <shivsood@microsoft.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-18 18:44:16 -08:00
Yuming Wang 28a502c6e9 [SPARK-28527][FOLLOW-UP][SQL][TEST] Add guides for ThriftServerQueryTestSuite
### What changes were proposed in this pull request?
This PR add guides for `ThriftServerQueryTestSuite`.

### Why are the changes needed?
Add guides

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

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

Closes #26587 from wangyum/SPARK-28527-FOLLOW-UP.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-18 18:13:11 -08:00
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
lajin 4de7131cff [SPARK-29421][SQL] Supporting Create Table Like Using Provider
### What changes were proposed in this pull request?
Hive support STORED AS new file format syntax:
```sql
CREATE TABLE tbl(a int) STORED AS TEXTFILE;
CREATE TABLE tbl2 LIKE tbl STORED AS PARQUET;
```
We add a similar syntax for Spark. Here we separate to two features:

1. specify a different table provider in CREATE TABLE LIKE
2. Hive compatibility

In this PR, we address the first one:
- [ ] Using `USING provider` to specify a different table provider in CREATE TABLE LIKE.
- [  ] Using `STORED AS file_format` in CREATE TABLE LIKE to address Hive compatibility.

### Why are the changes needed?
Use CREATE TABLE tb1 LIKE tb2 command to create an empty table tb1 based on the definition of table tb2. The most user case is to create tb1 with the same schema of tb2. But an inconvenient case here is this command also copies the FileFormat from tb2, it cannot change the input/output format and serde. Add the ability of changing file format is useful for some scenarios like upgrading a table from a low performance file format to a high performance one (parquet, orc).

### Does this PR introduce any user-facing change?
Add a new syntax based on current CTL:
```sql
CREATE TABLE tbl2 LIKE tbl [USING parquet];
```

### How was this patch tested?
Modify some exist UTs.

Closes #26097 from LantaoJin/SPARK-29421.

Authored-by: lajin <lajin@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-11 15:25:56 +08:00
Maxim Gekk 18440151b0 [SPARK-29393][SQL] Add make_interval function
### What changes were proposed in this pull request?
In the PR, I propose new expression `MakeInterval` and register it as the function `make_interval`. The function accepts the following parameters:
- `years` - the number of years in the interval, positive or negative. The parameter is multiplied by 12, and added to interval's `months`.
- `months` - the number of months in the interval, positive or negative.
- `weeks` - the number of months in the interval, positive or negative. The parameter is multiplied by 7, and added to interval's `days`.
- `hours`, `mins` - the number of hours and minutes. The parameters can be negative or positive. They are converted to microseconds and added to interval's `microseconds`.
- `seconds` - the number of seconds with the fractional part in microseconds precision. It is converted to microseconds, and added to total interval's `microseconds` as `hours` and `minutes`.

For example:
```sql
spark-sql> select make_interval(2019, 11, 1, 1, 12, 30, 01.001001);
2019 years 11 months 8 days 12 hours 30 minutes 1.001001 seconds
```

### Why are the changes needed?
- To improve user experience with Spark SQL, and allow users making `INTERVAL` columns from other columns containing `years`, `months` ... `seconds`. Currently, users can make an `INTERVAL` column from other columns only by constructing a `STRING` column and cast it to `INTERVAL`. Have a look at the `IntervalBenchmark` as an example.
- To maintain feature parity with PostgreSQL which provides such function:
```sql
# SELECT make_interval(2019, 11);
   make_interval
--------------------
 2019 years 11 mons
```

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

### How was this patch tested?
- By new tests for the `MakeInterval` expression to `IntervalExpressionsSuite`
- By tests in `interval.sql`

Closes #26446 from MaxGekk/make_interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-10 14:34:52 -08:00
Pavithra Ramachandran e2ca7f396f [SPARK-29601][WEBUI] JDBC ODBC Tab Statement column provide ellipsis for big SQL statement
### What changes were proposed in this pull request?
Provide Ellipses in Statement column , just like description in Jobs page .

### Why are the changes needed?
When a query is executed the whole query statement is displayed no matter how big it is. When bigger queries are executed, it covers a large portion of the page display, when we have multiple queries it is difficult to scroll down to view all.

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

Before:
![Screenshot from 2019-11-01 23-15-23](https://user-images.githubusercontent.com/51401130/68064468-ebaa0300-fd41-11e9-8787-c5144c1468d4.png)

After:
![Screenshot from 2019-11-02 07-07-21](https://user-images.githubusercontent.com/51401130/68064471-f19fe400-fd41-11e9-85c6-65f0faa64cc3.png)

### How was this patch tested?
Manual

Closes #26364 from PavithraRamachandran/ellipse_JDBC.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-10 13:08:26 -06:00
Maxim Gekk d4de01f567 [SPARK-29408][SQL] Support - before interval in interval literals
### What changes were proposed in this pull request?
- `SqlBase.g4` is modified to support a negative sign `-` in the interval type constructor from a string and in interval literals
- Negate interval in `AstBuilder` if a sign presents.
- Interval related SQL statements are moved from `inputs/datetime.sql` to new file `inputs/interval.sql`

For example:
```sql
spark-sql> select -interval '-1 month 1 day -1 second';
1 months -1 days 1 seconds
spark-sql> select -interval -1 month 1 day -1 second;
1 months -1 days 1 seconds
```

### Why are the changes needed?
For feature parity with PostgreSQL which supports that:
```sql
# select -interval '-1 month 1 day -1 second';
        ?column?
-------------------------
 1 mon -1 days +00:00:01
(1 row)
```

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

### How was this patch tested?
- Added tests to `ExpressionParserSuite`
- by `interval.sql`

Closes #26438 from MaxGekk/negative-interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-10 10:10:04 -08:00
Maxim Gekk 7ddcb5b46d [SPARK-29819][SQL] Introduce an enum for interval units
### What changes were proposed in this pull request?
In the PR, I propose an enumeration for interval units with the value `YEAR`, `MONTH`, `WEEK`, `DAY`, `HOUR`, `MINUTE`, `SECOND`, `MILLISECOND`, `MICROSECOND` and `NANOSECOND`.

### Why are the changes needed?
- This should prevent typos in interval unit names
- Stronger type checking of unit parameters.

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

### How was this patch tested?
By existing test suites `ExpressionParserSuite` and `IntervalUtilsSuite`

Closes #26455 from MaxGekk/interval-unit-enum.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-10 08:41:55 -08:00
Huaxin Gao 57b954e825 [SPARK-29730][SQL] ALTER VIEW QUERY should look up catalog/table like v2 commands
Add AlterViewAsStatement and make ALTER VIEW ... QUERY go through the same catalog/table resolution framework of v2 commands.

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 v // success and describe the view v from my_catalog
ALTER VIEW v SELECT 1 // report view not found as there is no view v in the session catalog
```

Yes. When running ALTER VIEW ... QUERY, Spark fails the command if the current catalog is set to a v2 catalog, or the view name specified a v2 catalog.

unit tests

Closes #26453 from huaxingao/spark-29730.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-09 17:06:09 -08:00
Xiao Li 1e2d76e80a [HOT-FIX] Fix the SQLBase.g4
### What changes were proposed in this pull request?
Remove the duplicate code

See the build failure: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-3.2/986/

### Why are the changes needed?
Fix the compilation

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

### How was this patch tested?
The existing tests

Closes #26445 from gatorsmile/hotfixPraser.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-11-08 22:39:07 -08:00
xy_xin 7cfd589868 [SPARK-28893][SQL] Support MERGE INTO in the parser and add the corresponding logical plan
### What changes were proposed in this pull request?
This PR supports MERGE INTO in the parser and add the corresponding logical plan. The SQL syntax likes,
```
MERGE INTO [ds_catalog.][multi_part_namespaces.]target_table [AS target_alias]
USING [ds_catalog.][multi_part_namespaces.]source_table | subquery [AS source_alias]
ON <merge_condition>
[ WHEN MATCHED [ AND <condition> ] THEN <matched_action> ]
[ WHEN MATCHED [ AND <condition> ] THEN <matched_action> ]
[ WHEN NOT MATCHED [ AND <condition> ]  THEN <not_matched_action> ]
```
where
```
<matched_action>  =
  DELETE  |
  UPDATE SET *  |
  UPDATE SET column1 = value1 [, column2 = value2 ...]

<not_matched_action>  =
  INSERT *  |
  INSERT (column1 [, column2 ...]) VALUES (value1 [, value2 ...])
```

### Why are the changes needed?
This is a start work for introduce `MERGE INTO` support for the builtin datasource, and the design work for the `MERGE INTO` support in DSV2.

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

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

Closes #26167 from xianyinxin/SPARK-28893.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-09 11:45:24 +08:00
Liang-Chi Hsieh 70987d8144 [SPARK-29680][SQL][FOLLOWUP] Replace qualifiedName with multipartIdentifier
### What changes were proposed in this pull request?

Replace qualifiedName with multipartIdentifier in parser rules of DDL commands.

### Why are the changes needed?

There are identifiers in some DDL rules we use `qualifiedName`. We should use `multipartIdentifier` because it can capture wrong identifiers such as `test-table`, `test-col`.

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

Yes. Wrong identifiers such as test-table, will be captured now after this change.

### How was this patch tested?

Unit tests.

Closes #26419 from viirya/SPARK-29680-followup2.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2019-11-08 14:18:06 -08:00
Kent Yao e026412d9c [SPARK-29679][SQL] Make interval type comparable and orderable
### What changes were proposed in this pull request?

interval type support >, >=, <, <=, =, <=>, order by, min,max..

### Why are the changes needed?

Part of SPARK-27764 Feature Parity between PostgreSQL and Spark
### Does this PR introduce any user-facing change?

yes, we now support compare intervals

### How was this patch tested?

add ut

Closes #26337 from yaooqinn/SPARK-29679.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-08 22:45:11 +08:00
Kent Yao e7f7990bc3 [SPARK-29688][SQL] Support average for interval type values
### What changes were proposed in this pull request?

avg aggregate support interval type values

### Why are the changes needed?

Part of SPARK-27764 Feature Parity between PostgreSQL and Spark

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

yes, we can do avg on intervals

### How was this patch tested?

add ut

Closes #26347 from yaooqinn/SPARK-29688.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-08 21:55:07 +08:00
davidvrba afc943ff8a [SPARK-28477][SQL] Rewrite CaseWhen with single branch to If
### What changes were proposed in this pull request?
Spark org.apache.spark.sql.functions do not have `if` function so conditions are expressed using `when-otherwise` function. However `If` (which is available in SQL) has more efficient code gen. This pr rewrites `when-otherwise` conditions to `If` if it is possible (`when-otherwise` with single branch)

### Why are the changes needed?
It is an optimization enhancement. Here is a simple performance comparison (tested in local mode (with 4 cores)):
```
val df = spark.range(10000000000L).withColumn("x", rand)
val resultA = df.withColumn("r", when($"x" < 0.5, lit(1)).otherwise(lit(0))).agg(sum($"r"))
val resultB = df.withColumn("r", expr("if(x < 0.5, 1, 0)")).agg(sum($"r"))

resultA.collect() // takes 56s to finish
resultB.collect() // takes 30s to finish
```
### Does this PR introduce any user-facing change?
No

### How was this patch tested?
New test is added.

Closes #26294 from davidvrba/spark-28477_rewriteCaseWhenToIf.

Authored-by: davidvrba <vrba.dave@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-08 21:25:48 +08:00
ulysses 7759f7179c [SPARK-29772][TESTS][SQL] Add withNamespace in SQLTestUtils
### What changes were proposed in this pull request?

V2 catalog support namespace, we should add `withNamespace` like `withDatabase`.

### Why are the changes needed?

Make test easy.

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

No.

### How was this patch tested?

Add UT.

Closes #26411 from ulysses-you/Add-test-with-namespace.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-08 11:53:44 +08:00
Kent Yao 0a03839366 [SPARK-29787][SQL] Move methods add/subtract/negate from CalendarInterval to IntervalUtils
### What changes were proposed in this pull request?

Move method add/subtract/negate from CalendarInterval to IntervalUtils

### Why are the changes needed?

https://github.com/apache/spark/pull/26410#discussion_r343125468 suggested here
### Does this PR introduce any user-facing change?

no
### How was this patch tested?

add uts and move some

Closes #26423 from yaooqinn/SPARK-29787.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-08 10:28:58 +08:00
Dongjoon Hyun da848b1897 [SPARK-29796][SQL][TESTS] HiveExternalCatalogVersionsSuite should ignore preview release
### What changes were proposed in this pull request?

This aims to exclude the `preview` release to recover `HiveExternalCatalogVersionsSuite`. Currently, new preview release breaks `branch-2.4` PRBuilder since yesterday. New release (especially `preview`) should not affect `branch-2.4`.
- https://github.com/apache/spark/pull/26417 (Failed 4 times)

### Why are the changes needed?

**BEFORE**
```scala
scala> scala.io.Source.fromURL("https://dist.apache.org/repos/dist/release/spark/").mkString.split("\n").filter(_.contains("""<li><a href="spark-""")).map("""<a href="spark-(\d.\d.\d)/">""".r.findFirstMatchIn(_).get.group(1))
java.util.NoSuchElementException: None.get
```

**AFTER**
```scala
scala> scala.io.Source.fromURL("https://dist.apache.org/repos/dist/release/spark/").mkString.split("\n").filter(_.contains("""<li><a href="spark-""")).filterNot(_.contains("preview")).map("""<a href="spark-(\d.\d.\d)/">""".r.findFirstMatchIn(_).get.group(1))
res5: Array[String] = Array(2.3.4, 2.4.4)
```

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

No.

### How was this patch tested?

This should pass the PRBuilder.

Closes #26428 from dongjoon-hyun/SPARK-HiveExternalCatalogVersionsSuite.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-07 10:28:32 -08:00
Kent Yao 9562b26914 [SPARK-29757][SQL] Move calendar interval constants together
### What changes were proposed in this pull request?
```java
  public static final int YEARS_PER_DECADE = 10;
  public static final int YEARS_PER_CENTURY = 100;
  public static final int YEARS_PER_MILLENNIUM = 1000;

  public static final byte MONTHS_PER_QUARTER = 3;
  public static final int MONTHS_PER_YEAR = 12;

  public static final byte DAYS_PER_WEEK = 7;
  public static final long DAYS_PER_MONTH = 30L;

  public static final long HOURS_PER_DAY = 24L;

  public static final long MINUTES_PER_HOUR = 60L;

  public static final long SECONDS_PER_MINUTE = 60L;
  public static final long SECONDS_PER_HOUR = MINUTES_PER_HOUR * SECONDS_PER_MINUTE;
  public static final long SECONDS_PER_DAY = HOURS_PER_DAY * SECONDS_PER_HOUR;

  public static final long MILLIS_PER_SECOND = 1000L;
  public static final long MILLIS_PER_MINUTE = SECONDS_PER_MINUTE * MILLIS_PER_SECOND;
  public static final long MILLIS_PER_HOUR = MINUTES_PER_HOUR * MILLIS_PER_MINUTE;
  public static final long MILLIS_PER_DAY = HOURS_PER_DAY * MILLIS_PER_HOUR;

  public static final long MICROS_PER_MILLIS = 1000L;
  public static final long MICROS_PER_SECOND = MILLIS_PER_SECOND * MICROS_PER_MILLIS;
  public static final long MICROS_PER_MINUTE = SECONDS_PER_MINUTE * MICROS_PER_SECOND;
  public static final long MICROS_PER_HOUR = MINUTES_PER_HOUR * MICROS_PER_MINUTE;
  public static final long MICROS_PER_DAY = HOURS_PER_DAY * MICROS_PER_HOUR;
  public static final long MICROS_PER_MONTH = DAYS_PER_MONTH * MICROS_PER_DAY;
  /* 365.25 days per year assumes leap year every four years */
  public static final long MICROS_PER_YEAR = (36525L * MICROS_PER_DAY) / 100;

  public static final long NANOS_PER_MICROS = 1000L;
  public static final long NANOS_PER_MILLIS = MICROS_PER_MILLIS * NANOS_PER_MICROS;
  public static final long NANOS_PER_SECOND = MILLIS_PER_SECOND * NANOS_PER_MILLIS;
```
The above parameters are defined in IntervalUtils, DateTimeUtils, and CalendarInterval, some of them are redundant, some of them are cross-referenced.

### Why are the changes needed?
To simplify code, enhance consistency and reduce risks

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

no
### How was this patch tested?

modified uts

Closes #26399 from yaooqinn/SPARK-29757.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 19:48:19 +08:00
Wenchen Fan 9b61f90987 [SPARK-29761][SQL] do not output leading 'interval' in CalendarInterval.toString
### What changes were proposed in this pull request?

remove the leading "interval" in `CalendarInterval.toString`.

### Why are the changes needed?

Although it's allowed to have "interval" prefix when casting string to int, it's not recommended.

This is also consistent with pgsql:
```
cloud0fan=# select interval '1' day;
 interval
----------
 1 day
(1 row)
```

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

yes, when display a dataframe with interval type column, the result is different.

### How was this patch tested?

updated tests.

Closes #26401 from cloud-fan/interval.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 15:44:50 +08:00
Maxim Gekk 29dc59ac29 [SPARK-29605][SQL] Optimize string to interval casting
### What changes were proposed in this pull request?
In the PR, I propose new function `stringToInterval()` in `IntervalUtils` for converting `UTF8String` to `CalendarInterval`. The function is used in casting a `STRING` column to an `INTERVAL` column.

### Why are the changes needed?
The proposed implementation is ~10 times faster. For example, parsing 9 interval units on JDK 8:
Before:
```
9 units w/ interval                               14004          14125         116          0.1       14003.6       0.0X
9 units w/o interval                              13785          14056         290          0.1       13784.9       0.0X
```
After:
```
9 units w/ interval                                1343           1344           1          0.7        1343.0       0.3X
9 units w/o interval                               1345           1349           8          0.7        1344.6       0.3X
```

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

### How was this patch tested?
- By new tests for `stringToInterval` in `IntervalUtilsSuite`
- By existing tests

Closes #26256 from MaxGekk/string-to-interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 12:39:52 +08:00
Kent Yao 3437862975 [SPARK-29387][SQL][FOLLOWUP] Fix issues of the multiply and divide for intervals
### What changes were proposed in this pull request?

Handle the inconsistence dividing zeros between literals and columns.
fix the null issue too.

### Why are the changes needed?
BUG FIX
### 1 Handle the inconsistence dividing zeros between literals and columns
```sql
-- !query 24
select
    k,
    v,
    cast(k as interval) / v,
    cast(k as interval) * v
from VALUES
     ('1 seconds', 1),
     ('2 seconds', 0),
     ('3 seconds', null),
     (null, null),
     (null, 0) t(k, v)
-- !query 24 schema
struct<k:string,v:int,divide_interval(CAST(k AS INTERVAL), CAST(v AS DOUBLE)):interval,multiply_interval(CAST(k AS INTERVAL), CAST(v AS DOUBLE)):interval>
-- !query 24 output
1 seconds   1   interval 1 seconds  interval 1 seconds
2 seconds   0   interval 0 microseconds interval 0 microseconds
3 seconds   NULL    NULL    NULL
NULL    0   NULL    NULL
NULL    NULL    NULL    NULL
```
```sql
-- !query 21
select interval '1 year 2 month' / 0
-- !query 21 schema
struct<divide_interval(interval 1 years 2 months, CAST(0 AS DOUBLE)):interval>
-- !query 21 output
NULL
```

in the first case, interval ’2 seconds ‘ / 0, it produces `interval 0 microseconds `
in the second case, it is `null`

### 2 null literal issues

```sql

  -- !query 20
select interval '1 year 2 month' / null
-- !query 20 schema
struct<>
-- !query 20 output
org.apache.spark.sql.AnalysisException
cannot resolve '(interval 1 years 2 months / NULL)' due to data type mismatch: differing types in '(interval 1 years 2 months / NULL)' (interval and null).; line 1 pos 7

-- !query 22
select interval '4 months 2 weeks 6 days' * null
-- !query 22 schema
struct<>
-- !query 22 output
org.apache.spark.sql.AnalysisException
cannot resolve '(interval 4 months 20 days * NULL)' due to data type mismatch: differing types in '(interval 4 months 20 days * NULL)' (interval and null).; line 1 pos 7

-- !query 23
select null * interval '4 months 2 weeks 6 days'
-- !query 23 schema
struct<>
-- !query 23 output
org.apache.spark.sql.AnalysisException
cannot resolve '(NULL * interval 4 months 20 days)' due to data type mismatch: differing types in '(NULL * interval 4 months 20 days)' (null and interval).; line 1 pos 7
```
 dividing or multiplying null literals, error occurs; where in column is fine as the first case
### Does this PR introduce any user-facing change?

NO, maybe yes, but it is just a follow-up

### How was this patch tested?

add uts

cc cloud-fan MaxGekk maropu

Closes #26410 from yaooqinn/SPARK-29387.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 12:19:03 +08:00
Wenchen Fan 1f3863c856 [SPARK-29759][SQL] LocalShuffleReaderExec.outputPartitioning should use the corrected attributes
### What changes were proposed in this pull request?

Update `LocalShuffleReaderExec.outputPartitioning` to use attributes from `ReusedQueryStage`.

This also removes the override `doCanonicalize` in local/coalesced shuffle reader, as these 2 operators change the output partitioning. It's not safe to strip them in the canonicalized query plan.

### Why are the changes needed?

We will have an invalid output partitioning if we don fix it.

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

no

### How was this patch tested?

existing tests

Closes #26400 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-11-06 14:33:52 -08:00
Jungtaek Lim (HeartSaVioR) 782992c7ed [SPARK-29642][SS] Change the element type of underlying array to UnsafeRow for ContinuousRecordEndpoint
### What changes were proposed in this pull request?

This patch fixes the bug that `ContinuousMemoryStream[String]` throws error regarding ClassCastException - cast String to UTFString. This is because ContinuousMemoryStream and ContinuousRecordEndpoint uses origin input as it is for underlying data structure of Row, and encoding is missing here.

To force encoding, this patch changes the element type of underlying array to UnsafeRow instead of Any for ContinuousRecordEndpoint - ContinuousMemoryStream and TextSocketContinuousStream are modified to reflect the change.

### Why are the changes needed?

Above section describes the bug.

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

No.

### How was this patch tested?

Add new UT to check for availability on couple of types.

Closes #26300 from HeartSaVioR/SPARK-29642.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-06 10:37:00 -08:00
Wenchen Fan 411015300e [SPARK-29752][SQL][TEST] make AdaptiveQueryExecSuite more robust
### What changes were proposed in this pull request?

instead of checking the exact number of local shuffle readers, we should check whether the number of shuffles is equal to the number of local readers.

### Why are the changes needed?

AQE is known to have randomness. We may pick different build side for broadcast join depending on which query stage finishes first. The decision to build side may add/remove shuffles downstream, so it's flaky to check the exact number of local shuffle readers.

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

no

### How was this patch tested?

test only PR.

Closes #26394 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-11-06 10:27:39 -08:00
shahid 90df858a26 [SPARK-29725][SQL][TESTS] Add ThriftServerPageSuite
### What changes were proposed in this pull request?
Added UT for the classes `ThriftServerPage.scala` and `ThriftServerSessionPage.scala`

### Why are the changes needed?

Currently, there are no UTs for testing Thriftserver UI page
### Does this PR introduce any user-facing change?

No

### How was this patch tested?

UT

Closes #26403 from shahidki31/ut.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-06 20:59:45 +09:00
Aman Omer 0dcd739534 [SPARK-29462] The data type of "array()" should be array<null>
### What changes were proposed in this pull request?
During creation of array, if CreateArray does not gets any children to set data type for array, it will create an array of null type .

### Why are the changes needed?
When empty array is created, it should be declared as array<null>.

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

### How was this patch tested?
Tested manually

Closes #26324 from amanomer/29462.

Authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-06 18:39:46 +09:00
Liang-Chi Hsieh 6233958ab6 [SPARK-29680][SQL] Remove ALTER TABLE CHANGE COLUMN syntax
### What changes were proposed in this pull request?

This patch removes v1 ALTER TABLE CHANGE COLUMN syntax.

### Why are the changes needed?

Since in v2 we have ALTER TABLE CHANGE COLUMN and ALTER TABLE RENAME COLUMN, this old syntax is not necessary now and can be confusing.

The v2 ALTER TABLE CHANGE COLUMN should fallback to v1 AlterTableChangeColumnCommand (#26354).

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

Yes, the old v1 ALTER TABLE CHANGE COLUMN syntax is removed.

### How was this patch tested?

Unit tests.

Closes #26338 from viirya/SPARK-29680.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-06 10:42:44 +08:00
Takeshi Yamamuro 20b9d8259b [SPARK-29714][SQL][TESTS] Port insert.sql
### What changes were proposed in this pull request?

This PR ports insert.sql from PostgreSQL regression tests https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/insert.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/expected/insert.out

### Why are the changes needed?

To check behaviour differences between Spark and PostgreSQL

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

No

### How was this patch tested?

Pass the Jenkins. And, Comparison with PgSQL results

Closes #26360 from maropu/InsertTest.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-05 16:44:54 -08:00
Maxim Gekk 4c53ac1822 [SPARK-29387][SQL] Support * and / operators for intervals
### What changes were proposed in this pull request?
Added new expressions `MultiplyInterval` and `DivideInterval` to multiply/divide an interval by a numeric. Updated `TypeCoercion.DateTimeOperations` to turn the `Multiply`/`Divide` expressions of `CalendarIntervalType` and `NumericType` to `MultiplyInterval`/`DivideInterval`.

To support new operations, added new methods `multiply()` and `divide()` to `CalendarInterval`.

### Why are the changes needed?
- To maintain feature parity with PostgreSQL which supports multiplication and division of intervals by doubles:
```sql
# select interval '1 hour' / double precision '1.5';
 ?column?
----------
 00:40:00
```
- To conform the SQL standard which defines those operations: `numeric * interval`, `interval * numeric` and `interval / numeric`. See [4.5.3  Operations involving datetimes and intervals](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt).
- Improve Spark SQL UX and allow users to adjust interval columns. For example:
```sql
spark-sql> select (timestamp'now' - timestamp'yesterday') * 1.3;
interval 2 days 10 hours 39 minutes 38 seconds 568 milliseconds 900 microseconds
```

### Does this PR introduce any user-facing change?
Yes, previously the following query fails with the error:
```sql
spark-sql> select interval 1 hour 30 minutes * 1.5;
Error in query: cannot resolve '(interval 1 hours 30 minutes * 1.5BD)' due to data type mismatch: differing types in '(interval 1 hours 30 minutes * 1.5BD)' (interval and decimal(2,1)).; line 1 pos 7;
```
After:
```sql
spark-sql> select interval 1 hour 30 minutes * 1.5;
interval 2 hours 15 minutes
```

### How was this patch tested?
- Added tests for the `multiply()` and `divide()` methods to `CalendarIntervalSuite.java`
- New test suite `IntervalExpressionsSuite`
- by tests for `Multiply` -> `MultiplyInterval` and `Divide` -> `DivideInterval` in `TypeCoercionSuite`
- updated `datetime.sql`

Closes #26132 from MaxGekk/interval-mul-div.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-06 00:37:43 +08:00
Takeshi Yamamuro 41be5125a1 [SPARK-29648][SQL][TESTS] Port limit.sql
### What changes were proposed in this pull request?

This PR ports limit.sql from PostgreSQL regression tests https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/limit.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/expected/limit.out

### Why are the changes needed?

To check behaviour differences between Spark and PostgreSQL

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

No

### How was this patch tested?

Pass the Jenkins. And, Comparison with PgSQL results

Closes #26311 from maropu/SPARK-29648.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-04 22:12:27 -08:00
Huaxin Gao 02eecfec99 [SPARK-29695][SQL] ALTER TABLE (SerDe properties) should look up catalog/table like v2 commands
### What changes were proposed in this pull request?
Add AlterTableSerDePropertiesStatement and make ALTER TABLE ... SET SERDE/SERDEPROPERTIES 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
ALTER TABLE t SET SERDE 'org.apache.class' // 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 ALTER TABLE ... SET SERDE/SERDEPROPERTIES, 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 #26374 from huaxingao/spark_29695.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-04 21:42:39 -08:00
Terry Kim 66619b84d8 [SPARK-29630][SQL] Disallow creating a permanent view that references a temporary view in an expression
### What changes were proposed in this pull request?

Disallow creating a permanent view that references a temporary view in **expressions**.

### Why are the changes needed?

Creating a permanent view that references a temporary view is currently disallowed. For example,
```SQL
# The following throws org.apache.spark.sql.AnalysisException
# Not allowed to create a permanent view `per_view` by referencing a temporary view `tmp`;
CREATE VIEW per_view AS SELECT t1.a, t2.b FROM base_table t1, (SELECT * FROM tmp) t2"
```
However, the following is allowed.
```SQL

CREATE VIEW per_view AS SELECT * FROM base_table WHERE EXISTS (SELECT * FROM tmp);
```
This PR fixes the bug where temporary views used inside expressions are not checked.

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

Yes. Now the following SQL query throws an exception as expected:
```SQL
# The following throws org.apache.spark.sql.AnalysisException
# Not allowed to create a permanent view `per_view` by referencing a temporary view `tmp`;
CREATE VIEW per_view AS SELECT * FROM base_table WHERE EXISTS (SELECT * FROM tmp);
```

### How was this patch tested?

Added new unit tests.

Closes #26361 from imback82/spark-29630.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-05 13:19:46 +08:00