Commit graph

8973 commits

Author SHA1 Message Date
Yuchen Huo d0800fc8e2 [SPARK-30314] Add identifier and catalog information to DataSourceV2Relation
### What changes were proposed in this pull request?

Add identifier and catalog information in DataSourceV2Relation so it would be possible to do richer checks in checkAnalysis step.

### Why are the changes needed?

In data source v2, table implementations are all customized so we may not be able to get the resolved identifier from tables them selves. Therefore we encode the table and catalog information in DSV2Relation so no external changes are needed to make sure this information is available.

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

No

### How was this patch tested?

Unit tests in the following suites:
CatalogManagerSuite.scala
CatalogV2UtilSuite.scala
SupportsCatalogOptionsSuite.scala
PlanResolutionSuite.scala

Closes #26957 from yuchenhuo/SPARK-30314.

Authored-by: Yuchen Huo <yuchen.huo@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-01-26 12:59:24 -08:00
Xiao Li 48f647882a [SPARK-30644][SQL][TEST] Remove query index from the golden files of SQLQueryTestSuite
### What changes were proposed in this pull request?

This PR is to remove query index from the golden files of SQLQueryTestSuite

### Why are the changes needed?

Because the SQLQueryTestSuite's golden files have the query index for each query, removal of any query statement [except the last one] will generate many unneeded difference. This will make code review harder. The number of changed lines is misleading.

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

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

Closes #27361 from gatorsmile/removeIndexNum.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-25 23:17:36 -08:00
Xiao Li d69ed9afdf Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp"
This reverts commit 1d20d13149.

Closes #27351 from gatorsmile/revertSPARK25496.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-25 21:34:12 -08:00
Liang-Chi Hsieh a0e63b61e7 [SPARK-29721][SQL] Prune unnecessary nested fields from Generate without Project
### What changes were proposed in this pull request?

This patch proposes to prune unnecessary nested fields from Generate which has no Project on top of it.

### Why are the changes needed?

In Optimizer, we can prune nested columns from Project(projectList, Generate). However, unnecessary columns could still possibly be read in Generate, if no Project on top of it. We should prune it too.

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

No

### How was this patch tested?

Unit test.

Closes #26978 from viirya/SPARK-29721.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-24 22:17:28 -08:00
Gengliang Wang ed44926117 [SPARK-30627][SQL] Disable all the V2 file sources by default
### What changes were proposed in this pull request?

Disable all the V2 file sources in Spark 3.0 by default.

### Why are the changes needed?

There are still some missing parts in the file source V2 framework:
1. It doesn't support reporting file scan metrics such as "numOutputRows"/"numFiles"/"fileSize" like `FileSourceScanExec`. This requires another patch in the data source V2 framework. Tracked by [SPARK-30362](https://issues.apache.org/jira/browse/SPARK-30362)
2. It doesn't support partition pruning with subqueries(including dynamic partition pruning) for now. Tracked by [SPARK-30628](https://issues.apache.org/jira/browse/SPARK-30628)

As we are going to code freeze on Jan 31st, this PR proposes to disable all the V2 file sources in Spark 3.0 by default.

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

No

### How was this patch tested?

Existing tests.

Closes #27348 from gengliangwang/disableFileSourceV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 21:42:43 -08:00
Xiao Li ddf83159a8 [SPARK-28962][SQL][FOLLOW-UP] Add the parameter description for the Scala function API filter
### What changes were proposed in this pull request?
This PR is a follow-up PR https://github.com/apache/spark/pull/25666 for adding the description and example for the Scala function API `filter`.

### Why are the changes needed?
It is hard to tell which parameter is the index column.

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

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

Closes #27336 from gatorsmile/spark28962.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 16:23:16 -08:00
Xiao Li 3f76bd4002 [SPARK-27083][SQL][FOLLOW-UP] Rename spark.sql.subquery.reuse to spark.sql.execution.subquery.reuse.enabled
### What changes were proposed in this pull request?
This PR is to rename spark.sql.subquery.reuse to spark.sql.execution.subquery.reuse.enabled

### Why are the changes needed?
Make it consistent and clear.

### Does this PR introduce any user-facing change?
N/A. This is a [new conf added in Spark 3.0](https://github.com/apache/spark/pull/23998)

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

Closes #27346 from gatorsmile/spark27083.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-23 15:34:54 -08:00
Terry Kim 4847f7380d [SPARK-30298][SQL] Respect aliases in output partitioning of projects and aggregates
### What changes were proposed in this pull request?

Currently, in the following scenario, bucket join is not utilized:
```scala
val df = (0 until 20).map(i => (i, i)).toDF("i", "j").as("df")
df.write.format("parquet").bucketBy(8, "i").saveAsTable("t")
sql("CREATE VIEW v AS SELECT * FROM t")
sql("SELECT * FROM t a JOIN v b ON a.i = b.i").explain
```
```
== Physical Plan ==
*(4) SortMergeJoin [i#13], [i#15], Inner
:- *(1) Sort [i#13 ASC NULLS FIRST], false, 0
:  +- *(1) Project [i#13, j#14]
:     +- *(1) Filter isnotnull(i#13)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int>, SelectedBucketsCount: 8 out of 8
+- *(3) Sort [i#15 ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(i#15, 8), true, [id=#64] <----- Exchange node introduced
      +- *(2) Project [i#13 AS i#15, j#14 AS j#16]
         +- *(2) Filter isnotnull(i#13)
            +- *(2) ColumnarToRow
               +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int>, SelectedBucketsCount: 8 out of 8
```
Notice that `Exchange` is present. This is because `Project` introduces aliases and `outputPartitioning` and `requiredChildDistribution` do not consider aliases while considering bucket join in `EnsureRequirements`. This PR addresses to allow this scenario.

### Why are the changes needed?

This allows bucket join to be utilized in the above example.

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

Yes, now with the fix, the `explain` out is as follows:
```
== Physical Plan ==
*(3) SortMergeJoin [i#13], [i#15], Inner
:- *(1) Sort [i#13 ASC NULLS FIRST], false, 0
:  +- *(1) Project [i#13, j#14]
:     +- *(1) Filter isnotnull(i#13)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:.., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int>, SelectedBucketsCount: 8 out of 8
+- *(2) Sort [i#15 ASC NULLS FIRST], false, 0
   +- *(2) Project [i#13 AS i#15, j#14 AS j#16]
      +- *(2) Filter isnotnull(i#13)
         +- *(2) ColumnarToRow
            +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:.., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int>, SelectedBucketsCount: 8 out of 8
```
Note that the `Exchange` is no longer present.

### How was this patch tested?

Closes #26943 from imback82/bucket_alias.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-24 08:23:07 +09:00
Kent Yao 3228d723a4 [SPARK-30603][SQL] Move RESERVED_PROPERTIES from SupportsNamespaces and TableCatalog to CatalogV2Util
### What changes were proposed in this pull request?
In this PR, I propose to move the `RESERVED_PROPERTIES `s from `SupportsNamespaces` and `TableCatalog` to `CatalogV2Util`, which can keep `RESERVED_PROPERTIES ` safe for interval usages only.

### Why are the changes needed?

 the `RESERVED_PROPERTIES` should not be changed by subclasses

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

no

### How was this patch tested?

existing uts

Closes #27318 from yaooqinn/SPARK-30603.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 13:13:25 -08:00
Wenchen Fan 976946a910 [SPARK-29947][SQL][FOLLOWUP] Fix table lookup cache
### What changes were proposed in this pull request?

Fix a bug in https://github.com/apache/spark/pull/26589 , to make this feature work.

### Why are the changes needed?

This feature doesn't work actually.

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

no

### How was this patch tested?

new test

Closes #27341 from cloud-fan/cache.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 13:02:10 -08:00
Wenchen Fan 3c8b3609a1 [SPARK-30620][SQL] avoid unnecessary serialization in AggregateExpression
### What changes were proposed in this pull request?

Expressions are very likely to be serialized and sent to executors, we should avoid unnecessary serialization overhead as much as we can.

This PR fixes `AggregateExpression`.

### Why are the changes needed?

small improvement

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

no

### How was this patch tested?

existing tests

Closes #27342 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-24 01:15:57 +08:00
Yuanjian Li 3d7359ad42 [SPARK-29175][SQL][FOLLOW-UP] Rename the config name to spark.sql.maven.additionalRemoteRepositories
### What changes were proposed in this pull request?
Rename the config added in #25849 to `spark.sql.maven.additionalRemoteRepositories`.

### Why are the changes needed?
Follow the advice in [SPARK-29175](https://issues.apache.org/jira/browse/SPARK-29175?focusedCommentId=17021586&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17021586), the new name is more clear.

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

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

Closes #27339 from xuanyuanking/SPARK-29175.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 08:35:32 -08:00
Maxim Gekk 705fc6ad93 [SPARK-30188][SQL][TESTS][FOLLOW-UP] Remove sorted in asserts of comparing two strings
### What changes were proposed in this pull request?
In the PR, I propose to remove sorting in the asserts of checking output of:
- expression examples,
- SQL tests in `SQLQueryTestSuite`.

### Why are the changes needed?
* Sorted `actual` and `expected` make assert output unusable. Instead of `"[true]" did not equal "[false]"`, it looks like `"[ertu]" did not equal "[aefls]"`.
* Output of expression examples should be always the same except nondeterministic expressions listed in the `ignoreSet` of the `check outputs of expression examples` test.

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

### How was this patch tested?
By running `SQLQuerySuite` via `./build/sbt "sql/test:testOnly org.apache.spark.sql.SQLQuerySuite"`.

Closes #27324 from MaxGekk/remove-sorting-in-examples-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-23 23:01:25 +08:00
Wenchen Fan dbed4c72f3 [SPARK-30605][SQL] move defaultNamespace from SupportsNamespace to CatalogPlugin
### What changes were proposed in this pull request?

Move the `defaultNamespace` method from the interface `SupportsNamespace` to `CatalogPlugin`.

### Why are the changes needed?

While I'm implementing JDBC V2, I realize that the default namespace is very an important information. Even if you don't want to implement namespace manipulation functionalities like CREATE/DROP/ALTER namespace, you still need to report the default namespace.

The default namespace is not about functionality but a matter of correctness. If you don't know the default namespace of a catalog, it's wrong to assume it's `[]`. You may get table not found exception if you do so.

I think it's more reasonable to put the `defaultNamespace` method in the base class `CatalogPlugin`. It returns `[]` by default so won't bother implementation if they really don't have namespace concept.

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

yes, but for an unreleased API.

### How was this patch tested?

existing tests

Closes #27319 from cloud-fan/ns.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-23 21:56:54 +08:00
Xiao Li ffd435b57f [SPARK-27871][SQL][FOLLOW-UP] Remove the conf spark.sql.optimizer.reassignLambdaVariableID.enabled
### What changes were proposed in this pull request?
This PR is to remove the conf

### Why are the changes needed?
This rule can be excluded using spark.sql.optimizer.excludedRules without an extra conf

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

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

Closes #27334 from gatorsmile/spark27871.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-23 21:41:56 +08:00
Burak Yavuz db528e4fe1 [SPARK-30535][SQL] Revert "[] Migrate ALTER TABLE commands to the new framework
### What changes were proposed in this pull request?

This reverts commit b5cb9abdd5.

### Why are the changes needed?

The merged commit (#27243) was too risky for several reasons:
 1. It doesn't fix a bug
 2. It makes the resolution of the table that's going to be altered a child. We had avoided this on purpose as having an arbitrary rule change the child of AlterTable seemed risky. This change alone is a big -1 for me for this change.
 3. While the code may look cleaner, I think this approach makes certain things harder, e.g. differentiating between the Hive based Alter table CHANGE COLUMN and ALTER COLUMN syntax. Resolving and normalizing columns for ALTER COLUMN also becomes a bit harder, as we now have to check every single AlterTable command instead of just a single ALTER TABLE ALTER COLUMN statement

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

No

### How was this patch tested?

Existing unit tests

This closes #27315

Closes #27327 from brkyvz/revAlter.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-22 22:43:46 -08:00
Tathagata Das d2bca8ff70 [SPARK-30609] Allow default merge command resolution to be bypassed by DSv2 tables
### What changes were proposed in this pull request?
Skip resolving the merge expressions if the target is a DSv2 table with ACCEPT_ANY_SCHEMA capability.

### Why are the changes needed?
Some DSv2 sources may want to customize the merge resolution logic. For example, a table that can accept any schema (TableCapability.ACCEPT_ANY_SCHEMA) may want to allow certain merge queries that are blocked (that is, throws AnalysisError) by the default resolution logic. So there should be a way to completely bypass the merge resolution logic in the Analyzer.

### Does this PR introduce any user-facing change?
No, since merge itself is an unreleased feature

### How was this patch tested?
added unit test to specifically test the skipping.

Closes #27326 from tdas/SPARK-30609.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2020-01-22 19:20:25 -08:00
Ajith bbab2bb961 [SPARK-30556][SQL] Copy sparkContext.localproperties to child thread inSubqueryExec.executionContext
### What changes were proposed in this pull request?
In `org.apache.spark.sql.execution.SubqueryExec#relationFuture` make a copy of `org.apache.spark.SparkContext#localProperties` and pass it to the sub-execution thread in `org.apache.spark.sql.execution.SubqueryExec#executionContext`

### Why are the changes needed?
Local properties set via sparkContext are not available as TaskContext properties when executing  jobs and threadpools have idle threads which are reused

Explanation:
When `SubqueryExec`, the relationFuture is evaluated via a separate thread. The threads inherit the `localProperties` from `sparkContext` as they are the child threads.
These threads are created in the `executionContext` (thread pools). Each Thread pool has a default keepAliveSeconds of 60 seconds for idle threads.
Scenarios where the thread pool has threads which are idle and reused for a subsequent new query, the thread local properties will not be inherited from spark context (thread properties are inherited only on thread creation) hence end up having old or no properties set. This will cause taskset properties to be missing when properties are transferred by child thread via `sparkContext.runJob/submitJob`

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

### How was this patch tested?
Added UT

Closes #27267 from ajithme/subquerylocalprop.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-22 18:21:11 -08:00
Enrico Minack eccae13a5f [SPARK-30531][WEB UI] Do not render plan viz when it exists already
### What changes were proposed in this pull request?
When you save a Spark UI SQL query page to disk and then display the html file with your browser, the query plan will be rendered a second time. This change avoids rendering the plan visualization when it exists already.

This is master:
![grafik](https://user-images.githubusercontent.com/44700269/72543429-fcb8d980-3885-11ea-82aa-c0b3638847e5.png)

And with the fix:
![grafik](https://user-images.githubusercontent.com/44700269/72543641-57523580-3886-11ea-8cdf-5fb0cdffa983.png)

### Why are the changes needed?
The duplicate query plan is unexpected and redundant.

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

### How was this patch tested?
Manually tested. Testing this in a reproducible way requires a running browser or HTML rendering engine that executes the JavaScript.

Closes #27238 from EnricoMi/branch-sql-ui-duplicate-plan.

Authored-by: Enrico Minack <github@enrico.minack.dev>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-22 19:51:08 -06:00
Maxim Gekk 4ca31b470f [SPARK-30606][SQL] Fix the like function with 2 parameters
### What changes were proposed in this pull request?
In the PR, I propose to add additional constructor in the `Like` expression. The constructor can be used on applying the `like` function with 2 parameters.

### Why are the changes needed?
`FunctionRegistry` cannot find a constructor if the `like` function is applied to 2 parameters.

### Does this PR introduce any user-facing change?
Yes, before:
```sql
spark-sql> SELECT like('Spark', '_park');

Invalid arguments for function like; line 1 pos 7
org.apache.spark.sql.AnalysisException: Invalid arguments for function like; line 1 pos 7
	at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$.$anonfun$expression$7(FunctionRegistry.scala:618)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$.$anonfun$expression$4(FunctionRegistry.scala:602)
	at org.apache.spark.sql.catalyst.analysis.SimpleFunctionRegistry.lookupFunction(FunctionRegistry.scala:121)
	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupFunction(SessionCatalog.scala:1412)
```
After:
```sql
spark-sql> SELECT like('Spark', '_park');
true
```

### How was this patch tested?
By running `check outputs of expression examples` from `SQLQuerySuite`.

Closes #27323 from MaxGekk/fix-like-func.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-22 15:40:24 -08:00
jiake 6dfaa0783f [SPARK-30549][SQL] Fix the subquery shown issue in UI When enable AQE
### What changes were proposed in this pull request?
After [PR#25316](https://github.com/apache/spark/pull/25316) fixed the dead lock issue in [PR#25308](https://github.com/apache/spark/pull/25308), the subquery metrics can not be shown in UI as following screenshot.
![image](https://user-images.githubusercontent.com/11972570/72891385-160ec980-3d4f-11ea-91fc-ccaad890f7dc.png)

 This PR fix the subquery UI shown issue by adding `SparkListenerSQLAdaptiveSQLMetricUpdates` event to update the suquery  sql metric. After with this PR, the suquery UI can show correctly as following screenshot:
![image](https://user-images.githubusercontent.com/11972570/72893610-66d4f100-3d54-11ea-93c9-f444b2f31952.png)

### Why are the changes needed?
Showing the subquery metric in UI when enable AQE

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

### How was this patch tested?
Existing UT

Closes #27260 from JkSelf/fixSubqueryUI.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-22 09:02:34 -08:00
Kent Yao 8e280cebf2 [SPARK-30592][SQL] Interval support for csv and json funtions
### What changes were proposed in this pull request?

In this PR, I'd propose to fully support interval for the CSV and JSON functions.

On one hand, CSV and JSON records consist of string values, in the cast logic, we can cast string from/to interval now, so we can make those functions support intervals easily.

Before this change we can only use this as a workaround.
```sql
SELECT cast(from_csv('1, 1 day', 'a INT, b string').b as interval)
struct<CAST(from_csv(1, 1 day).b AS INTERVAL):interval>
1 days
```

On the other hand,  we ban reading or writing intervals from CSV and JSON files. To directly read and write  with external json/csv storage, you still need explicit cast, e.g.
```scala
spark.read.schema("a string").json("a.json").selectExpr("cast(a as interval)").show
+------+
|     a|
+------+
|1 days|
+------+
```

### Why are the changes needed?

for interval's future-proofing purpose

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

yes, the `to_json`/`from_json` function can deal with intervals now. e.g.
for `from_json` there is no such use case because we do not support `a interval`
for `to_json`, we can use interval values now

#### before

 ```sql

 SELECT to_json(map('a', interval 25 month 100 day 130 minute));
Error in query: cannot resolve 'to_json(map('a', INTERVAL '2 years 1 months 100 days 2 hours 10 minutes'))' due to data type mismatch: Unable to convert column a of type interval to JSON.; line 1 pos 7;
'Project [unresolvedalias(to_json(map(a, 2 years 1 months 100 days 2 hours 10 minutes), Some(Asia/Shanghai)), None)]
+- OneRowRelation
```
#### after
```sql
SELECT to_json(map('a', interval 25 month 100 day 130 minute))
{"a":"2 years 1 months 100 days 2 hours 10 minutes"}

```
### How was this patch tested?

add ut

Closes #27317 from yaooqinn/SPARK-30592.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-23 00:41:46 +08:00
Wenchen Fan b8cb52a8a7 [SPARK-30555][SQL] MERGE INTO insert action should only access columns from source table
### What changes were proposed in this pull request?

when resolving the `Assignment` of insert action in MERGE INTO, only resolve with the source table, to avoid ambiguous attribute failure if there is a same-name column in the target table.

### Why are the changes needed?

The insert action is used when NOT MATCHED, so it can't access the row from the target table anyway.

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

on

### How was this patch tested?

new tests

Closes #27265 from cloud-fan/merge.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-22 21:31:11 +08:00
Kent Yao f2d71f5838 [SPARK-30591][SQL] Remove the nonstandard SET OWNER syntax for namespaces
### What changes were proposed in this pull request?

This pr removes the nonstandard `SET OWNER` syntax for namespaces and changes the owner reserved properties from `ownerName` and `ownerType` to `owner`.

### Why are the changes needed?

the `SET OWNER` syntax for namespaces is hive-specific and non-sql standard, we need a more future-proofing design before we implement user-facing changes for SQL security issues

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

no, just revert an unpublic syntax

### How was this patch tested?

modified uts

Closes #27300 from yaooqinn/SPARK-30591.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-22 16:00:05 +08:00
fuwhu cfb1706eaa [SPARK-15616][SQL] Add optimizer rule PruneHiveTablePartitions
### What changes were proposed in this pull request?
Add optimizer rule PruneHiveTablePartitions pruning hive table partitions based on filters on partition columns.
Doing so, the total size of pruned partitions may be small enough for broadcast join in JoinSelection strategy.

### Why are the changes needed?
In JoinSelection strategy, spark use the "plan.stats.sizeInBytes" to decide whether the plan is suitable for broadcast join.
Currently, "plan.stats.sizeInBytes" does not take "pruned partitions" into account, so it may miss some broadcast join and take sort-merge join instead, which will definitely impact join performance.
This PR aim at taking "pruned partitions" into account for hive table in "plan.stats.sizeInBytes" and then improve performance by using broadcast join if possible.

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

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

This is based on #25919, credits should go to lianhuiwang and advancedxy.

Closes #26805 from fuwhu/SPARK-15616.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 21:26:30 +08:00
yi.wu ff39c9271c [SPARK-30252][SQL] Disallow negative scale of Decimal
### What changes were proposed in this pull request?

This PR propose to disallow negative `scale` of `Decimal` in Spark. And this PR brings two behavior changes:

1) for literals like `1.23E4BD` or `1.23E4`(with `spark.sql.legacy.exponentLiteralAsDecimal.enabled`=true, see [SPARK-29956](https://issues.apache.org/jira/browse/SPARK-29956)), we set its `(precision, scale)` to (5, 0) rather than (3, -2);
2) add negative `scale` check inside the decimal method if it exposes to set `scale` explicitly. If check fails, `AnalysisException` throws.

And user could still use `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled` to restore the previous behavior.

### Why are the changes needed?

According to SQL standard,
> 4.4.2 Characteristics of numbers
An exact numeric type has a precision P and a scale S. P is a positive integer that determines the number of significant digits in a particular radix R, where R is either 2 or 10. S is a non-negative integer.

scale of Decimal should always be non-negative. And other mainstream databases, like Presto, PostgreSQL, also don't allow negative scale.

Presto:
```
presto:default> create table t (i decimal(2, -1));
Query 20191213_081238_00017_i448h failed: line 1:30: mismatched input '-'. Expecting: <integer>, <type>
create table t (i decimal(2, -1))
```

PostgrelSQL:
```
postgres=# create table t(i decimal(2, -1));
ERROR:  NUMERIC scale -1 must be between 0 and precision 2
LINE 1: create table t(i decimal(2, -1));
                         ^
```

And, actually, Spark itself already doesn't allow to create table with negative decimal types using SQL:
```
scala> spark.sql("create table t(i decimal(2, -1))");
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'create table t(i decimal(2, -'(line 1, pos 28)

== SQL ==
create table t(i decimal(2, -1))
----------------------------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605)
  ... 35 elided
```

However, it is still possible to create such table or `DatFrame` using Spark SQL programming API:
```
scala> val tb =
 CatalogTable(
  TableIdentifier("test", None),
  CatalogTableType.MANAGED,
  CatalogStorageFormat.empty,
  StructType(StructField("i", DecimalType(2, -1) ) :: Nil))
```
```
scala> spark.sql("SELECT 1.23E4BD")
res2: org.apache.spark.sql.DataFrame = [1.23E+4: decimal(3,-2)]
```
while, these two different behavior could make user confused.

On the other side, even if user creates such table or `DataFrame` with negative scale decimal type, it can't write data out if using format, like `parquet` or `orc`. Because these formats have their own check for negative scale and fail on it.
```
scala> spark.sql("SELECT 1.23E4BD").write.saveAsTable("parquet")
19/12/13 17:37:04 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.IllegalArgumentException: Invalid DECIMAL scale: -2
	at org.apache.parquet.Preconditions.checkArgument(Preconditions.java:53)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.decimalMetadata(Types.java:495)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:403)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:309)
	at org.apache.parquet.schema.Types$Builder.named(Types.java:290)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:428)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:334)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.$anonfun$convert$2(ParquetSchemaConverter.scala:326)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.IterableLike.foreach(IterableLike.scala:74)
	at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
	at org.apache.spark.sql.types.StructType.foreach(StructType.scala:99)
	at scala.collection.TraversableLike.map(TraversableLike.scala:238)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
	at org.apache.spark.sql.types.StructType.map(StructType.scala:99)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convert(ParquetSchemaConverter.scala:326)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.init(ParquetWriteSupport.scala:97)
	at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:388)
	at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:150)
	at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:124)
	at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:109)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:264)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:441)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:444)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

So, I think it would be better to disallow negative scale totally and make behaviors above be consistent.

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

Yes, if `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=false`, user couldn't create Decimal value with negative scale anymore.

### How was this patch tested?

Added new tests in `ExpressionParserSuite` and `DecimalSuite`;
Updated `SQLQueryTestSuite`.

Closes #26881 from Ngone51/nonnegative-scale.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 21:09:48 +08:00
Kent Yao af705421db [SPARK-30593][SQL] Revert interval ISO/ANSI SQL Standard output since we decide not to follow ANSI and no round trip
### What changes were proposed in this pull request?

This revert https://github.com/apache/spark/pull/26418, file a new ticket under  https://issues.apache.org/jira/browse/SPARK-30546 for better tracking interval behavior
### Why are the changes needed?

Revert interval ISO/ANSI SQL Standard output since we decide not to follow ANSI and there is no round trip

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

no, not released yet

### How was this patch tested?

existing uts

Closes #27304 from yaooqinn/SPARK-30593.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 20:51:10 +08:00
yi.wu 78df532556 [SPARK-30433][SQL][FOLLOW-UP] Optimize collect conflict plans
### What changes were proposed in this pull request?

For LogicalPlan(e.g. `MultiInstanceRelation`, `Project`, `Aggregate`, etc)  whose output doesn't inherit directly from its children, we could just stop collect on it. Because we could always replace all the lower conflict attributes with the new attributes from the new plan.

Otherwise, we should recursively collect conflict plans, like `Generate`, `Window`.

### Why are the changes needed?

Performance improvement.

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

No.

### How was this patch tested?

Pass existed tests.

Closes #27263 from Ngone51/spark_30433_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 14:23:55 +08:00
Guy Khazma 2d59ca464e [SPARK-30475][SQL] File source V2: Push data filters for file listing
### What changes were proposed in this pull request?
Follow up on [SPARK-30428](https://github.com/apache/spark/pull/27112) which added support for partition pruning in File source V2.
This PR implements the necessary changes in order to pass the `dataFilters` to the `listFiles`. This enables having `FileIndex` implementations which use the `dataFilters` for further pruning the file listing (see the discussion [here](https://github.com/apache/spark/pull/27112#discussion_r364757217)).

### Why are the changes needed?
Datasources such as `csv` and `json` do not implement the `SupportsPushDownFilters` trait. In order to support data skipping uniformly for all file based data sources, one can override the `listFiles` method in a `FileIndex` implementation, which consults external metadata and prunes the list of files.

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

### How was this patch tested?
Modifying the unit tests for v2 file sources to verify the `dataFilters` are passed

Closes #27157 from guykhazma/PushdataFiltersInFileListing.

Authored-by: Guy Khazma <guykhag@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-01-20 20:20:37 -08:00
Maxim Gekk 94284c8ecc [SPARK-30587][SQL][TESTS] Add test suites for CSV and JSON v1
### What changes were proposed in this pull request?
In the PR, I propose to make `JsonSuite` and `CSVSuite` abstract classes, and add sub-classes that check JSON/CSV datasource v1 and v2.

### Why are the changes needed?
To improve test coverage and test JSON/CSV v1 which is still supported, and can be enabled by users.

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

### How was this patch tested?
By running new test suites `JsonV1Suite` and `CSVv1Suite`.

Closes #27294 from MaxGekk/csv-json-v1-test-suites.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 11:38:05 +08:00
Kent Yao 0388b7a3ec [SPARK-30568][SQL] Invalidate interval type as a field table schema
### What changes were proposed in this pull request?

After this commit d67b98ea01, we are able to create table or alter table with interval column types if the external catalog accepts which is varying the interval type's purpose for internal usage. With d67b98ea01 's original purpose it should only work from cast logic.

Instead of adding type checker for the interval type from commands to commands to work among catalogs, It much simpler to treat interval as an invalid data type but can be identified by cast only.

### Why are the changes needed?

enhance interval internal usage purpose.

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

NO,
Additionally, this PR restores user behavior when using interval type to create/alter table schema, e.g. for hive catalog
for 2.4,
```java
Caused by: org.apache.spark.sql.catalyst.parser.ParseException:
DataType calendarinterval is not supported.(line 1, pos 0)
```
for master after  d67b98ea01
```java
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.IllegalArgumentException: Error: type expected at the position 0 of 'interval' but 'interval' is found.
  at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:862)
```
now with this pr, we restore the type checker in spark side.

### How was this patch tested?

add more ut

Closes #27277 from yaooqinn/SPARK-30568.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 11:14:26 +08:00
Kent Yao 24efa43826 [SPARK-30019][SQL] Add the owner property to v2 table
### What changes were proposed in this pull request?

Add `owner` property to v2 table, it is reversed by `TableCatalog`, indicates the table's owner.

### Why are the changes needed?

enhance ownership management of catalog API

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

yes, add 1 reserved property - `owner` , and it is not allowed to use in OPTIONS/TBLPROPERTIES anymore, only if legacy on

### How was this patch tested?

add uts

Closes #27249 from yaooqinn/SPARK-30019.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 10:37:49 +08:00
HyukjinKwon 14bc2a2162 [SPARK-30530][SQL][FOLLOW-UP] Remove unnecessary codes and fix comments accordingly in UnivocityParser
### What changes were proposed in this pull request?

This PR proposes to clean up `UnivocityParser`.

### Why are the changes needed?

It will slightly improve the performance since we don't do unnecessary computation for Array concatenations/creation.

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

No.

### How was this patch tested?

Manually ran the existing tests.

Closes #27287 from HyukjinKwon/SPARK-30530-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-21 10:20:01 +09:00
Maxim Gekk fd69533593 [SPARK-30482][CORE][SQL][TESTS][FOLLOW-UP] Output caller info in log appenders while reaching the limit
### What changes were proposed in this pull request?
In the PR, I propose to output additional msg from the tests where a log appender is added. The message is printed as a part of `IllegalStateException` in the case of reaching the limit of maximum number of logged events.

### Why are the changes needed?
If a log appender is not removed from the log4j appenders list. the caller message could help to investigate the problem and find the test which doesn't remove the log appender.

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

### How was this patch tested?
By running the modified test suites `AvroSuite`, `CSVSuite`, `ResolveHintsSuite` and etc.

Closes #27296 from MaxGekk/assign-name-to-log-appender.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-21 10:19:07 +09:00
yi.wu f5b345cf3d [SPARK-30578][SQL][TEST] Explicitly set conf to use DSv2 for orc in OrcFilterSuite
### What changes were proposed in this pull request?

Explicitly set conf to let orc use DSv2 in `OrcFilterSuite` in both v1.2 and v2.3.

### Why are the changes needed?

Tests should not rely on default conf when they're going to test something intentionally, which can be fail when conf changes.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27285 from Ngone51/fix-orcfilter-test.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-20 21:42:33 +08:00
Terry Kim b5cb9abdd5 [SPARK-30535][SQL] Migrate ALTER TABLE commands to the new framework
### What changes were proposed in this pull request?

Use the new framework to resolve the ALTER TABLE commands.

This PR also refactors ALTER TABLE logical plans such that they extend a base class `AlterTable`. Each plan now implements `def changes: Seq[TableChange]` for any table change operations.

Additionally, `UnresolvedV2Relation` and its usage is completely removed.

### Why are the changes needed?

This is a part of effort to make the relation lookup behavior consistent: [SPARK-29900](https://issues.apache.org/jira/browse/SPARK-29900).

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

No

### How was this patch tested?

Updated existing tests

Closes #27243 from imback82/v2commands_newframework.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-20 21:33:44 +08:00
Maxim Gekk 00039cc482 [SPARK-30554][SQL] Return Iterable from FailureSafeParser.rawParser
### What changes were proposed in this pull request?
Changed signature of `rawParser` passed to `FailureSafeParser`. I propose to change return type from `Seq` to `Iterable`. I took `Iterable` to easier port the changes on Scala collections 2.13. Also, I replaced `Seq` by `Option` in CSV datasource - `UnivocityParser`, and in JSON parser exception one place in the case when specified schema is `StructType`, and JSON input is an array.

### Why are the changes needed?
`Seq` is unnecessary requirement for return type from rawParser which may not have multiple rows per input like CSV datasource.

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

### How was this patch tested?
By existing test suites `JsonSuite`, `UnivocityParserSuite`, `JsonFunctionsSuite`, `JsonExpressionsSuite`, `CsvSuite`, and `CsvFunctionsSuite`.

Closes #27264 from MaxGekk/failuresafe-parser-seq.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-20 13:59:22 +09:00
Josh Rosen d50f8df929 [SPARK-30413][SQL] Avoid WrappedArray roundtrip in GenericArrayData constructor, plus related optimization in ParquetMapConverter
### What changes were proposed in this pull request?

This PR implements a tiny performance optimization for a `GenericArrayData` constructor, avoiding an unnecessary roundtrip through `WrappedArray` when the provided value is already an array of objects.

It also fixes a related performance problem in `ParquetRowConverter`.

### Why are the changes needed?

`GenericArrayData` has a `this(seqOrArray: Any)` constructor, which was originally added in #13138 for use in `RowEncoder` (where we may not know concrete types until runtime) but is also called (perhaps unintentionally) in a few other code paths.

In this constructor's existing implementation, a call to `new WrappedArray(Array[Object](""))` is dispatched to the `this(seqOrArray: Any)` constructor, where we then call `this(array.toSeq)`: this wraps the provided array into a `WrappedArray`, which is subsequently unwrapped in a `this(seq.toArray)` call. For an interactive example, see https://scastie.scala-lang.org/7jOHydbNTaGSU677FWA8nA

This PR changes the `this(seqOrArray: Any)` constructor so that it calls the primary `this(array: Array[Any])` constructor, allowing us to save a `.toSeq.toArray` call; this comes at the cost of one additional `case` in the `match` statement (but I believe this has a negligible performance impact relative to the other savings).

As code cleanup, I also reverted the JVM 1.7 workaround from #14271.

I also fixed a related performance problem in `ParquetRowConverter`: previously, this code called `ArrayBasedMapData.apply` which, in turn, called the `this(Any)` constructor for `GenericArrayData`: this PR's micro-benchmarks show that this is _significantly_ slower than calling the `this(Array[Any])` constructor (and I also observed time spent here during other Parquet scan benchmarking work). To fix this performance problem, I replaced the call to the  `ArrayBasedMapData.apply` method with direct calls to the `ArrayBasedMapData` and `GenericArrayData` constructors.

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

No.

### How was this patch tested?

I tested this by running code in a debugger and by running microbenchmarks (which I've added to a new `GenericArrayDataBenchmark` in this PR):

- With JDK8 benchmarks: this PR's changes more than double the performance of calls to the `this(Any)` constructor. Even after improvements, however, calls to the `this(Array[Any])` constructor are still ~60x faster than calls to `this(Any)` when passing a non-primitive array (thereby motivating this patch's other change in `ParquetRowConverter`).
- With JDK11 benchmarks: the changes more-or-less completely eliminate the performance penalty associated with the `this(Any)` constructor.

Closes #27088 from JoshRosen/joshrosen/GenericArrayData-optimization.

Authored-by: Josh Rosen <rosenville@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-19 19:12:19 -08:00
Maxim Gekk d4c6ec6ba7 [SPARK-30530][SQL] Fix filter pushdown for bad CSV records
### What changes were proposed in this pull request?
In the PR, I propose to fix the bug reported in SPARK-30530. CSV datasource returns invalid records in the case when `parsedSchema` is shorter than number of tokens returned by UniVocity parser. In the case `UnivocityParser.convert()` always throws `BadRecordException` independently from the result of applying filters.

For the described case, I propose to save the exception in `badRecordException` and continue value conversion according to `parsedSchema`. If a bad record doesn't pass filters, `convert()` returns empty Seq otherwise throws `badRecordException`.

### Why are the changes needed?
It fixes the bug reported in the JIRA ticket.

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

### How was this patch tested?
Added new test from the JIRA ticket.

Closes #27239 from MaxGekk/spark-30530-csv-filter-is-null.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-19 17:22:38 +08:00
Kent Yao 17857f9b8b [SPARK-30551][SQL] Disable comparison for interval type
### What changes were proposed in this pull request?

As we are not going to follow ANSI to implement year-month and day-time interval types, it is weird to compare the year-month part to the day-time part for our current implementation of interval type now.

Additionally, the current ordering logic comes from PostgreSQL where the implementation of the interval is messy. And we are not aiming PostgreSQL compliance at all.

THIS PR will revert https://github.com/apache/spark/pull/26681 and https://github.com/apache/spark/pull/26337

### Why are the changes needed?

make interval type more future-proofing

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

there are new in 3.0, so no

### How was this patch tested?

existing uts shall work

Closes #27262 from yaooqinn/SPARK-30551.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-19 15:27:51 +08:00
jiake 0d99d7e3f2 [SPARK-30524] [SQL] follow up SPARK-30524 to resolve comments
### What changes were proposed in this pull request?
Resolve the remaining comments in [PR#27226](https://github.com/apache/spark/pull/27226).

### Why are the changes needed?
Resolve the comments.

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

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

Closes #27253 from JkSelf/followup-skewjoinoptimization2.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-19 15:10:05 +08:00
Sean Owen 789a4abfa9 [MINOR][HIVE] Pick up HIVE-22708 HTTP transport fix
### What changes were proposed in this pull request?

Pick up the HTTP fix from https://issues.apache.org/jira/browse/HIVE-22708

### Why are the changes needed?

This is a small but important fix to digest handling we should pick up from Hive.

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

No.

### How was this patch tested?

Existing tests

Closes #27273 from srowen/Hive22708.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-18 11:50:59 -08:00
HyukjinKwon a6bdea3ad4 [SPARK-30539][PYTHON][SQL] Add DataFrame.tail in PySpark
### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/26809 added `Dataset.tail` API. It should be good to have it in PySpark API as well.

### Why are the changes needed?

To support consistent APIs.

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

No. It adds a new API.

### How was this patch tested?

Manually tested and doctest was added.

Closes #27251 from HyukjinKwon/SPARK-30539.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-18 00:18:12 -08:00
Kevin Yu 96a344511e [SPARK-25993][SQL][TESTS] Add test cases for CREATE EXTERNAL TABLE with subdirectories
### What changes were proposed in this pull request?

This PR aims to add these test cases for resolution of ORC table location reported by [SPARK-25993](https://issues.apache.org/jira/browse/SPARK-25993)
also add corresponding test cases for Parquet table.

### Why are the changes needed?

The current behavior is complex, this test case suites are designed to prevent the accidental behavior change. This pr is rebased on master, the original pr is [23108](https://github.com/apache/spark/pull/23108)

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

No. This adds test cases only.

### How was this patch tested?

This is a new test case.

Closes #27130 from kevinyu98/spark-25993-2.

Authored-by: Kevin Yu <qyu@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-17 17:17:29 -08:00
Gabor Somogyi abf759a91e [SPARK-29876][SS] Delete/archive file source completed files in separate thread
### What changes were proposed in this pull request?
[SPARK-20568](https://issues.apache.org/jira/browse/SPARK-20568) added the possibility to clean up completed files in streaming query. Deleting/archiving uses the main thread which can slow down processing. In this PR I've created thread pool to handle file delete/archival. The number of threads can be configured with `spark.sql.streaming.fileSource.cleaner.numThreads`.

### Why are the changes needed?
Do file delete/archival in separate thread.

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

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

Closes #26502 from gaborgsomogyi/SPARK-29876.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-17 10:45:36 -08:00
Luca Canali fd308ade52 [SPARK-30041][SQL][WEBUI] Add Codegen Stage Id to Stage DAG visualization in Web UI
### What changes were proposed in this pull request?
SPARK-29894 provides information on the Codegen Stage Id in WEBUI for SQL Plan graphs. Similarly, this proposes to add Codegen Stage Id in the DAG visualization for Stage execution. DAGs for Stage execution are available in the WEBUI under the Jobs and Stages tabs.

### Why are the changes needed?
This is proposed as an aid for drill-down analysis of complex SQL statement execution, as it is not always easy to match parts of the SQL Plan graph with the corresponding Stage DAG execution graph. Adding Codegen Stage Id for WholeStageCodegen operations makes this task easier.

### Does this PR introduce any user-facing change?
Stage DAG visualization in the WEBUI will show codegen stage id for WholeStageCodegen operations, as in the example snippet from the WEBUI, Jobs tab  (the query used in the example is TPCDS 2.4 q14a):
![](https://issues.apache.org/jira/secure/attachment/12987461/Snippet_StagesDags_with_CodegenId%20_annotated.png)

### How was this patch tested?
Manually tested, see also example snippet.

Closes #26675 from LucaCanali/addCodegenStageIdtoStageGraph.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-18 01:00:45 +08:00
Terry Kim 64fe192fef [SPARK-30282][SQL] Migrate SHOW TBLPROPERTIES to new framework
### What changes were proposed in this pull request?

Use the new framework to resolve the SHOW TBLPROPERTIES command. This PR along with #27243 should update all the existing V2 commands with `UnresolvedV2Relation`.

### Why are the changes needed?

This is a part of effort to make the relation lookup behavior consistent: [SPARK-2990](https://issues.apache.org/jira/browse/SPARK-29900).

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

Yes `SHOW TBLPROPERTIES temp_view` now fails with `AnalysisException` will be thrown with a message `temp_view is a temp view not table`. Previously, it was returning empty row.

### How was this patch tested?

Existing tests

Closes #26921 from imback82/consistnet_v2command.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-17 16:51:44 +08:00
Wenchen Fan 0bd7a3dfab [SPARK-29572][SQL] add v1 read fallback API in DS v2
### What changes were proposed in this pull request?

Add a `V1Scan` interface, so that data source v1 implementations can migrate to DS v2 much easier.

### Why are the changes needed?

It's a lot of work to migrate v1 sources to DS v2. The new API added here can allow v1 sources to go through v2 code paths without implementing all the Batch, Stream, PartitionReaderFactory, ... stuff.

We already have a v1 write fallback API after https://github.com/apache/spark/pull/25348

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

no

### How was this patch tested?

new test suite

Closes #26231 from cloud-fan/v1-read-fallback.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-17 12:40:51 +08:00
Maxim Gekk 1a9de8c31f [SPARK-30499][SQL] Remove SQL config spark.sql.execution.pandas.respectSessionTimeZone
### What changes were proposed in this pull request?
In the PR, I propose to remove the SQL config `spark.sql.execution.pandas.respectSessionTimeZone` which has been deprecated since Spark 2.3.

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

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

### How was this patch tested?
by running python tests, https://spark.apache.org/docs/latest/building-spark.html#pyspark-tests-with-maven-or-sbt

Closes #27218 from MaxGekk/remove-respectSessionTimeZone.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-17 11:44:49 +09:00
jiake 6e5b4bf113 [SPARK-30524][SQL] Disable OptimizeSkewedJoin rule when introducing additional shuffle
### What changes were proposed in this pull request?
`OptimizeSkewedJoin `rule change the `outputPartitioning `after inserting `PartialShuffleReaderExec `or `SkewedPartitionReaderExec`. So it may  need to introduce additional to ensure the right result. This PR disable `OptimizeSkewedJoin  ` rule when introducing additional shuffle.

### Why are the changes needed?
bug fix

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

### How was this patch tested?
Add new ut

Closes #27226 from JkSelf/followup-skewedoptimization.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-16 22:52:00 +08:00