Commit graph

6569 commits

Author SHA1 Message Date
herman b25359cca3 [SPARK-30780][SQL] Empty LocalTableScan should use RDD without partitions
### What changes were proposed in this pull request?
This is a small follow-up for https://github.com/apache/spark/pull/27400. This PR makes an empty `LocalTableScanExec` return an `RDD` without partitions.

### Why are the changes needed?
It is a bit unexpected that the RDD contains partitions if there is not work to do. It also can save a bit of work when this is used in a more complex plan.

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

### How was this patch tested?
Added test to `SparkPlanSuite`.

Closes #27530 from hvanhovell/SPARK-30780.

Authored-by: herman <herman@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-12 10:48:29 +09:00
Maxim Gekk 45db48e2d2 Revert "[SPARK-30625][SQL] Support escape as third parameter of the like function
### What changes were proposed in this pull request?

In the PR, I propose to revert the commit 8aebc80e0e.

### Why are the changes needed?
See the concerns https://github.com/apache/spark/pull/27355#issuecomment-584344438

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

### How was this patch tested?
By existing test suites.

Closes #27531 from MaxGekk/revert-like-3-args.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-11 10:15:34 -08:00
fuwhu f1d0dce484 [MINOR][DOC] Add class document for PruneFileSourcePartitions and PruneHiveTablePartitions
### What changes were proposed in this pull request?
Add class document for PruneFileSourcePartitions and PruneHiveTablePartitions.

### Why are the changes needed?
To describe these two classes.

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

### How was this patch tested?
no

Closes #27535 from fuwhu/SPARK-15616-FOLLOW-UP.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-11 22:16:44 +08:00
HyukjinKwon 0045be766b [SPARK-29462][SQL] The data type of "array()" should be array<null>
### What changes were proposed in this pull request?

This brings https://github.com/apache/spark/pull/26324 back. It was reverted basically because, firstly Hive compatibility, and the lack of investigations in other DBMSes and ANSI.

- In case of PostgreSQL seems coercing NULL literal to TEXT type.
- Presto seems coercing `array() + array(1)` -> array of int.
- Hive seems  `array() + array(1)` -> array of strings

 Given that, the design choices have been differently made for some reasons. If we pick one of both, seems coercing to array of int makes much more sense.

Another investigation was made offline internally. Seems ANSI SQL 2011, section 6.5 "<contextually typed value specification>" states:

> If ES is specified, then let ET be the element type determined by the context in which ES appears. The declared type DT of ES is Case:
>
> a) If ES simply contains ARRAY, then ET ARRAY[0].
>
> b) If ES simply contains MULTISET, then ET MULTISET.
>
> ES is effectively replaced by CAST ( ES AS DT )

From reading other related context, doing it to `NullType`. Given the investigation made, choosing to `null` seems correct, and we have a reference Presto now. Therefore, this PR proposes to bring it back.

### 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?
Yes, `array()` creates `array<null>`. Now `array(1) + array()` can correctly create `array(1)` instead of `array("1")`.

### How was this patch tested?
Tested manually

Closes #27521 from HyukjinKwon/SPARK-29462.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-11 17:22:08 +09:00
Yuanjian Li a6b91d2bf7 [SPARK-30556][SQL][FOLLOWUP] Reset the status changed in SQLExecution.withThreadLocalCaptured
### What changes were proposed in this pull request?
Follow up for #27267, reset the status changed in SQLExecution.withThreadLocalCaptured.

### Why are the changes needed?
For code safety.

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

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

Closes #27516 from xuanyuanking/SPARK-30556-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-10 22:16:25 +01:00
Liang-Chi Hsieh acfdb46a60 [SPARK-27946][SQL][FOLLOW-UP] Change doc and error message for SHOW CREATE TABLE
### What changes were proposed in this pull request?

This is a follow-up for #24938 to tweak error message and migration doc.

### Why are the changes needed?

Making user know workaround if SHOW CREATE TABLE doesn't work for some Hive tables.

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

No

### How was this patch tested?

Existing unit tests.

Closes #27505 from viirya/SPARK-27946-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2020-02-10 10:45:00 -08:00
jiake 5a240603fd [SPARK-30719][SQL] Add unit test to verify the log warning print when intentionally skip AQE
### What changes were proposed in this pull request?

This is a follow up in [#27452](https://github.com/apache/spark/pull/27452).
Add a unit test to verify whether the log warning is print when intentionally skip AQE.

### Why are the changes needed?

Add unit test

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

No

### How was this patch tested?

adding unit test

Closes #27515 from JkSelf/aqeLoggingWarningTest.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-10 21:48:00 +08:00
Kent Yao 58b9ca1e6f [SPARK-30592][SQL][FOLLOWUP] Add some round-trip test cases
### What changes were proposed in this pull request?

Add round-trip tests for CSV and JSON functions as  https://github.com/apache/spark/pull/27317#discussion_r376745135 asked.

### Why are the changes needed?

improve test coverage

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

no
### How was this patch tested?

add uts

Closes #27510 from yaooqinn/SPARK-30592-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-10 16:23:44 +09:00
Liang-Chi Hsieh 9f8172e96a Revert "[SPARK-29721][SQL] Prune unnecessary nested fields from Generate without Project
This reverts commit a0e63b61e7.

### What changes were proposed in this pull request?

This reverts the patch at #26978 based on gatorsmile's suggestion.

### Why are the changes needed?

Original patch #26978 has not considered a corner case. We may need to put more time on ensuring we can cover all cases.

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

No

### How was this patch tested?

Unit test.

Closes #27504 from viirya/revert-SPARK-29721.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-09 19:45:16 -08:00
Gengliang Wang b877aac146 [SPARK-30684 ][WEBUI][FollowUp] A new approach for SPARK-30684
### What changes were proposed in this pull request?

Simplify the changes for adding metrics description for WholeStageCodegen in https://github.com/apache/spark/pull/27405

### Why are the changes needed?

In https://github.com/apache/spark/pull/27405, the UI changes can be made without using the function `adjustPositionOfOperationName` to adjust the position of operation name and mark as an operation-name class.

I suggest we make simpler changes so that it would be easier for future development.

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

No

### How was this patch tested?

Manual test with the queries provided in https://github.com/apache/spark/pull/27405
```
sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").show
sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").write.format("json").mode("overwrite").save("/tmp/test_output")
sc.parallelize(1 to 10).toDF.write.format("json").mode("append").save("/tmp/test_output")
```
![image](https://user-images.githubusercontent.com/1097932/74073629-e3f09f00-49bf-11ea-90dc-1edb5ca29e5e.png)

Closes #27490 from gengliangwang/wholeCodegenUI.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-02-09 14:18:51 -08:00
Nicholas Chammas 339c0f9a62 [SPARK-30510][SQL][DOCS] Publicly document Spark SQL configuration options
### What changes were proposed in this pull request?

This PR adds a doc builder for Spark SQL's configuration options.

Here's what the new Spark SQL config docs look like ([configuration.html.zip](https://github.com/apache/spark/files/4172109/configuration.html.zip)):

![Screen Shot 2020-02-07 at 12 13 23 PM](https://user-images.githubusercontent.com/1039369/74050007-425b5480-49a3-11ea-818c-42700c54d1fb.png)

Compare this to the [current docs](http://spark.apache.org/docs/3.0.0-preview2/configuration.html#spark-sql):

![Screen Shot 2020-02-04 at 4 55 10 PM](https://user-images.githubusercontent.com/1039369/73790828-24a5a980-476f-11ea-998c-12cd613883e8.png)

### Why are the changes needed?

There is no visibility into the various Spark SQL configs on [the config docs page](http://spark.apache.org/docs/3.0.0-preview2/configuration.html#spark-sql).

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

No, apart from new documentation.

### How was this patch tested?

I tested this manually by building the docs and reviewing them in my browser.

Closes #27459 from nchammas/SPARK-30510-spark-sql-options.

Authored-by: Nicholas Chammas <nicholas.chammas@liveramp.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-09 19:20:47 +09:00
Yuanjian Li 3db3e39f11 [SPARK-28228][SQL] Change the default behavior for name conflict in nested WITH clause
### What changes were proposed in this pull request?
This is a follow-up for #25029, in this PR we throw an AnalysisException when name conflict is detected in nested WITH clause. In this way, the config `spark.sql.legacy.ctePrecedence.enabled` should be set explicitly for the expected behavior.

### Why are the changes needed?
The original change might risky to end-users, it changes behavior silently.

### Does this PR introduce any user-facing change?
Yes, change the config `spark.sql.legacy.ctePrecedence.enabled` as optional.

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

Closes #27454 from xuanyuanking/SPARK-28228-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-08 14:10:28 -08:00
Terry Kim a7451f44d2 [SPARK-30614][SQL] The native ALTER COLUMN syntax should change one property at a time
### What changes were proposed in this pull request?

The current ALTER COLUMN syntax allows to change multiple properties at a time:
```
ALTER TABLE table=multipartIdentifier
  (ALTER | CHANGE) COLUMN? column=multipartIdentifier
  (TYPE dataType)?
  (COMMENT comment=STRING)?
  colPosition?
```
The SQL standard (section 11.12) only allows changing one property at a time. This is also true on other recent SQL systems like [snowflake](https://docs.snowflake.net/manuals/sql-reference/sql/alter-table-column.html) and [redshift](https://docs.aws.amazon.com/redshift/latest/dg/r_ALTER_TABLE.html). (credit to cloud-fan)

This PR proposes to change ALTER COLUMN to follow SQL standard, thus allows altering only one column property at a time.

Note that ALTER COLUMN syntax being changed here is newly added in Spark 3.0, so it doesn't affect Spark 2.4 behavior.

### Why are the changes needed?

To follow SQL standard (and other recent SQL systems) behavior.

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

Yes, now the user can update the column properties only one at a time.

For example,
```
ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment'
```
should be broken into
```
ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint
ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment'
```

### How was this patch tested?

Updated existing tests.

Closes #27444 from imback82/alter_column_one_at_a_time.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-08 02:47:44 +08:00
Maxim Gekk a3e77773cf [SPARK-30752][SQL] Fix to_utc_timestamp on daylight saving day
### What changes were proposed in this pull request?
- Rewrite the `convertTz` method of `DateTimeUtils` using Java 8 time API
- Change types of `convertTz` parameters from `TimeZone` to `ZoneId`. This allows to avoid unnecessary conversions `TimeZone` -> `ZoneId` and performance regressions as a consequence.

### Why are the changes needed?
- Fixes incorrect behavior of `to_utc_timestamp` on daylight saving day. For example:
```scala
scala> df.select(to_utc_timestamp(lit("2019-11-03T12:00:00"), "Asia/Hong_Kong").as("local UTC")).show
+-------------------+
|          local UTC|
+-------------------+
|2019-11-03 03:00:00|
+-------------------+
```
but the result must be 2019-11-03 04:00:00:
<img width="1013" alt="Screen Shot 2020-02-06 at 20 09 36" src="https://user-images.githubusercontent.com/1580697/73960846-a129bb00-491c-11ea-92f5-45831cb28a62.png">

- Simplifies the code, and make it more maintainable
- Switches `convertTz` on Proleptic Gregorian calendar used by Java 8 time classes by default. That makes the function consistent to other date-time functions.

### Does this PR introduce any user-facing change?
Yes, after the changes `to_utc_timestamp` returns the correct result `2019-11-03 04:00:00`.

### How was this patch tested?
- By existing test suite `DateTimeUtilsSuite`, `DateFunctionsSuite` and `DateExpressionsSuite`.
- Added `convert time zones on a daylight saving day` to DateFunctionsSuite

Closes #27474 from MaxGekk/port-convertTz-on-Java8-api.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-08 02:32:07 +08:00
Wenchen Fan 5a4c70b4e2 [SPARK-27986][SQL][FOLLOWUP] window aggregate function with filter predicate is not supported
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/26656.

We don't support window aggregate function with filter predicate yet and we should fail explicitly.

Observable metrics has the same issue. This PR fixes it as well.

### Why are the changes needed?

If we simply ignore filter predicate when we don't support it, the result is wrong.

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

yea, fix the query result.

### How was this patch tested?

new tests

Closes #27476 from cloud-fan/filter.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-06 13:33:39 -08:00
Wenchen Fan 8ce58627eb [SPARK-30719][SQL] do not log warning if AQE is intentionally skipped and add a config to force apply
### What changes were proposed in this pull request?

Update `InsertAdaptiveSparkPlan` to not log warning if AQE is skipped intentionally.

This PR also add a config to not skip AQE.

### Why are the changes needed?

It's not a warning at all if we intentionally skip AQE.

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

no

### How was this patch tested?

run `AdaptiveQueryExecSuite` locally and verify that there is no warning logs.

Closes #27452 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-06 09:16:14 -08:00
yi.wu 368ee62a5d [SPARK-27297][DOC][FOLLOW-UP] Improve documentation for various Scala functions
### What changes were proposed in this pull request?

Add examples and parameter description for these Scala functions:

* transform
* exists
* forall
* aggregate
* zip_with
* transform_keys
* transform_values
* map_filter
* map_zip_with

### Why are the changes needed?

Better documentation for UX.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27449 from Ngone51/doc-funcs.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-06 20:34:29 +08:00
yi.wu 3f5b23340e [SPARK-30744][SQL] Optimize AnalyzePartitionCommand by calculating location sizes in parallel
### What changes were proposed in this pull request?

Use `CommandUtils.calculateTotalLocationSize` for `AnalyzePartitionCommand` in order to calculate location sizes in parallel.

### Why are the changes needed?

For better performance.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27471 from Ngone51/dev_calculate_in_parallel.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-06 20:20:44 +08:00
beliefer c8ef1dee90 [SPARK-29108][SQL][TESTS][FOLLOWUP] Comment out no use test case and add 'insert into' statement of window.sql (Part 2)
### What changes were proposed in this pull request?
When I running the `window_part2.sql` tests find it lack insert sql. Therefore, the output is empty.
I checked the postgresql and reference https://github.com/postgres/postgres/blob/master/src/test/regress/sql/window.sql
Although `window_part1.sql` and `window_part3.sql` exists the insert sql, I think should also add it into `window_part2.sql`.
Because only one case reference the table `empsalary` and it throws `AnalysisException`.
```
-- !query
select last(salary) over(order by salary range between 1000 preceding and 1000 following),
lag(salary) over(order by salary range between 1000 preceding and 1000 following),
salary from empsalary
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
Window Frame specifiedwindowframe(RangeFrame, -1000, 1000) must match the required frame specifiedwindowframe(RowFrame, -1, -1);
```

So we should do four work:
1. comment out the only one case and create a new ticket.
2. Add `INSERT INTO empsalary`.

Note: window_part4.sql not use the table `empsalary`.

### Why are the changes needed?
Supplementary test data.

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

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

Closes #27439 from beliefer/add-insert-to-window.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-06 15:24:26 +09:00
Terry Kim c27a616450 [SPARK-30612][SQL] Resolve qualified column name with v2 tables
### What changes were proposed in this pull request?

This PR fixes the issue where queries with qualified columns like `SELECT t.a FROM t` would fail to resolve for v2 tables.

This PR would allow qualified column names in query as following:
```SQL
SELECT testcat.ns1.ns2.tbl.foo FROM testcat.ns1.ns2.tbl
SELECT ns1.ns2.tbl.foo FROM testcat.ns1.ns2.tbl
SELECT ns2.tbl.foo FROM testcat.ns1.ns2.tbl
SELECT tbl.foo FROM testcat.ns1.ns2.tbl
```

### Why are the changes needed?

This is a bug because you cannot qualify column names in queries.

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

Yes, now users can qualify column names for v2 tables.

### How was this patch tested?

Added new tests.

Closes #27391 from imback82/qualified_col.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-06 13:54:17 +08:00
Wenchen Fan 3b26f807a0 [SPARK-30721][SQL][TESTS] Fix DataFrameAggregateSuite when enabling AQE
### What changes were proposed in this pull request?

update `DataFrameAggregateSuite` to make it pass with AQE

### Why are the changes needed?

We don't need to turn off AQE in `DataFrameAggregateSuite`

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

no

### How was this patch tested?

run `DataFrameAggregateSuite` locally with AQE on.

Closes #27451 from cloud-fan/aqe-test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-05 12:36:51 -08:00
Yuanjian Li 4938905a1c [SPARK-29864][SQL][FOLLOWUP] Reference the config for the old behavior in error message
### What changes were proposed in this pull request?
Follow up work for SPARK-29864, reference the config  `spark.sql.legacy.fromDayTimeString.enabled` in error message.

### Why are the changes needed?
For better usability.

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

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

Closes #27464 from xuanyuanking/SPARK-29864-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-05 11:19:42 -08:00
turbofei 6d507b4a31 [SPARK-26218][SQL][FOLLOW UP] Fix the corner case when casting float to Integer
### What changes were proposed in this pull request?
When spark.sql.ansi.enabled is true, for the statement:
```
select cast(cast(2147483648 as Float) as Integer) //result is 2147483647
```
Its result is 2147483647 and does not throw `ArithmeticException`.

The root cause is that, the below code does not work for some corner cases.
94fc0e3235/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala (L129-L141)

For example:

![image](https://user-images.githubusercontent.com/6757692/72074911-badfde80-332d-11ea-963e-2db0e43c33e8.png)

In this PR, I fix it by comparing Math.floor(x) with Int.MaxValue directly.

### Why are the changes needed?
Result corrupt.

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

### How was this patch tested?

Added Unit test.

Closes #27151 from turboFei/SPARK-26218-follow-up-int-overflow.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-05 21:24:02 +08:00
Maxim Gekk 459e757ed4 [SPARK-30668][SQL] Support SimpleDateFormat patterns in parsing timestamps/dates strings
### What changes were proposed in this pull request?
In the PR, I propose to partially revert the commit 51a6ba0181, and provide a legacy parser based on `FastDateFormat` which is compatible to `SimpleDateFormat`.

To enable the legacy parser, set `spark.sql.legacy.timeParser.enabled` to `true`.

### Why are the changes needed?
To allow users to restore old behavior in parsing timestamps/dates using `SimpleDateFormat` patterns. The main reason for restoring is `DateTimeFormatter`'s patterns are not fully compatible to `SimpleDateFormat` patterns, see https://issues.apache.org/jira/browse/SPARK-30668

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

### How was this patch tested?
- Added new test to `DateFunctionsSuite`
- Restored additional test cases in `JsonInferSchemaSuite`.

Closes #27441 from MaxGekk/support-simpledateformat.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-05 18:48:45 +08:00
Liang-Chi Hsieh 7631275f97 [SPARK-25040][SQL][FOLLOWUP] Add legacy config for allowing empty strings for certain types in json parser
### What changes were proposed in this pull request?

This is a follow-up for #22787. In #22787 we disallowed empty strings for json parser except for string and binary types. This follow-up adds a legacy config for restoring previous behavior of allowing empty string.

### Why are the changes needed?

Adding a legacy config to make migration easy for Spark users.

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

Yes. If set this legacy config to true, the users can restore previous behavior prior to Spark 3.0.0.

### How was this patch tested?

Unit test.

Closes #27456 from viirya/SPARK-25040-followup.

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-02-04 17:22:23 -08:00
maryannxue 6097b343ba [SPARK-30717][SQL] AQE subquery map should cache SubqueryExec instead of ExecSubqueryExpression
### What changes were proposed in this pull request?
This PR is to fix a potential bug in AQE where an `ExecSubqueryExpression` could be mistakenly replaced with another `ExecSubqueryExpression` with the same `ListQuery` but a different `child` expression.
This is because a ListQuery's id can only identify the ListQuery itself, not the parent expression `InSubquery`, but right now the `subqueryMap` in `InsertAdaptiveSparkPlan` uses the `ListQuery`'s id as key and the corresponding `InSubqueryExec` for the `ListQuery`'s parent expression as value. So the fix uses the corresponding `SubqueryExec` for the `ListQuery` itself as the map's value.

### Why are the changes needed?
This logical bug could potentially cause a wrong query plan, which could throw an exception related to unresolved columns.

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

### How was this patch tested?
Passed existing UTs.

Closes #27446 from maryannxue/spark-30717.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-04 12:31:44 +08:00
Yuanjian Li a4912cee61
[SPARK-29543][SS][FOLLOWUP] Move spark.sql.streaming.ui.* configs to StaticSQLConf
### What changes were proposed in this pull request?
Put the configs below needed by Structured Streaming UI into StaticSQLConf:

- spark.sql.streaming.ui.enabled
- spark.sql.streaming.ui.retainedProgressUpdates
- spark.sql.streaming.ui.retainedQueries

### Why are the changes needed?
Make all SS UI configs consistent with other similar configs in usage and naming.

### Does this PR introduce any user-facing change?
Yes, add new static config `spark.sql.streaming.ui.retainedProgressUpdates`.

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

Closes #27425 from xuanyuanking/SPARK-29543-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-02-02 23:37:13 -08:00
Burak Yavuz 2eccfd8a73 [SPARK-30697][SQL] Handle database and namespace exceptions in catalog.isView
### What changes were proposed in this pull request?

Adds NoSuchDatabaseException and NoSuchNamespaceException to the `isView` method for SessionCatalog.

### Why are the changes needed?

This method prevents specialized resolutions from kicking in within Analysis when using V2 Catalogs if the identifier is a specialized identifier.

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

No

### How was this patch tested?

Added test to DataSourceV2SessionCatalogSuite

Closes #27423 from brkyvz/isViewF.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-03 14:08:59 +08:00
Liang-Chi Hsieh 8eecc20b11 [SPARK-27946][SQL] Hive DDL to Spark DDL conversion USING "show create table"
## What changes were proposed in this pull request?

This patch adds a DDL command `SHOW CREATE TABLE AS SERDE`. It is used to generate Hive DDL for a Hive table.

For original `SHOW CREATE TABLE`, it now shows Spark DDL always. If given a Hive table, it tries to generate Spark DDL.

For Hive serde to data source conversion, this uses the existing mapping inside `HiveSerDe`. If can't find a mapping there, throws an analysis exception on unsupported serde configuration.

It is arguably that some Hive fileformat + row serde might be mapped to Spark data source, e.g., CSV. It is not included in this PR. To be conservative, it may not be supported.

For Hive serde properties, for now this doesn't save it to Spark DDL because it may not useful to keep Hive serde properties in Spark table.

## How was this patch tested?

Added test.

Closes #24938 from viirya/SPARK-27946.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-31 19:55:25 -08:00
yi.wu 82b4f753a0 [SPARK-30508][SQL] Add SparkSession.executeCommand API for external datasource
### What changes were proposed in this pull request?

This PR adds `SparkSession.executeCommand` API for external datasource to execute a random command like

```
val df = spark.executeCommand("xxxCommand", "xxxSource", "xxxOptions")
```
Note that the command doesn't execute in Spark, but inside an external execution engine depending on data source. And it will be eagerly executed after `executeCommand` called and the returned `DataFrame` will contain the output of the command(if any).

### Why are the changes needed?

This can be useful when user wants to execute some commands out of Spark. For example, executing custom DDL/DML command for JDBC, creating index for ElasticSearch, creating cores for Solr and so on(as HyukjinKwon suggested).

Previously, user needs to use an option to achieve the goal, e.g. `spark.read.format("xxxSource").option("command", "xxxCommand").load()`, which is kind of cumbersome. With this change, it can be more convenient for user to achieve the same goal.

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

Yes, new API from `SparkSession` and a new interface `ExternalCommandRunnableProvider`.

### How was this patch tested?

Added a new test suite.

Closes #27199 from Ngone51/dev-executeCommand.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-31 15:05:26 -08:00
Maxim Gekk 2d4b5eaee4 [SPARK-30676][CORE][TESTS] Eliminate warnings from deprecated constructors of java.lang.Integer and java.lang.Double
### What changes were proposed in this pull request?
- Replace `new Integer(0)` by a serializable instance in RDD.scala
- Use `.valueOf()` instead of constructors of `java.lang.Integer` and `java.lang.Double` because constructors has been deprecated, see https://docs.oracle.com/javase/9/docs/api/java/lang/Integer.html

### Why are the changes needed?
This fixes the following warnings:
1. RDD.scala:240: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
2. MutableProjectionSuite.scala:63: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
3. UDFSuite.scala:446: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
4. UDFSuite.scala:451: constructor Double in class Double is deprecated: see corresponding Javadoc for more information.
5. HiveUserDefinedTypeSuite.scala:71: constructor Double in class Double is deprecated: see corresponding Javadoc for more information.

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

### How was this patch tested?
- By RDDSuite, MutableProjectionSuite, UDFSuite and HiveUserDefinedTypeSuite

Closes #27399 from MaxGekk/eliminate-warning-part4.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-31 15:03:16 -06:00
Kousuke Saruta 18bc4e55ef [SPARK-30684][WEBUI] Show the descripton of metrics for WholeStageCodegen in DAG viz
### What changes were proposed in this pull request?

Added description for metrics shown in the WholeStageCodegen-node in DAG viz.

This is before the change is applied.
![before-changed](https://user-images.githubusercontent.com/4736016/73469870-5cf16480-43ca-11ea-9a13-714083508a3b.png)

And following is after change.
![after-fixing-layout](https://user-images.githubusercontent.com/4736016/73469364-983f6380-43c9-11ea-8b7e-ddab030d0270.png)

For this change, I also modify the layout of DAG viz.
Actually, I noticed  it's not enough to just added the description.
Following is without changing the layout.
![layout-is-broken](https://user-images.githubusercontent.com/4736016/73470178-cffadb00-43ca-11ea-86d7-aed109b105e6.png)

### Why are the changes needed?

Users can't understand what those metrics mean.

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

Yes. The layout is a little bit changed.

### How was this patch tested?

I confirm the result of DAG viz with following 3 operations.

`sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").show`
`sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").write.format("json").mode("overwrite").save("/tmp/test_output")`
`sc.parallelize(1 to 10).toDF.write.format("json").mode("append").save("/tmp/test_output")`

Closes #27405 from sarutak/sql-dag-metrics.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-31 11:58:52 -08:00
Wenchen Fan 33546d637d Revert "[SPARK-30036][SQL] Fix: REPARTITION hint does not work with order by"
This reverts commit a2de20c0e6.
2020-02-01 03:02:52 +08:00
Jungtaek Lim (HeartSaVioR) 5e0faf9a3d [SPARK-29779][SPARK-30479][CORE][SQL][FOLLOWUP] Reflect review comments on post-hoc review
### What changes were proposed in this pull request?

This PR reflects review comments on post-hoc review among PRs for SPARK-29779 (#27085), SPARK-30479 (#27164). The list of review comments this PR addresses are below:

* https://github.com/apache/spark/pull/27085#discussion_r373304218
* https://github.com/apache/spark/pull/27164#discussion_r373300793
* https://github.com/apache/spark/pull/27164#discussion_r373301193
* https://github.com/apache/spark/pull/27164#discussion_r373301351

I also applied review comments to the CORE module (BasicEventFilterBuilder.scala) as well, as the review comments for SQL/core module (SQLEventFilterBuilder.scala) can be applied there as well.

### Why are the changes needed?

There're post-hoc reviews on PRs for such issues, like links in above section.

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

No

### How was this patch tested?

Existing UTs.

Closes #27414 from HeartSaVioR/SPARK-28869-SPARK-29779-SPARK-30479-FOLLOWUP-posthoc-reviews.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-31 10:17:07 -08:00
Tathagata Das 481e5211d2
[SPARK-30657][SPARK-30658][SS] Fixed two bugs in streaming limits
This PR solves two bugs related to streaming limits

**Bug 1 (SPARK-30658)**: Limit before a streaming aggregate (i.e. `df.limit(5).groupBy().count()`) in complete mode was not being planned as a stateful streaming limit. The planner rule planned a logical limit with a stateful streaming limit plan only if the query is in append mode. As a result, instead of allowing max 5 rows across batches, the planned streaming query was allowing 5 rows in every batch thus producing incorrect results.

**Solution**: Change the planner rule to plan the logical limit with a streaming limit plan even when the query is in complete mode if the logical limit has no stateful operator before it.

**Bug 2 (SPARK-30657)**: `LocalLimitExec` does not consume the iterator of the child plan. So if there is a limit after a stateful operator like streaming dedup in append mode (e.g. `df.dropDuplicates().limit(5)`), the state changes of streaming duplicate may not be committed (most stateful ops commit state changes only after the generated iterator is fully consumed).

**Solution**: Change the planner rule to always use a new `StreamingLocalLimitExec` which always fully consumes the iterator. This is the safest thing to do. However, this will introduce a performance regression as consuming the iterator is extra work. To minimize this performance impact, add an additional post-planner optimization rule to replace `StreamingLocalLimitExec` with `LocalLimitExec` when there is no stateful operator before the limit that could be affected by it.

No

Updated incorrect unit tests and added new ones

Closes #27373 from tdas/SPARK-30657.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-01-31 09:27:34 -08:00
yi.wu 5ccbb38a71 [SPARK-29938][SQL][FOLLOW-UP] Improve AlterTableAddPartitionCommand
All credit to Ngone51, Closes #27293.
### What changes were proposed in this pull request?
This PR improves `AlterTableAddPartitionCommand` by:
1. adds an internal config for partitions batch size to avoid hard code
2. reuse `InMemoryFileIndex.bulkListLeafFiles` to perform parallel file listing to improve code reuse

### Why are the changes needed?
Improve code quality.

### Does this PR introduce any user-facing change?
Yes. We renamed `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `spark.sql.parallelFileListingInCommands.enabled` as a side effect of this change.

### How was this patch tested?
Pass Jenkins.

Closes #27413 from xuanyuanking/SPARK-29938.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-01 01:03:00 +08:00
Burak Yavuz 290a528bff [SPARK-30615][SQL] Introduce Analyzer rule for V2 AlterTable column change resolution
### What changes were proposed in this pull request?

Adds an Analyzer rule to normalize the column names used in V2 AlterTable table changes. We need to handle all ColumnChange operations. We add an extra match statement for future proofing new changes that may be added. This prevents downstream consumers (e.g. catalogs) to deal about case sensitivity or check that columns exist, etc.

We also fix the behavior for ALTER TABLE CHANGE COLUMN (Hive style syntax) for adding comments to complex data types. Currently, the data type needs to be provided as part of the Hive style syntax. This assumes that the data type as changed when it may have not and the user only wants to add a comment, which fails in CheckAnalysis.

### Why are the changes needed?

Currently we do not handle case sensitivity correctly for ALTER TABLE ALTER COLUMN operations.

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

No, fixes a bug.

### How was this patch tested?

Introduced v2CommandsCaseSensitivitySuite and added a test around HiveStyle Change columns to PlanResolutionSuite

Closes #27350 from brkyvz/normalizeAlter.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-31 16:41:10 +08:00
herman a5c7090ffa [SPARK-30671][SQL] emptyDataFrame should use a LocalRelation
### What changes were proposed in this pull request?
This PR makes `SparkSession.emptyDataFrame` use an empty local relation instead of an empty RDD. This allows to optimizer to recognize this as an empty relation, and creates the opportunity to do some more aggressive optimizations.

### Why are the changes needed?
It allows us to optimize empty dataframes better.

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

### How was this patch tested?
Added a test case to `DataFrameSuite`.

Closes #27400 from hvanhovell/SPARK-30671.

Authored-by: herman <herman@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-31 16:14:07 +09:00
Burak Yavuz 1cd19ad92d [SPARK-30669][SS] Introduce AdmissionControl APIs for StructuredStreaming
### What changes were proposed in this pull request?

We propose to add a new interface `SupportsAdmissionControl` and `ReadLimit`. A ReadLimit defines how much data should be read in the next micro-batch. `SupportsAdmissionControl` specifies that a source can rate limit its ingest into the system. The source can tell the system what the user specified as a read limit, and the system can enforce this limit within each micro-batch or impose its own limit if the Trigger is Trigger.Once() for example.

We then use this interface in FileStreamSource, KafkaSource, and KafkaMicroBatchStream.

### Why are the changes needed?

Sources currently have no information around execution semantics such as whether the stream is being executed in Trigger.Once() mode. This interface will pass this information into the sources as part of planning. With a trigger like Trigger.Once(), the semantics are to process all the data available to the datasource in a single micro-batch. However, this semantic can be broken when data source options such as `maxOffsetsPerTrigger` (in the Kafka source) rate limit the amount of data read for that micro-batch without this interface.

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

DataSource developers can extend this interface for their streaming sources to add admission control into their system and correctly support Trigger.Once().

### How was this patch tested?

Existing tests, as this API is mostly internal

Closes #27380 from brkyvz/rateLimit.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-01-30 22:02:48 -08:00
sandeep katta 5f3ec6250f [SPARK-30362][CORE] Update InputMetrics in DataSourceRDD
### What changes were proposed in this pull request?
Incase of DS v2 InputMetrics are not updated

**Before Fix**
![inputMetrics](https://user-images.githubusercontent.com/35216143/71501010-c216df00-288d-11ea-8522-fdd50b13eae1.png)

**After Fix** we can see that `Input Size / Records` is updated in the UI
![image](https://user-images.githubusercontent.com/35216143/71501000-b88d7700-288d-11ea-92fe-a727b2b79908.png)

### Why are the changes needed?
InputMetrics like bytesread and recordread should be updated

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

### How was this patch tested?
Added UT and also verified manually

Closes #27021 from sandeep-katta/dsv2inputmetrics.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-31 14:01:32 +08:00
Wenchen Fan 9f42be25eb [SPARK-29665][SQL] refine the TableProvider interface
### What changes were proposed in this pull request?

Instead of having several overloads of `getTable` method in `TableProvider`, it's better to have 2 methods explicitly: `inferSchema` and `inferPartitioning`. With a single `getTable` method that takes everything: schema, partitioning and properties.

This PR also adds a `supportsExternalMetadata` method in `TableProvider`, to indicate if the source support external table metadata. If this flag is false:
1. spark.read.schema... is disallowed and fails
2. when we support creating v2 tables in session catalog,  spark only keeps table properties in the catalog.

### Why are the changes needed?

API improvement.

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

no

### How was this patch tested?

existing tests

Closes #26868 from cloud-fan/provider2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-31 13:37:43 +08:00
Jungtaek Lim (HeartSaVioR) cbb714f67e [SPARK-29438][SS] Use partition ID of StateStoreAwareZipPartitionsRDD for determining partition ID of state store in stream-stream join
### What changes were proposed in this pull request?

Credit to uncleGen for discovering the problem and providing simple reproducer as UT. New UT in this patch is borrowed from #26156 and I'm retaining a commit from #26156 (except unnecessary part on this path) to properly give a credit.

This patch fixes the issue that partition ID could be mis-assigned when the query contains UNION and stream-stream join is placed on the right side. We assume the range of partition IDs as `(0 ~ number of shuffle partitions - 1)` for stateful operators, but when we use stream-stream join on the right side of UNION, the range of partition ID of task goes to `(number of partitions in left side, number of partitions in left side + number of shuffle partitions - 1)`, which `number of partitions in left side` can be changed in some cases (new UT points out the one of the cases).

The root reason of bug is that stream-stream join picks the partition ID from TaskContext, which wouldn't be same as partition ID from source if union is being used. Hopefully we can pick the right partition ID from source in StateStoreAwareZipPartitionsRDD - this patch leverages that partition ID.

### Why are the changes needed?

This patch will fix the broken of assumption of partition range on stateful operator, as well as fix the issue reported in JIRA issue SPARK-29438.

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

Yes, if their query is using UNION and stream-stream join is placed on the right side. They may encounter the problem to read state from checkpoint and may need to discard checkpoint to continue.

### How was this patch tested?

Added UT which fails on current master branch, and passes with this patch.

Closes #26162 from HeartSaVioR/SPARK-29438.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2020-01-30 20:21:43 -08:00
Wenchen Fan e5f572af06 [SPARK-30680][SQL] ResolvedNamespace does not require a namespace catalog
### What changes were proposed in this pull request?

Update `ResolvedNamespace` to accept catalog as `CatalogPlugin` not `SupportsNamespaces`.

This is extracted from https://github.com/apache/spark/pull/27345

### Why are the changes needed?

not all commands that need to resolve namespaces require a namespace catalog. For example, `SHOW TABLE` is implemented by `TableCatalog.listTables`, and is nothing to do with `SupportsNamespace`.

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

no

### How was this patch tested?

existing tests

Closes #27403 from cloud-fan/ns.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-30 10:34:59 -08:00
zero323 b1f81f0072 [MINOR][SQL][DOCS] Fix typos in scaladoc strings of higher order functions
### What changes were proposed in this pull request?

Fix following typos:

- tranformation -> transformation
- the boolean -> the Boolean
- signle -> single

### Why are the changes needed?

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

No

### How was this patch tested?

Scala linter.

Closes #27382 from zero323/functions-typos.

Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-29 18:42:18 -06:00
uncleGen 7173786153
[SPARK-29543][SS][UI] Structured Streaming Web UI
### What changes were proposed in this pull request?

This PR adds two pages to Web UI for Structured Streaming:
   - "/streamingquery": Streaming Query Page, providing some aggregate information for running/completed streaming queries.
  - "/streamingquery/statistics": Streaming Query Statistics Page, providing detailed information for streaming query, including `Input Rate`, `Process Rate`, `Input Rows`, `Batch Duration` and `Operation Duration`

![Screen Shot 2020-01-29 at 1 38 00 PM](https://user-images.githubusercontent.com/1000778/73399837-cd01cc80-429c-11ea-9d4b-1d200a41b8d5.png)
![Screen Shot 2020-01-29 at 1 39 16 PM](https://user-images.githubusercontent.com/1000778/73399838-cd01cc80-429c-11ea-8185-4e56db6866bd.png)

### Why are the changes needed?

It helps users to better monitor Structured Streaming query.

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

No

### How was this patch tested?

- new added and existing UTs
- manual test

Closes #26201 from uncleGen/SPARK-29543.

Lead-authored-by: uncleGen <hustyugm@gmail.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Genmao Yu <hustyugm@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-01-29 13:43:51 -08:00
Takeshi Yamamuro ec1fb6b4e1 [SPARK-30234][SQL][FOLLOWUP] Add .enabled in the suffix of the ADD FILE legacy option
### What changes were proposed in this pull request?

This pr intends to rename `spark.sql.legacy.addDirectory.recursive` into `spark.sql.legacy.addDirectory.recursive.enabled`.

### Why are the changes needed?

For consistent option names.

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

No.

### How was this patch tested?

N/A

Closes #27372 from maropu/SPARK-30234-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-29 12:23:59 +09:00
Maxim Gekk 8aebc80e0e [SPARK-30625][SQL] Support escape as third parameter of the like function
### What changes were proposed in this pull request?
In the PR, I propose to transform the `Like` expression to `TernaryExpression`, and add third parameter `escape`. So, the `like` function will have feature parity with `LIKE ... ESCAPE` syntax supported by 187f3c1773.

### Why are the changes needed?
The `like` functions can be called with 2 or 3 parameters, and functionally equivalent to `LIKE` and `LIKE ... ESCAPE` SQL expressions.

### Does this PR introduce any user-facing change?
Yes, before `like` fails with the exception:
```sql
spark-sql> SELECT like('_Apache Spark_', '__%Spark__', '_');
Error in query: Invalid number of arguments for function like. Expected: 2; Found: 3; line 1 pos 7
```
After:
```sql
spark-sql> SELECT like('_Apache Spark_', '__%Spark__', '_');
true
```

### How was this patch tested?
- Add new example for the `like` function which is checked by `SQLQuerySuite`
- Run `RegexpExpressionsSuite` and `ExpressionParserSuite`.

Closes #27355 from MaxGekk/like-3-args.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-27 11:19:32 -08:00
Jungtaek Lim (HeartSaVioR) 0436b3d3f8 [SPARK-30653][INFRA][SQL] EOL character enforcement for java/scala/xml/py/R files
### What changes were proposed in this pull request?

This patch converts CR/LF into LF in 3 source files, which most files are only using LF. This patch also add rules to enforce EOL as LF for all java, scala, xml, py, R files.

### Why are the changes needed?

The majority of source code files are using LF and only three files are CR/LF. While using IDE would let us don't bother with the difference, it still has a chance to make unnecessary diff if the file is modified with the editor which doesn't handle it automatically.

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

No

### How was this patch tested?

```
grep -IUrl --color "^M" . | grep "\.java\|\.scala\|\.xml\|\.py\|\.R" | grep -v "/target/" | grep -v "/build/" | grep -v "/dist/" | grep -v "dependency-reduced-pom.xml" | grep -v ".pyc"
```

(Please note you'll need to type CTRL+V -> CTRL+M in bash shell to get `^M` because it's representing CR/LF, not a combination of `^` and `M`.)

Before the patch, the result is:

```
./sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java
./sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
```

and after the patch, the result is None.

And git shows WARNING message if EOL of any of source files in given types are modified to CR/LF, like below:

```
warning: CRLF will be replaced by LF in sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala.
The file will have its original line endings in your working directory.
```

Closes #27365 from HeartSaVioR/MINOR-remove-CRLF-in-source-codes.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-27 10:20:51 -08:00
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
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
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
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
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
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
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
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
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
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
Kent Yao 82f25f5855 [SPARK-30507][SQL] TableCalalog reserved properties shoudn't be changed via options or tblpropeties
### What changes were proposed in this pull request?

TableCatalog reserves some properties, e,g `provider`, `location` for internal usage. Some of them are static once create, some of them need specific syntax to modify. Instead of using `OPTIONS (k='v')` or TBLPROPERTIES (k='v'), if k is a reserved TableCatalog property, we should use its specific syntax to add/modify/delete it. e.g. `provider` is a reserved property, we should use the `USING` clause to specify it, and should not allow `ALTER TABLE ... UNSET TBLPROPERTIES('provider')` to delete it. Also, there are two paths for v1/v2 catalog tables to resolve these properties, e.g. the v1 session catalog tables will only use the `USING` clause to decide `provider` but v2 tables will also lookup OPTION/TBLPROPERTIES(although there is a bug prohibit it).

Additionally, 'path' is not reserved but holds special meaning for `LOCATION` and it is used in `CREATE/REPLACE TABLE`'s `OPTIONS` sub-clause. Now for the session catalog tables, the `path` is case-insensitive, but for the non-session catalog tables, it is case-sensitive, we should make it both case insensitive for disambiguation.

### Why are the changes needed?
prevent reserved properties from being modified unexpectedly
unify the property resolution for v1/v2.
fix some bugs.

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

yes
1 . `location` and `provider` (case sensitive) cannot be used in  `CREATE/REPLACE TABLE ... OPTIONS/TBLPROPETIES` and `ALTER TABLE ... SET TBLPROPERTIES (...)`, if legacy on, they will be ignored to let the command success without having side effects
3. Once `path` in `CREATE/REPLACE TABLE ... OPTIONS`  is case insensitive for v1 but sensitive for v2, but now we change it case insensitive for both kinds of tables, then v2 tables will also fail if `LOCATION` and `OPTIONS('PaTh' ='abc')` are both specified or will pick `PaTh`'s value as table location if `LOCATION` is missing.
4. Now we will detect if there are two different case `path` keys or more in  `CREATE/REPLACE TABLE ... OPTIONS`, once it is a kind of unexpected last-win policy for v1, and v2 is case sensitive.

### How was this patch tested?

add ut

Closes #27197 from yaooqinn/SPARK-30507.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-16 21:46:07 +08:00
Maxim Gekk 018bdcc53c [SPARK-30521][SQL][TESTS] Eliminate deprecation warnings for ExpressionInfo
### What changes were proposed in this pull request?
In the PR, I propose to use non-deprecated constructor of `ExpressionInfo` in `SparkSessionExtensionSuite`, and pass valid strings as `examples`, `note`, `since` and `deprecated` parameters.

### Why are the changes needed?
Using another constructor allows to eliminate the following deprecation warnings while compiling Spark:
```
Warning:(335, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information.
    new ExpressionInfo("noClass", "myDb", "myFunction", "usage", "extended usage"),
Warning:(732, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information.
    new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"),
Warning:(751, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information.
    new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"),
```

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

### How was this patch tested?
By compiling and running `SparkSessionExtensionSuite`.

Closes #27221 from MaxGekk/eliminate-expr-info-warnings.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-16 13:36:28 +09:00
Maxim Gekk 4e50f0291f [SPARK-30323][SQL] Support filters pushdown in CSV datasource
### What changes were proposed in this pull request?

In the PR, I propose to support pushed down filters in CSV datasource. The reason of pushing a filter up to `UnivocityParser` is to apply the filter as soon as all its attributes become available i.e. converted from CSV fields to desired values according to the schema. This allows to skip conversions of other values if the filter returns `false`. This can improve performance when pushed filters are highly selective and conversion of CSV string fields to desired values are comparably expensive ( for example, conversion to `TIMESTAMP` values).

Here are details of the implementation:
- `UnivocityParser.convert()` converts parsed CSV tokens one-by-one sequentially starting from index 0 up to `parsedSchema.length - 1`. At current index `i`, it applies filters that refer to attributes at row fields indexes `0..i`. If any filter returns `false`, it skips conversions of other input tokens.
- Pushed filters are converted to expressions. The expressions are bound to row positions according to `requiredSchema`. The expressions are compiled to predicates via generating Java code.
- To be able to apply predicates to partially initialized rows, the predicates are grouped, and combined via the `And` expression. Final predicate at index `N` can refer to row fields at the positions `0..N`, and can be applied to a row even if other fields at the positions `N+1..requiredSchema.lenght-1` are not set.

### Why are the changes needed?
The changes improve performance on synthetic benchmarks more **than 9 times** (on JDK 8 & 11):
```
OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2
Intel(R) Core(TM) i7-4850HQ CPU  2.30GHz
Filters pushdown:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
w/o filters                                       11889          11945          52          0.0      118893.1       1.0X
pushdown disabled                                 11790          11860         115          0.0      117902.3       1.0X
w/ filters                                         1240           1278          33          0.1       12400.8       9.6X
```

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

### How was this patch tested?
- Added new test suite `CSVFiltersSuite`
- Added tests to `CSVSuite` and `UnivocityParserSuite`

Closes #26973 from MaxGekk/csv-filters-pushdown.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-16 13:10:08 +09:00
Takeshi Yamamuro a3a42b30d0 [SPARK-27986][SQL][FOLLOWUP] Respect filter in sql/toString of AggregateExpression
### What changes were proposed in this pull request?

This pr intends to add filter information in the explain output of an aggregate (This is a follow-up of #26656).

Without this pr:
```
scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").explain(true)
== Parsed Logical Plan ==
'Aggregate ['k], ['k, unresolvedalias('SUM('v, ('v > 3)), None)]
+- 'UnresolvedRelation [t]

== Analyzed Logical Plan ==
k: int, sum(v): bigint
Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) AS sum(v)#3L]
+- SubqueryAlias `default`.`t`
   +- Relation[k#0,v#1] parquet

== Optimized Logical Plan ==
Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) AS sum(v)#3L]
+- Relation[k#0,v#1] parquet

== Physical Plan ==
HashAggregate(keys=[k#0], functions=[sum(cast(v#1 as bigint))], output=[k#0, sum(v)#3L])
+- Exchange hashpartitioning(k#0, 200), true, [id=#20]
   +- HashAggregate(keys=[k#0], functions=[partial_sum(cast(v#1 as bigint))], output=[k#0, sum#7L])
      +- *(1) ColumnarToRow
         +- FileScan parquet default.t[k#0,v#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-master/spark-warehouse/t], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<k:int,v:int>

scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").show()
+---+------+
|  k|sum(v)|
+---+------+
+---+------+
```

With this pr:
```
scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").explain(true)
== Parsed Logical Plan ==
'Aggregate ['k], ['k, unresolvedalias('SUM('v, ('v > 3)), None)]
+- 'UnresolvedRelation [t]

== Analyzed Logical Plan ==
k: int, sum(v) FILTER (v > 3): bigint
Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) filter (v#1 > 3) AS sum(v) FILTER (v > 3)#5L]
+- SubqueryAlias `default`.`t`
   +- Relation[k#0,v#1] parquet

== Optimized Logical Plan ==
Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) filter (v#1 > 3) AS sum(v) FILTER (v > 3)#5L]
+- Relation[k#0,v#1] parquet

== Physical Plan ==
HashAggregate(keys=[k#0], functions=[sum(cast(v#1 as bigint))], output=[k#0, sum(v) FILTER (v > 3)#5L])
+- Exchange hashpartitioning(k#0, 200), true, [id=#20]
   +- HashAggregate(keys=[k#0], functions=[partial_sum(cast(v#1 as bigint)) filter (v#1 > 3)], output=[k#0, sum#9L])
      +- *(1) ColumnarToRow
         +- FileScan parquet default.t[k#0,v#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-master/spark-warehouse/t], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<k:int,v:int>

scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").show()
+---+---------------------+
|  k|sum(v) FILTER (v > 3)|
+---+---------------------+
+---+---------------------+
```

### Why are the changes needed?

For better usability.

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

No.

### How was this patch tested?

Manually.

Closes #27198 from maropu/SPARK-27986-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-16 11:11:36 +09:00
Wenchen Fan 883ae331c3 [SPARK-30497][SQL] migrate DESCRIBE TABLE to the new framework
### What changes were proposed in this pull request?

Use the new framework to resolve the DESCRIBE TABLE command.

The v1 DESCRIBE TABLE command supports both table and view. Checked with Hive and Presto, they don't have DESCRIBE TABLE syntax but only DESCRIBE, which supports both table and view:
1. https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-DescribeTable/View/MaterializedView/Column
2. https://prestodb.io/docs/current/sql/describe.html

We should make it clear that DESCRIBE support both table and view, by renaming the command to `DescribeRelation`.

This PR also tunes the framework a little bit to support the case that a command accepts both table and view.

### Why are the changes needed?

This is a part of effort to make the relation lookup behavior consistent: SPARK-29900.

Note that I make a separate PR here instead of #26921, as I need to update the framework to support a new use case: accept both table and view.

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

no

### How was this patch tested?

existing tests

Closes #27187 from cloud-fan/describe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-15 17:38:52 -08:00
Jungtaek Lim (HeartSaVioR) e751bc66a0 [SPARK-30479][SQL] Apply compaction of event log to SQL events
### What changes were proposed in this pull request?

This patch addresses adding event filter to handle SQL related events. This patch is next task of SPARK-29779 (#27085), please refer the description of PR #27085 to see overall rationalization of this patch.

Below functionalities will be addressed in later parts:

* integrate compaction into FsHistoryProvider
* documentation about new configuration

### Why are the changes needed?

One of major goal of SPARK-28594 is to prevent the event logs to become too huge, and SPARK-29779 achieves the goal. We've got another approach in prior, but the old approach required models in both KVStore and live entities to guarantee compatibility, while they're not designed to do so.

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

No.

### How was this patch tested?

Added UTs.

Closes #27164 from HeartSaVioR/SPARK-30479.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-15 10:47:31 -08:00
Takeshi Yamamuro 5f6cd61913 [SPARK-29708][SQL] Correct aggregated values when grouping sets are duplicated
### What changes were proposed in this pull request?

This pr intends to fix wrong aggregated values in `GROUPING SETS` when there are duplicated grouping sets in a query (e.g., `GROUPING SETS ((k1),(k1))`).

For example;
```
scala> spark.table("t").show()
+---+---+---+
| k1| k2|  v|
+---+---+---+
|  0|  0|  3|
+---+---+---+

scala> sql("""select grouping_id(), k1, k2, sum(v) from t group by grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2))""").show()
+-------------+---+----+------+
|grouping_id()| k1|  k2|sum(v)|
+-------------+---+----+------+
|            0|  0|   0|     9| <---- wrong aggregate value and the correct answer is `3`
|            1|  0|null|     3|
+-------------+---+----+------+

// PostgreSQL case
postgres=#  select k1, k2, sum(v) from t group by grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2));
 k1 |  k2  | sum
----+------+-----
  0 |    0 |   3
  0 |    0 |   3
  0 |    0 |   3
  0 | NULL |   3
(4 rows)

// Hive case
hive> select GROUPING__ID, k1, k2, sum(v) from t group by k1, k2 grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2));
1	0	NULL	3
0	0	0	3
```
[MS SQL Server has the same behaviour with PostgreSQL](https://github.com/apache/spark/pull/26961#issuecomment-573638442). This pr follows the behaviour of PostgreSQL/SQL server; it adds one more virtual attribute in `Expand` for avoiding wrongly grouping rows with the same grouping ID.

### Why are the changes needed?

To fix bugs.

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

No.

### How was this patch tested?

The existing tests.

Closes #26961 from maropu/SPARK-29708.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-15 22:02:16 +09:00
Ajith 0c6bd3bd0b [SPARK-27142][SQL] Provide REST API for SQL information
### What changes were proposed in this pull request?

Currently for Monitoring Spark application SQL information is not available from REST but only via UI. REST provides only applications,jobs,stages,environment. This Jira is targeted to provide a REST API so that SQL level information can be found
A single SQL query can result into multiple jobs. So for end user who is using STS or spark-sql, the intended highest level of probe is the SQL which he has executed. This information can be seen from SQL tab. Attaching a sample.
![image](https://user-images.githubusercontent.com/22072336/54298729-5524a800-45df-11e9-8e4d-b99a8b882031.png)
But same information he cannot access using the REST API exposed by spark and he always have to rely on jobs API which may be difficult. So i intend to expose the information seen in SQL tab in UI via REST API

Mainly:

Id :  Long - execution id of the sql
status : String - possible values COMPLETED/RUNNING/FAILED
description : String - executed SQL string
planDescription : String - Plan representation
metrics : Seq[Metrics] - `Metrics` contain `metricName: String, metricValue: String`
submissionTime : String - formatted `Date` time of SQL submission
duration : Long - total run time in milliseconds
runningJobIds : Seq[Int] - sequence of running job ids
failedJobIds : Seq[Int] - sequence of failed job ids
successJobIds : Seq[Int] - sequence of success job ids

* To fetch sql executions: /sql?details=boolean&offset=integer&length=integer
* To fetch single execution:  /sql/{executionID}?details=boolean

| parameter | type | remarks |
| ------------- |:-------------:| -----|
| details | boolean | Optional. Set true to get plan description and metrics information, defaults to false |
| offset | integer | Optional. offset to fetch the executions, defaults to 0 |
| length | integer | Optional. total number of executions to be fetched, defaults to 20 |

### Why are the changes needed?
To support users query SQL information via REST API

### Does this PR introduce any user-facing change?
Yes. It provides a new monitoring URL for SQL

### How was this patch tested?
Tested manually

![image](https://user-images.githubusercontent.com/22072336/54282168-6d85ca00-45c1-11e9-8935-7586ccf0efff.png)

![image](https://user-images.githubusercontent.com/22072336/54282191-7b3b4f80-45c1-11e9-941c-f0ec37026192.png)

Closes #24076 from ajithme/restapi.

Lead-authored-by: Ajith <ajith2489@gmail.com>
Co-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-14 10:05:47 -08:00
Erik Erlandson 176b69642e [SPARK-30423][SQL] Deprecate UserDefinedAggregateFunction
### What changes were proposed in this pull request?
* Annotate UserDefinedAggregateFunction as deprecated by SPARK-27296
* Update user doc examples to reflect new ability to register typed Aggregator[IN, BUF, OUT] as an untyped aggregating UDF
### Why are the changes needed?
UserDefinedAggregateFunction is being deprecated

### Does this PR introduce any user-facing change?
Changes are to user documentation, and deprecation annotations.

### How was this patch tested?
Testing was via package build to verify doc generation, deprecation warnings, and successful example compilation.

Closes #27193 from erikerlandson/spark-30423.

Authored-by: Erik Erlandson <eerlands@redhat.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 22:07:13 +08:00
jiake a2aa966ef6 [SPARK-29544][SQL] optimize skewed partition based on data size
### What changes were proposed in this pull request?
Skew Join is common and can severely downgrade performance of queries, especially those with joins. This PR aim to optimization the skew join based on the runtime Map output statistics by adding "OptimizeSkewedPartitions" rule. And The details design doc is [here](https://docs.google.com/document/d/1NkXN-ck8jUOS0COz3f8LUW5xzF8j9HFjoZXWGGX2HAg/edit). Currently we can support "Inner, Cross, LeftSemi, LeftAnti, LeftOuter, RightOuter" join type.

### Why are the changes needed?
To optimize the skewed partition in runtime based on AQE

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

### How was this patch tested?
UT

Closes #26434 from JkSelf/skewedPartitionBasedSize.

Lead-authored-by: jiake <ke.a.jia@intel.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: JiaKe <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 20:31:44 +08:00
root1 e0efd213eb [SPARK-30292][SQL] Throw Exception when invalid string is cast to numeric type in ANSI mode
### What changes were proposed in this pull request?
If spark.sql.ansi.enabled is set,
throw exception when cast to any numeric type do not follow the ANSI SQL standards.

### Why are the changes needed?
ANSI SQL standards do not allow invalid strings to get casted into numeric types and throw exception for that. Currently spark sql gives NULL in such cases.

Before:
`select cast('str' as decimal)  => NULL`

After :
`select cast('str' as decimal) => invalid input syntax for type numeric: str`

These results are after setting `spark.sql.ansi.enabled=true`

### Does this PR introduce any user-facing change?
Yes. Now when ansi mode is on users will get arithmetic exception for invalid strings.

### How was this patch tested?
Unit Tests Added.

Closes #26933 from iRakson/castDecimalANSI.

Lead-authored-by: root1 <raksonrakesh@gmail.com>
Co-authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 17:03:10 +08:00
Maxim Gekk 88fc8dbc09 [SPARK-30482][SQL][CORE][TESTS] Add sub-class of AppenderSkeleton reusable in tests
### What changes were proposed in this pull request?
In the PR, I propose to define a sub-class of `AppenderSkeleton` in `SparkFunSuite` and reuse it from other tests. The class stores incoming `LoggingEvent` in an array which is available to tests for future analysis of logged events.

### Why are the changes needed?
This eliminates code duplication in tests.

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

### How was this patch tested?
By existing test suites - `CSVSuite`, `OptimizerLoggingSuite`, `JoinHintSuite`, `CodeGenerationSuite` and `SQLConfSuite`.

Closes #27166 from MaxGekk/dedup-appender-skeleton.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-14 16:03:10 +09:00
Maxim Gekk 1846b0261b [SPARK-30500][SPARK-30501][SQL] Remove SQL configs deprecated in Spark 2.1 and 2.3
### What changes were proposed in this pull request?
In the PR, I propose to remove already deprecated SQL configs:
- `spark.sql.variable.substitute.depth` deprecated in Spark 2.1
- `spark.sql.parquet.int64AsTimestampMillis` deprecated in Spark 2.3

Also I moved `removedSQLConfigs` closer to `deprecatedSQLConfigs`. This will allow to have references to other config entries.

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

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

### How was this patch tested?
By existing test suites `ParquetQuerySuite` and `SQLConfSuite`.

Closes #27169 from MaxGekk/remove-deprecated-conf-2.4.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-14 11:06:48 +09:00
HyukjinKwon 6646b3e13e Revert "[SPARK-28670][SQL] create function should thrown Exception if the resource is not found"
This reverts commit 16e5e79877.
2020-01-14 10:40:35 +09:00
jiake b389b8c5f0 [SPARK-30188][SQL] Resolve the failed unit tests when enable AQE
### What changes were proposed in this pull request?
Fix all the failed tests when enable AQE.

### Why are the changes needed?
Run more tests with AQE to catch bugs, and make it easier to enable AQE by default in the future.

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

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

Closes #26813 from JkSelf/enableAQEDefault.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-13 22:55:19 +08:00
Dongjoon Hyun 28fc0437ce [SPARK-28152][SQL][FOLLOWUP] Add a legacy conf for old MsSqlServerDialect numeric mapping
### What changes were proposed in this pull request?

This is a follow-up for https://github.com/apache/spark/pull/25248 .

### Why are the changes needed?

The new behavior cannot access the existing table which is created by old behavior.
This PR provides a way to avoid new behavior for the existing users.

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

Yes. This will fix the broken behavior on the existing tables.

### How was this patch tested?

Pass the Jenkins and manually run JDBC integration test.
```
build/mvn install -DskipTests
build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12 test
```

Closes #27184 from dongjoon-hyun/SPARK-28152-CONF.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-12 23:03:34 -08:00
Dongjoon Hyun 361583d1f5 [SPARK-30409][TEST][FOLLOWUP][HOTFIX] Remove dangling JSONBenchmark-jdk11-results.txt
### What changes were proposed in this pull request?

This PR removes a dangling test result, `JSONBenchmark-jdk11-results.txt`.
This causes a case-sensitive issue on Mac.

```
$ git clone https://gitbox.apache.org/repos/asf/spark.git spark-gitbox
Cloning into 'spark-gitbox'...
remote: Counting objects: 671717, done.
remote: Compressing objects: 100% (258021/258021), done.
remote: Total 671717 (delta 329181), reused 560390 (delta 228097)
Receiving objects: 100% (671717/671717), 149.69 MiB | 950.00 KiB/s, done.
Resolving deltas: 100% (329181/329181), done.
Updating files: 100% (16090/16090), done.
warning: the following paths have collided (e.g. case-sensitive paths
on a case-insensitive filesystem) and only one from the same
colliding group is in the working tree:

  'sql/core/benchmarks/JSONBenchmark-jdk11-results.txt'
  'sql/core/benchmarks/JsonBenchmark-jdk11-results.txt'
```

### Why are the changes needed?

Previously, since the file name didn't match with `object JSONBenchmark`, it made a confusion when we ran the benchmark. So, 4e0e4e51c4 renamed `JSONBenchmark` to `JsonBenchmark`. However, at the same time frame, https://github.com/apache/spark/pull/26003 regenerated this file.

Recently, https://github.com/apache/spark/pull/27078 regenerates the results with the correct file name, `JsonBenchmark-jdk11-results.txt`. So, we can remove the old one.

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

No. This is a test result.

### How was this patch tested?

Manually check the following correctly generated files in the master. And, check this PR removes the dangling one.
- https://github.com/apache/spark/blob/master/sql/core/benchmarks/JsonBenchmark-results.txt
- https://github.com/apache/spark/blob/master/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt

Closes #27180 from dongjoon-hyun/SPARK-REMOVE.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-12 23:45:31 +00:00
Maxim Gekk f5118f81e3 [SPARK-30409][SPARK-29173][SQL][TESTS] Use NoOp datasource in SQL benchmarks
### What changes were proposed in this pull request?
In the PR, I propose to replace `.collect()`, `.count()` and `.foreach(_ => ())` in SQL benchmarks and use the `NoOp` datasource. I added an implicit class to `SqlBasedBenchmark` with the `.noop()` method. It can be used in benchmark like: `ds.noop()`. The last one is unfolded to `ds.write.format("noop").mode(Overwrite).save()`.

### Why are the changes needed?
To avoid additional overhead that `collect()` (and other actions) has. For example, `.collect()` has to convert values according to external types and pull data to the driver. This can hide actual performance regressions or improvements of benchmarked operations.

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

### How was this patch tested?
Re-run all modified benchmarks using Amazon EC2.

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge (spot instance) |
| AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) |
| Java | OpenJDK8/10 |

- Run `TPCDSQueryBenchmark` using instructions from the PR #26049
```
# `spark-tpcds-datagen` needs this. (JDK8)
$ git clone https://github.com/apache/spark.git -b branch-2.4 --depth 1 spark-2.4
$ export SPARK_HOME=$PWD
$ ./build/mvn clean package -DskipTests

# Generate data. (JDK8)
$ git clone gitgithub.com:maropu/spark-tpcds-datagen.git
$ cd spark-tpcds-datagen/
$ build/mvn clean package
$ mkdir -p /data/tpcds
$ ./bin/dsdgen --output-location /data/tpcds/s1  // This need `Spark 2.4`
```
- Other benchmarks ran by the script:
```
#!/usr/bin/env python3

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.AggregateBenchmark'],
    ['avro/test', 'org.apache.spark.sql.execution.benchmark.AvroReadBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.BloomFilterBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.DataSourceReadBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.DateTimeBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.ExtractBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.FilterPushdownBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.InExpressionBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.IntervalBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.JoinBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.MakeDateTimeBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.MiscBenchmark'],
    ['hive/test', 'org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.OrcNestedSchemaPruningBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.OrcV2NestedSchemaPruningBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.ParquetNestedSchemaPruningBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.RangeBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.UDFBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.WideSchemaBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.WideTableBenchmark'],
    ['hive/test', 'org.apache.spark.sql.hive.orc.OrcReadBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.datasources.csv.CSVBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.datasources.json.JsonBenchmark']
]

print('Set SPARK_GENERATE_BENCHMARK_FILES=1')
os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1'

for b in benchmarks:
    print("Run benchmark: %s" % b[1])
    run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])])
```

Closes #27078 from MaxGekk/noop-in-benchmarks.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-12 13:18:19 -08:00
Erik Erlandson 1f50a5875b [SPARK-27296][SQL] Allows Aggregator to be registered as a UDF
## What changes were proposed in this pull request?
Defines a new subclass of UDF: `UserDefinedAggregator`. Also allows `Aggregator` to be registered as a udf.  Under the hood, the implementation is based on the internal `TypedImperativeAggregate` class that spark's predefined aggregators make use of. The effect is that custom user defined aggregators are now serialized only on partition boundaries instead of being serialized and deserialized at each input row.

The two new modes of using `Aggregator` are as follows:
```scala
val agg: Aggregator[IN, BUF, OUT] = // typed aggregator
val udaf1 = UserDefinedAggregator(agg)
val udaf2 = spark.udf.register("agg", agg)
```

## How was this patch tested?
Unit testing has been added that corresponds to the testing suites for `UserDefinedAggregateFunction`. Additionally, unit tests explicitly count the number of aggregator ser/de cycles to ensure that it is governed only by the number of data partitions.

To evaluate the performance impact, I did two comparisons.
The code and REPL results are recorded on [this gist](https://gist.github.com/erikerlandson/b0e106a4dbaf7f80b4f4f3a21f05f892)
To characterize its behavior I benchmarked both a relatively simple aggregator and then an aggregator with a complex structure (a t-digest).

### performance
The following compares the new `Aggregator` based aggregation against UDAF. In this scenario, the new aggregation is about 100x faster. The difference in performance impact depends on the complexity of the aggregator. For very simple aggregators (e.g. implementing 'sum', etc), the performance impact is more like 25-30%.

```scala
scala> import scala.util.Random._, org.apache.spark.sql.Row, org.apache.spark.tdigest._
import scala.util.Random._
import org.apache.spark.sql.Row
import org.apache.spark.tdigest._

scala> val data = sc.parallelize(Vector.fill(50000){(nextInt(2), nextGaussian, nextGaussian.toFloat)}, 5).toDF("cat", "x1", "x2")
data: org.apache.spark.sql.DataFrame = [cat: int, x1: double ... 1 more field]

scala> val udaf = TDigestUDAF(0.5, 0)
udaf: org.apache.spark.tdigest.TDigestUDAF = TDigestUDAF(0.5,0)

scala> val bs = Benchmark.sample(10) { data.agg(udaf($"x1"), udaf($"x2")).first }
bs: Array[(Double, org.apache.spark.sql.Row)] = Array((16.523,[TDigestSQL(TDigest(0.5,0,130,TDigestMap(-4.9171836327285225 -> (1.0, 1.0), -3.9615949140987685 -> (1.0, 2.0), -3.792874086327091 -> (0.7500781537109753, 2.7500781537109753), -3.720534874164185 -> (1.796754196108008, 4.546832349818983), -3.702105588052377 -> (0.4531676501810167, 5.0), -3.665883591332569 -> (2.3434687534153142, 7.343468753415314), -3.649982231368131 -> (0.6565312465846858, 8.0), -3.5914188829817744 -> (4.0, 12.0), -3.530472305581248 -> (4.0, 16.0), -3.4060489584449467 -> (2.9372251939818383, 18.93722519398184), -3.3000694035428486 -> (8.12412890252889, 27.061354096510726), -3.2250016655261877 -> (8.30564453211017, 35.3669986286209), -3.180537395623448 -> (6.001782561137285, 41.3687811...

scala> bs.map(_._1)
res0: Array[Double] = Array(16.523, 17.138, 17.863, 17.801, 17.769, 17.786, 17.744, 17.8, 17.939, 17.854)

scala> val agg = TDigestAggregator(0.5, 0)
agg: org.apache.spark.tdigest.TDigestAggregator = TDigestAggregator(0.5,0)

scala> val udaa = spark.udf.register("tdigest", agg)
udaa: org.apache.spark.sql.expressions.UserDefinedAggregator[Double,org.apache.spark.tdigest.TDigestSQL,org.apache.spark.tdigest.TDigestSQL] = UserDefinedAggregator(TDigestAggregator(0.5,0),None,true,true)

scala> val bs = Benchmark.sample(10) { data.agg(udaa($"x1"), udaa($"x2")).first }
bs: Array[(Double, org.apache.spark.sql.Row)] = Array((0.313,[TDigestSQL(TDigest(0.5,0,130,TDigestMap(-4.9171836327285225 -> (1.0, 1.0), -3.9615949140987685 -> (1.0, 2.0), -3.792874086327091 -> (0.7500781537109753, 2.7500781537109753), -3.720534874164185 -> (1.796754196108008, 4.546832349818983), -3.702105588052377 -> (0.4531676501810167, 5.0), -3.665883591332569 -> (2.3434687534153142, 7.343468753415314), -3.649982231368131 -> (0.6565312465846858, 8.0), -3.5914188829817744 -> (4.0, 12.0), -3.530472305581248 -> (4.0, 16.0), -3.4060489584449467 -> (2.9372251939818383, 18.93722519398184), -3.3000694035428486 -> (8.12412890252889, 27.061354096510726), -3.2250016655261877 -> (8.30564453211017, 35.3669986286209), -3.180537395623448 -> (6.001782561137285, 41.36878118...

scala> bs.map(_._1)
res1: Array[Double] = Array(0.313, 0.193, 0.175, 0.185, 0.174, 0.176, 0.16, 0.186, 0.171, 0.179)

scala>
```

Closes #25024 from erikerlandson/spark-27296.

Authored-by: Erik Erlandson <eerlands@redhat.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-12 15:18:30 +08:00
Liang-Chi Hsieh b04407169b [SPARK-30312][SQL][FOLLOWUP] Use inequality check instead to be robust
### What changes were proposed in this pull request?

This is a followup to fix a brittle assert in a test case.

### Why are the changes needed?

Original assert assumes that default permission is `rwxr-xr-x`, but in jenkins [env](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-1.2/6/testReport/junit/org.apache.spark.sql.execution.command/InMemoryCatalogedDDLSuite/SPARK_30312__truncate_table___keep_acl_permission/) it could be `rwxrwxr-x`.

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

No

### How was this patch tested?

Unit test.

Closes #27175 from viirya/hot-fix.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-11 13:19:04 -08:00
Liang-Chi Hsieh b5bc3e12a6 [SPARK-30312][SQL] Preserve path permission and acl when truncate table
### What changes were proposed in this pull request?

This patch proposes to preserve existing permission/acls of paths when truncate table/partition.

### Why are the changes needed?

When Spark SQL truncates table, it deletes the paths of table/partitions, then re-create new ones. If permission/acls were set on the paths, the existing permission/acls will be deleted.

We should preserve the permission/acls if possible.

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

Yes. When truncate table/partition, Spark will keep permission/acls of paths.

### How was this patch tested?

Unit test.

Manual test:

1. Create a table.
2. Manually change it permission/acl
3. Truncate table
4. Check permission/acl

```scala
val df = Seq(1, 2, 3).toDF
df.write.mode("overwrite").saveAsTable("test.test_truncate_table")
val testTable = spark.table("test.test_truncate_table")
testTable.show()
+-----+
|value|
+-----+
|    1|
|    2|
|    3|
+-----+
// hdfs dfs -setfacl ...
// hdfs dfs -getfacl ...
sql("truncate table test.test_truncate_table")
// hdfs dfs -getfacl ...
val testTable2 = spark.table("test.test_truncate_table")
testTable2.show()
+-----+
|value|
+-----+
+-----+
```

![Screen Shot 2019-12-30 at 3 12 15 PM](https://user-images.githubusercontent.com/68855/71604577-c7875a00-2b17-11ea-913a-ba88096d20ab.jpg)

Closes #26956 from viirya/truncate-table-permission.

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-10 11:46:28 -08:00
Zhenhua Wang 2bd8731813 [SPARK-30468][SQL] Use multiple lines to display data columns for show create table command
### What changes were proposed in this pull request?
Currently data columns are displayed in one line for show create table command, when the table has many columns (to make things even worse, columns may have long names or comments), the displayed result is really hard to read.

To improve readability, we print each column in a separate line. Note that other systems like Hive/MySQL also display in this way.

Also, for data columns, table properties and options, we put the right parenthesis to the end of the last column/property/option, instead of occupying a separate line.

### Why are the changes needed?
for better readability

### Does this PR introduce any user-facing change?
before the change:
```
spark-sql> show create table test_table;
CREATE TABLE `test_table` (`col1` INT COMMENT 'This is comment for column 1', `col2` STRING COMMENT 'This is comment for column 2', `col3` DOUBLE COMMENT 'This is comment for column 3')
USING parquet
OPTIONS (
  `bar` '2',
  `foo` '1'
)
TBLPROPERTIES (
  'a' = 'x',
  'b' = 'y'
)
```
after the change:
```
spark-sql> show create table test_table;
CREATE TABLE `test_table` (
  `col1` INT COMMENT 'This is comment for column 1',
  `col2` STRING COMMENT 'This is comment for column 2',
  `col3` DOUBLE COMMENT 'This is comment for column 3')
USING parquet
OPTIONS (
  `bar` '2',
  `foo` '1')
TBLPROPERTIES (
  'a' = 'x',
  'b' = 'y')
```

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

Closes #27147 from wzhfy/multi_line_columns.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-10 10:55:53 -06:00
root1 2a629e5d10 [SPARK-30234][SQL] ADD FILE cannot add directories from sql CLI
### What changes were proposed in this pull request?
Now users can add directories from sql CLI as well using ADD FILE command and setting spark.sql.addDirectory.recursive to true.

### Why are the changes needed?
In SPARK-4687, support was added for adding directories as resources. But sql users cannot use that feature from CLI.

`ADD FILE /path/to/folder` gives the following error:
`org.apache.spark.SparkException: Added file /path/to/folder is a directory and recursive is not turned on.`

Users need to turn on `recursive` for adding directories. Thus a configuration was required which will allow users to turn on `recursive`.
Also Hive allow users to add directories from their shell.

### Does this PR introduce any user-facing change?
Yes. Users can set recursive using `spark.sql.addDirectory.recursive`.

### How was this patch tested?
Manually.
Will add test cases soon.

 SPARK SCREENSHOTS
When `spark.sql.addDirectory.recursive` is not turned on.
![Screenshot from 2019-12-13 08-02-13](https://user-images.githubusercontent.com/15366835/70765124-c6b4a100-1d7f-11ea-9352-9c010af5b38b.png)

After setting `spark.sql.addDirectory.recursive` to true.

![Screenshot from 2019-12-13 08-02-59](https://user-images.githubusercontent.com/15366835/70765118-be5c6600-1d7f-11ea-9faf-0b1c46ee299b.png)

HIVE SCREENSHOT

![Screenshot from 2019-12-13 14-44-41](https://user-images.githubusercontent.com/15366835/70788979-17e08700-1db8-11ea-9c0c-b6d6f6e80a35.png)

`RELEASE_NOTES.txt` is text file while `dummy` is a directory.

Closes #26863 from iRakson/SPARK-30234.

Lead-authored-by: root1 <raksonrakesh@gmail.com>
Co-authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-10 22:36:45 +09:00
Peter Toth 418f7dc973 [SPARK-30447][SQL] Constant propagation nullability issue
## What changes were proposed in this pull request?

This PR fixes `ConstantPropagation` rule as the current implementation produce incorrect results in some cases. E.g.
```
SELECT * FROM t WHERE NOT(c = 1 AND c + 1 = 1)
```
returns those rows where `c` is null due to `1 + 1 = 1` propagation but it shouldn't.

## Why are the changes needed?

To fix a bug.

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

Yes, fixes a bug.

## How was this patch tested?

New UTs.

Closes #27119 from peter-toth/SPARK-30447.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-10 21:42:10 +09:00
Kent Yao bcf07cbf5f [SPARK-30018][SQL] Support ALTER DATABASE SET OWNER syntax
### What changes were proposed in this pull request?
In this pull request, we are going to support `SET OWNER` syntax for databases and namespaces,

```sql
ALTER (DATABASE|SCHEME|NAMESPACE) database_name SET OWNER [USER|ROLE|GROUP] user_or_role_group;
```
Before this commit 332e252a14, we didn't care much about ownerships for the catalog objects. In 332e252a14, we determined to use properties to store ownership staff, and temporarily used `alter database ... set dbproperties ...` to support switch ownership of a database. This PR aims to use the formal syntax to replace it.

In hive, `ownerName/Type` are fields of the database objects, also they can be normal properties.
```
create schema test1 with dbproperties('ownerName'='yaooqinn')
```
The create/alter database syntax will not change the owner to `yaooqinn` but store it in parameters. e.g.
```
+----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+
| db_name  | comment  |                           location                            | owner_name  | owner_type  |      parameters       |
+----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+
| test1    |          | hdfs://quickstart.cloudera:8020/user/hive/warehouse/test1.db  | anonymous   | USER        | {ownerName=yaooqinn}  |
+----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+
```
In this pull request, because we let the `ownerName` become reversed, so it will neither change the owner nor store in dbproperties, just be omitted silently.

## Why are the changes needed?

Formal syntax support for changing database ownership

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

yes, add a new syntax

### How was this patch tested?

add unit tests

Closes #26775 from yaooqinn/SPARK-30018.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-10 16:47:08 +08:00
Wenchen Fan 0ec0355611 [SPARK-30439][SQL] Support non-nullable column in CREATE TABLE, ADD COLUMN and ALTER TABLE
### What changes were proposed in this pull request?

Allow users to specify NOT NULL in CREATE TABLE and ADD COLUMN column definition, and add a new SQL syntax to alter column nullability: ALTER TABLE ... ALTER COLUMN SET/DROP NOT NULL. This is a SQL standard syntax:
```
<alter column definition> ::=
  ALTER [ COLUMN ] <column name> <alter column action>

<alter column action> ::=
    <set column default clause>
  | <drop column default clause>
  | <set column not null clause>
  | <drop column not null clause>
  | ...

<set column not null clause> ::=
  SET NOT NULL

<drop column not null clause> ::=
  DROP NOT NULL
```

### Why are the changes needed?

Previously we don't support it because the table schema in hive catalog are always nullable. Since we have catalog plugin now, it makes more sense to support NOT NULL at spark side, and let catalog implementations to decide if they support it or not.

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

Yes, this is a new feature

### How was this patch tested?

new tests

Closes #27110 from cloud-fan/nullable.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-10 10:34:46 +09:00
Maxim Gekk 1ffa627ffb [SPARK-30416][SQL] Log a warning for deprecated SQL config in set() and unset()
### What changes were proposed in this pull request?
1. Put all deprecated SQL configs the map `SQLConf.deprecatedSQLConfigs` with extra info about when configs were deprecated and additional comments that explain why a config was deprecated, what an user can use instead of it. Here is the list of already deprecated configs:
    - spark.sql.hive.verifyPartitionPath
    - spark.sql.execution.pandas.respectSessionTimeZone
    - spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName
    - spark.sql.parquet.int64AsTimestampMillis
    - spark.sql.variable.substitute.depth
    - spark.sql.execution.arrow.enabled
    - spark.sql.execution.arrow.fallback.enabled

2. Output warning in `set()` and `unset()` about deprecated SQL configs

### Why are the changes needed?
This should improve UX with Spark SQL and notify users about already deprecated SQL configs.

### Does this PR introduce any user-facing change?
Yes, before:
```
spark-sql> set spark.sql.hive.verifyPartitionPath=true;
spark.sql.hive.verifyPartitionPath	true
```
After:
```
spark-sql> set spark.sql.hive.verifyPartitionPath=true;
20/01/03 21:28:17 WARN RuntimeConfig: The SQL config 'spark.sql.hive.verifyPartitionPath' has been deprecated in Spark v3.0.0 and may be removed in the future. This config is replaced by spark.files.ignoreMissingFiles.
spark.sql.hive.verifyPartitionPath	true
```

### How was this patch tested?
Add new test which registers new log appender and catches all logging to check that `set()` and `unset()` log any warning.

Closes #27092 from MaxGekk/group-deprecated-sql-configs.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-10 10:32:36 +09:00
shane knapp 4d23938893 [MINOR][SQL][TEST-HIVE1.2] Fix scalastyle error due to length line in hive-1.2 profile
### What changes were proposed in this pull request?

fixing a broken build:
https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7-hive-1.2/3/console

### Why are the changes needed?

the build is teh borked!

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

newp

### How was this patch tested?

by the build system

Closes #27156 from shaneknapp/fix-scala-style.

Authored-by: shane knapp <incomplete@gmail.com>
Signed-off-by: shane knapp <incomplete@gmail.com>
2020-01-09 15:28:45 -08:00
yi.wu c0e9f9ffb1 [SPARK-30459][SQL] Fix ignoreMissingFiles/ignoreCorruptFiles in data source v2
### What changes were proposed in this pull request?

Fix ignoreMissingFiles/ignoreCorruptFiles in DSv2:

When `FilePartitionReader` finds a missing or corrupt file, it should just skip and continue to read next file rather than stop with current behavior.

### Why are the changes needed?

ignoreMissingFiles/ignoreCorruptFiles in DSv2 is wrong comparing to DSv1.

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

No.

### How was this patch tested?

Updated existed test for `ignoreMissingFiles`. Note I didn't update tests for `ignoreCorruptFiles`, because  there're various datasources has tests for `ignoreCorruptFiles`. So I'm not sure if it's worth to touch all those tests since the basic logic of `ignoreCorruptFiles` should be same with `ignoreMissingFiles`.

Closes #27136 from Ngone51/improve-missing-files.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-01-09 11:35:29 -08:00
Burak Yavuz f8d59572b0 [SPARK-29219][SQL] Introduce SupportsCatalogOptions for TableProvider
### What changes were proposed in this pull request?

This PR introduces `SupportsCatalogOptions` as an interface for `TableProvider`. Through `SupportsCatalogOptions`, V2 DataSources can implement the two methods `extractIdentifier` and `extractCatalog` to support the creation, and existence check of tables without requiring a formal TableCatalog implementation.

We currently don't support all SaveModes for DataSourceV2 in DataFrameWriter.save. The idea here is that eventually File based tables can be written with `DataFrameWriter.save(path)` will create a PathIdentifier where the name is `path`, and the V2SessionCatalog will be able to perform FileSystem checks at `path` to support ErrorIfExists and Ignore SaveModes.

### Why are the changes needed?

To support all Save modes for V2 data sources with DataFrameWriter. Since we can now support table creation, we will be able to provide partitioning information when first creating the table as well.

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

Introduces a new interface

### How was this patch tested?

Will add tests once interface is vetted.

Closes #26913 from brkyvz/catalogOptions.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-01-09 11:18:16 -08:00
Gengliang Wang 94fc0e3235 [SPARK-30428][SQL] File source V2: support partition pruning
### What changes were proposed in this pull request?

File source V2: support partition pruning.
Note: subquery predicates are not pushed down for partition pruning even after this PR, due to the limitation for the current data source V2 API and framework. The rule `PlanSubqueries` requires the subquery expression to be in the children or class parameters in `SparkPlan`, while the condition is not satisfied for `BatchScanExec`.

### Why are the changes needed?

It's important for reading performance.

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

No

### How was this patch tested?

New unit tests for all the V2 file sources

Closes #27112 from gengliangwang/PartitionPruningInFileScan.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-09 21:53:37 +08:00
Kent Yao c37312342e [SPARK-30183][SQL] Disallow to specify reserved properties in CREATE/ALTER NAMESPACE syntax
### What changes were proposed in this pull request?
Currently, COMMENT and LOCATION are reserved properties for Datasource v2 namespaces. They can be set via specific clauses and via properties. And the ones specified in clauses take precede of properties. Since they are reserved, which means they are not able to visit directly. They should be used in COMMENT/LOCATION clauses ONLY.

### Why are the changes needed?
make reserved properties be reserved.

### Does this PR introduce any user-facing change?
yes, 'location', 'comment' are not allowed use in db properties

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

Closes #26806 from yaooqinn/SPARK-30183.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-09 10:52:36 +08:00
HyukjinKwon ee8d661058 [SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?

This PR proposes to move pandas related functionalities into pandas package. Namely:

```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py  # Conversion between pandas <> PySpark DataFrames
├── functions.py   # pandas_udf
├── group_ops.py   # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py     # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py       # Type utils between pandas <> PyArrow
└── utils.py       # Version requirement checks
```

In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:

```python
class PandasMapOpsMixin(object):
    def mapInPandas(self, ...):
        ...
        return ...

    # other Pandas <> PySpark APIs
```

```python
class DataFrame(PandasMapOpsMixin):

    # other DataFrame APIs equivalent to Scala side.

```

Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.

### Why are the changes needed?

There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.

Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.

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

No.

### How was this patch tested?

Existing tests should cover. Also, I manually built the PySpark API documentation and checked.

Closes #27109 from HyukjinKwon/pandas-refactoring.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-09 10:22:50 +09:00
maryannxue af2d3d0179 [SPARK-30315][SQL] Add adaptive execution context
### What changes were proposed in this pull request?
This is a minor code refactoring PR. It creates an adaptive execution context class to wrap objects shared across main query and sub-queries.

### Why are the changes needed?
This refactoring will improve code readability and reduce the number of parameters used to initialize `AdaptiveSparkPlanExec`.

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

### How was this patch tested?
Passed existing UTs.

Closes #26959 from maryannxue/aqe-context.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-08 16:11:46 -08:00
Jungtaek Lim (HeartSaVioR) bd7510bcb7 [SPARK-30281][SS] Consider partitioned/recursive option while verifying archive path on FileStreamSource
### What changes were proposed in this pull request?

This patch renews the verification logic of archive path for FileStreamSource, as we found the logic doesn't take partitioned/recursive options into account.

Before the patch, it only requires the archive path to have depth more than 2 (two subdirectories from root), leveraging the fact FileStreamSource normally reads the files where the parent directory matches the pattern or the file itself matches the pattern. Given 'archive' operation moves the files to the base archive path with retaining the full path, archive path is tend to be safe if the depth is more than 2, meaning FileStreamSource doesn't re-read archived files as new source files.

WIth partitioned/recursive options, the fact is invalid, as FileStreamSource can read any files in any depth of subdirectories for source pattern. To deal with this correctly, we have to renew the verification logic, which may not intuitive and simple but works for all cases.

The new verification logic prevents both cases:

1) archive path matches with source pattern as "prefix" (the depth of archive path > the depth of source pattern)

e.g.
* source pattern: `/hello*/spar?`
* archive path: `/hello/spark/structured/streaming`

Any files in archive path will match with source pattern when recursive option is enabled.

2) source pattern matches with archive path as "prefix" (the depth of source pattern > the depth of archive path)

e.g.
* source pattern: `/hello*/spar?/structured/hello2*`
* archive path: `/hello/spark/structured`

Some archive files will not match with source pattern, e.g. file path:  `/hello/spark/structured/hello2`, then final archived path: `/hello/spark/structured/hello/spark/structured/hello2`.

But some other archive files will still match with source pattern, e.g. file path: `/hello2/spark/structured/hello2`, then final archived path: `/hello/spark/structured/hello2/spark/structured/hello2` which matches with source pattern when recursive is enabled.

Implicitly it also prevents archive path matches with source pattern as full match (same depth).

We would want to prevent any source files to be archived and added to new source files again, so the patch takes most restrictive approach to prevent the possible cases.

### Why are the changes needed?

Without this patch, there's a chance archived files are included as new source files when partitioned/recursive option is enabled, as current condition doesn't take these options into account.

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

Only for Spark 3.0.0-preview (only preview 1 for now, but possibly preview 2 as well) - end users are required to provide archive path with ensuring a bit complicated conditions, instead of simply higher than 2 depths.

### How was this patch tested?

New UT.

Closes #26920 from HeartSaVioR/SPARK-30281.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-08 09:15:41 -08:00
zhengruifeng a93b996635 [MINOR][ML][INT] Array.fill(0) -> Array.ofDim; Array.empty -> Array.emptyIntArray
### What changes were proposed in this pull request?
1, for primitive types `Array.fill(n)(0)` -> `Array.ofDim(n)`;
2, for `AnyRef` types `Array.fill(n)(null)` -> `Array.ofDim(n)`;
3, for primitive types `Array.empty[XXX]` -> `Array.emptyXXXArray`

### Why are the changes needed?
`Array.ofDim` avoid assignments;
`Array.emptyXXXArray` avoid create new object;

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

### How was this patch tested?
existing testsuites

Closes #27133 from zhengruifeng/minor_fill_ofDim.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-09 00:07:42 +09:00
Zhenhua Wang fa36966b1e [SPARK-30410][SQL] Calculating size of table with large number of partitions causes flooding logs
### What changes were proposed in this pull request?

For a partitioned table, if the number of partitions are very large, e.g. tens of thousands or even larger, calculating its total size causes flooding logs.
The flooding happens in:
1. `calculateLocationSize` prints the starting and ending for calculating the location size, and it is called per partition;
2. `bulkListLeafFiles` prints all partition paths.

This pr is to simplify the logging when calculating the size of a partitioned table.

### How was this patch tested?

not related

Closes #27079 from wzhfy/improve_log.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-08 08:22:38 -06:00
fuwhu 047bff06c3 [SPARK-30215][SQL] Remove PrunedInMemoryFileIndex and merge its functionality into InMemoryFileIndex
### What changes were proposed in this pull request?
Remove PrunedInMemoryFileIndex and merge its functionality into InMemoryFileIndex.

### Why are the changes needed?
PrunedInMemoryFileIndex is only used in CatalogFileIndex.filterPartitions, and its name is kind of confusing, we can completely merge its functionality into InMemoryFileIndex and remove the class.

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

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

Closes #26850 from fuwhu/SPARK-30215.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-08 20:28:15 +08:00
Terry Kim b2ed6d0b88 [SPARK-30214][SQL][FOLLOWUP] Remove statement logical plans for namespace commands
### What changes were proposed in this pull request?

This is a follow-up to address the following comment: https://github.com/apache/spark/pull/27095#discussion_r363152180

Currently, a SQL command string is parsed to a "statement" logical plan, converted to a logical plan with catalog/namespace, then finally converted to a physical plan. With the new resolution framework, there is no need to create a "statement" logical plan; a logical plan can contain `UnresolvedNamespace` which will be resolved to a `ResolvedNamespace`. This should simply the code base and make it a bit easier to add a new command.

### Why are the changes needed?

Clean up codebase.

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

No

### How was this patch tested?

Existing tests should cover the changes.

Closes #27125 from imback82/SPARK-30214-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-08 19:33:19 +08:00
Zhenhua Wang 9535776e28 [SPARK-30302][SQL] Complete info for show create table for views
### What changes were proposed in this pull request?

Add table/column comments and table properties to the result of show create table of views.

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

When show create table for views, after this patch, the result can contain table/column comments and table properties if they exist.

### How was this patch tested?

add new tests

Closes #26944 from wzhfy/complete_show_create_view.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-08 11:28:37 +09:00
Pablo Langa 9479887ba1 [SPARK-30039][SQL] CREATE FUNCTION should do multi-catalog resolution
### What changes were proposed in this pull request?

Add CreateFunctionStatement and make CREATE FUNCTION 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
CREATE FUNCTION namespace.function

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

Yes. When running CREATE FUNCTION namespace.function Spark fails the command if the current catalog is set to a v2 catalog.

### How was this patch tested?

Unit tests.

Closes #26890 from planga82/feature/SPARK-30039_CreateFunctionV2Command.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-08 00:38:15 +08:00
Kent Yao 8c121b0827 [SPARK-30431][SQL] Update SqlBase.g4 to create commentSpec pattern like locationSpec
### What changes were proposed in this pull request?

In `SqlBase.g4`, the `comment` clause is used as `COMMENT comment=STRING` and `COMMENT STRING` in many places.

While the `location` clause often appears along with the `comment` clause with a pattern defined as
```sql
locationSpec
    : LOCATION STRING
    ;
```
Then, we have to visit `locationSpec` as a `List` but comment as a single token.

We defined `commentSpec` for the comment clause to simplify and unify the grammar and the invocations.

### Why are the changes needed?

To simplify the grammar.

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

no
### How was this patch tested?

existing tests

Closes #27102 from yaooqinn/SPARK-30431.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 22:12:09 +08:00
Terry Kim 314e70fe23 [SPARK-30214][SQL] V2 commands resolves namespaces with new resolution framework
### What changes were proposed in this pull request?

#26847 introduced new framework for resolving catalog/namespaces. This PR proposes to integrate commands that need to resolve namespaces into the new framework.

### Why are the changes needed?

This is one of the work items for moving into the new resolution framework. Resolving v1/v2 tables with the new framework will be followed up in different PRs.

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

No

### How was this patch tested?

Existing tests should cover the changes.

Closes #27095 from imback82/unresolved_ns.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 21:32:08 +08:00
HyukjinKwon 866b7df348 [SPARK-30335][SQL][DOCS] Add a note first, last, collect_list and collect_set can be non-deterministic in SQL function docs as well
### What changes were proposed in this pull request?
This PR adds a note first and last can be non-deterministic in SQL function docs as well.
This is already documented in `functions.scala`.

### Why are the changes needed?
Some people look reading SQL docs only.

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

### How was this patch tested?
Jenkins will test.

Closes #27099 from HyukjinKwon/SPARK-30335.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-07 14:31:59 +09:00
Josh Rosen 7a1a5db35f [SPARK-30414][SQL] ParquetRowConverter optimizations: arrays, maps, plus misc. constant factors
### What changes were proposed in this pull request?

This PR implements multiple performance optimizations for `ParquetRowConverter`, achieving some modest constant-factor wins for all fields and larger wins for map and array fields:

- Add `private[this]` to several `val`s (90cebf080a5d3857ea8cf2a89e8e060b8b5a2fbf)
- Keep a `fieldUpdaters` array, saving two`.updater()` calls per field (7318785d350cc924198d7514e40973fd76d54ad5): I suspect that these are often megamorphic calls, so cutting these out seems like it could be a relatively large performance win.
- Only call `currentRow.numFields` once per `start()` call (e05de150813b639929c18af1df09ec718d2d16fc): previously we'd call it once per field and this had a significant enough cost that it was visible during profiling.
- Reuse buffers in array and map converters (c7d1534685fbad5d2280b082f37bed6d75848e76, 6d16f596ef6af9fd8946a062f79d0eeace9e1959): previously we would create a brand-new Scala `ArrayBuffer` for each field read, but this isn't actually necessary because the data is already copied into a fresh array when `end()` constructs a `GenericArrayData`.

### Why are the changes needed?

To improve Parquet read performance; this is complementary to #26993's (orthogonal) improvements for nested struct read performance.

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

No.

### How was this patch tested?

Existing tests, plus manual benchmarking with both synthetic and realistic schemas (similar to the ones in #26993). I've seen ~10%+ improvements in scan performance on certain real-world datasets.

Closes #27089 from JoshRosen/joshrosen/more-ParquetRowConverter-optimizations.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-07 14:30:10 +09:00
Josh Rosen 93d3ab88cd [SPARK-30338][SQL] Avoid unnecessary InternalRow copies in ParquetRowConverter
### What changes were proposed in this pull request?

This PR modifies `ParquetRowConverter` to remove unnecessary `InternalRow.copy()` calls for structs that are directly nested in other structs.

### Why are the changes needed?

These changes  can significantly improve performance when reading Parquet files that contain deeply-nested structs with many fields.

The `ParquetRowConverter` uses per-field `Converter`s for handling individual fields. Internally, these converters may have mutable state and may return mutable objects. In most cases, each `converter` is only invoked once per Parquet record (this is true for top-level fields, for example). However, arrays and maps may call their child element converters multiple times per Parquet record: in these cases we must be careful to copy any mutable outputs returned by child converters.

In the existing code, `InternalRow`s are copied whenever they are stored into _any_ parent container (not just maps and arrays). This copying can be especially expensive for deeply-nested fields, since a deep copy is performed at every level of nesting.

This PR modifies the code to avoid copies for structs that are directly nested in structs; see inline code comments for an argument for why this is safe.

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

No.

### How was this patch tested?

**Correctness**:  I added new test cases to `ParquetIOSuite` to increase coverage of nested structs, including structs nested in arrays: previously this suite didn't test that case, so we used to lack mutation coverage of this `copy()` code (the suite's tests still passed if I incorrectly removed the `.copy()` in all cases). I also added a test for maps with struct keys and modified the existing "map with struct values" test case include maps with two elements (since the incorrect omission of a `copy()` can only be detected if the map has multiple elements).

**Performance**: I put together a simple local benchmark demonstrating the performance problems:

First, construct a nested schema:

```scala
  case class Inner(
    f1: Int,
    f2: Long,
    f3: String,
    f4: Int,
    f5: Long,
    f6: String,
    f7: Int,
    f8: Long,
    f9: String,
    f10: Int
  )

  case class Wrapper1(inner: Inner)
  case class Wrapper2(wrapper1: Wrapper1)
  case class Wrapper3(wrapper2: Wrapper2)
```

`Wrapper3`'s schema looks like:

```
root
 |-- wrapper2: struct (nullable = true)
 |    |-- wrapper1: struct (nullable = true)
 |    |    |-- inner: struct (nullable = true)
 |    |    |    |-- f1: integer (nullable = true)
 |    |    |    |-- f2: long (nullable = true)
 |    |    |    |-- f3: string (nullable = true)
 |    |    |    |-- f4: integer (nullable = true)
 |    |    |    |-- f5: long (nullable = true)
 |    |    |    |-- f6: string (nullable = true)
 |    |    |    |-- f7: integer (nullable = true)
 |    |    |    |-- f8: long (nullable = true)
 |    |    |    |-- f9: string (nullable = true)
 |    |    |    |-- f10: integer (nullable = true)
```

Next, generate some fake data:

```scala
  val data = spark.range(1, 1000 * 1000 * 25, 1, 1).map { i =>
    Wrapper3(Wrapper2(Wrapper1(Inner(
      i.toInt,
      i * 2,
      (i * 3).toString,
      (i * 4).toInt,
      i * 5,
      (i * 6).toString,
      (i * 7).toInt,
      i * 8,
      (i * 9).toString,
      (i * 10).toInt
    ))))
  }

  data.write.mode("overwrite").parquet("/tmp/parquet-test")
```

I then ran a simple benchmark consisting of

```
spark.read.parquet("/tmp/parquet-test").selectExpr("hash(*)").rdd.count()
```

where the `hash(*)` is designed to force decoding of all Parquet fields but avoids `RowEncoder` costs in the `.rdd.count()` stage.

In the old code, expensive copying takes place at every level of nesting; this is apparent in the following flame graph:

![image](https://user-images.githubusercontent.com/50748/71389014-88a15380-25af-11ea-9537-3e87a2aef179.png)

After this PR's changes, the above toy benchmark runs ~30% faster.

Closes #26993 from JoshRosen/joshrosen/faster-parquet-nested-scan-by-avoiding-copies.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 13:01:37 +08:00
Yuming Wang 17881a467a [SPARK-19784][SPARK-25403][SQL] Refresh the table even table stats is empty
## What changes were proposed in this pull request?

We invalidate table relation once table data is changed by [SPARK-21237](https://issues.apache.org/jira/browse/SPARK-21237). But there is a situation we have not invalidated(`spark.sql.statistics.size.autoUpdate.enabled=false` and `table.stats.isEmpty`):
07c4b9bd1f/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala (L44-L54)

This will introduce some issues, e.g. [SPARK-19784](https://issues.apache.org/jira/browse/SPARK-19784), [SPARK-19845](https://issues.apache.org/jira/browse/SPARK-19845), [SPARK-25403](https://issues.apache.org/jira/browse/SPARK-25403), [SPARK-25332](https://issues.apache.org/jira/browse/SPARK-25332) and [SPARK-28413](https://issues.apache.org/jira/browse/SPARK-28413).

This is a example to reproduce [SPARK-19784](https://issues.apache.org/jira/browse/SPARK-19784):
```scala
val path = "/tmp/spark/parquet"
spark.sql("CREATE TABLE t (a INT) USING parquet")
spark.sql("INSERT INTO TABLE t VALUES (1)")
spark.range(5).toDF("a").write.parquet(path)
spark.sql(s"ALTER TABLE t SET LOCATION '${path}'")
spark.table("t").count() // return 1
spark.sql("refresh table t")
spark.table("t").count() // return 5
```

This PR invalidates the table relation in this case(`spark.sql.statistics.size.autoUpdate.enabled=false` and `table.stats.isEmpty`) to fix this issue.

## How was this patch tested?

unit tests

Closes #22721 from wangyum/SPARK-25403.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 11:41:34 +08:00
Ximo Guanter 604d6799df [SPARK-30226][SQL] Remove withXXX functions in WriteBuilder
### What changes were proposed in this pull request?
Adding a `LogicalWriteInfo` interface as suggested by cloud-fan in https://github.com/apache/spark/pull/25990#issuecomment-555132991

### Why are the changes needed?
It provides compile-time guarantees where we previously had none, which will make it harder to introduce bugs in the future.

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

### How was this patch tested?
Compiles and passes tests

Closes #26678 from edrevo/add-logical-write-info.

Lead-authored-by: Ximo Guanter <joaquin.guantergonzalbez@telefonica.com>
Co-authored-by: Ximo Guanter
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-06 23:53:45 +08:00
angerszhu 3eade744f8 [SPARK-29800][SQL] Rewrite non-correlated EXISTS subquery use ScalaSubquery to optimize perf
### What changes were proposed in this pull request?

Current catalyst rewrite non-correlated exists subquery to BroadcastNestLoopJoin, it's performance is not good , now we rewrite non-correlated EXISTS subquery to ScalaSubquery to optimize the performance.
We rewrite
```
 WHERE EXISTS (SELECT A FROM TABLE B WHERE COL1 > 10)
```
to
```
 WHERE (SELECT 1 FROM (SELECT A FROM TABLE B WHERE COL1 > 10) LIMIT 1) IS NOT NULL
```
to avoid build join to solve EXISTS expression.

### Why are the changes needed?
Optimize EXISTS performance.

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

### How was this patch tested?
Manuel Tested

Closes #26437 from AngersZhuuuu/SPARK-29800.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-06 22:54:37 +08:00
Wenchen Fan be4faafee4 Revert "[SPARK-23264][SQL] Make INTERVAL keyword optional when ANSI enabled"
### What changes were proposed in this pull request?

Revert https://github.com/apache/spark/pull/20433 .
### Why are the changes needed?

According to the SQL standard, the INTERVAL prefix is required:
```
<interval literal> ::=
  INTERVAL [ <sign> ] <interval string> <interval qualifier>

<interval string> ::=
  <quote> <unquoted interval string> <quote>
```

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

yes, but omitting the INTERVAL prefix is a new feature in 3.0

### How was this patch tested?

existing tests

Closes #27080 from cloud-fan/interval.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-03 12:51:10 -08:00
yi.wu e38964c442 [SPARK-29768][SQL][FOLLOW-UP] Improve handling non-deterministic filter of ScanOperation
### What changes were proposed in this pull request?

1. For `ScanOperation`, if it collects more than one filters, then all filters must be deterministic. And filter can be non-deterministic iff there's only one collected filter.

2. `FileSourceStrategy` should filter out non-deterministic filter, as it will hit haven't initialized exception if it's a partition related filter.

### Why are the changes needed?

Strictly follow `CombineFilters`'s behavior which doesn't allow combine two filters where non-deterministic predicates exist. And avoid hitting exception for file source.

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

No

### How was this patch tested?

Test exists.

Closes #27073 from Ngone51/SPARK-29768-FOLLOWUP.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 21:48:14 +08:00
Kent Yao c49388a484 [SPARK-30214][SQL] A new framework to resolve v2 commands
### What changes were proposed in this pull request?
Currently, we have a v2 adapter for v1 catalog (`V2SessionCatalog`), all the table/namespace commands can be implemented via v2 APIs.

Usually, a command needs to know which catalog it needs to operate, but different commands have different requirements about what to resolve. A few examples:

  - `DROP NAMESPACE`: only need to know the name of the namespace.
  - `DESC NAMESPACE`: need to lookup the namespace and get metadata, but is done during execution
  - `DROP TABLE`: need to do lookup and make sure it's a table not (temp) view.
  - `DESC TABLE`: need to lookup the table and get metadata.

For namespaces, the analyzer only needs to find the catalog and the namespace name. The command can do lookup during execution if needed.

For tables, mostly commands need the analyzer to do lookup.

Note that, table and namespace have a difference: `DESC NAMESPACE testcat` works and describes the root namespace under `testcat`, while `DESC TABLE testcat` fails if there is no table `testcat` under the current catalog. It's because namespaces can be named [], but tables can't. The commands should explicitly specify it needs to operate on namespace or table.

In this Pull Request, we introduce a new framework to resolve v2 commands:
1. parser creates logical plans or commands with `UnresolvedNamespace`/`UnresolvedTable`/`UnresolvedView`/`UnresolvedRelation`. (CREATE TABLE still keeps Seq[String], as it doesn't need to look up relations)
2. analyzer converts
2.1 `UnresolvedNamespace` to `ResolvesNamespace` (contains catalog and namespace identifier)
2.2 `UnresolvedTable` to `ResolvedTable` (contains catalog, identifier and `Table`)
2.3 `UnresolvedView` to `ResolvedView` (will be added later when we migrate view commands)
2.4 `UnresolvedRelation` to relation.
3. an extra analyzer rule to match commands with `V1Table` and converts them to corresponding v1 commands. This will be added later when we migrate existing commands
4. planner matches commands and converts them to the corresponding physical nodes.

We also introduce brand new v2 commands - the `comment` syntaxes to illustrate how to work with the newly added framework.
```sql
COMMENT ON (DATABASE|SCHEMA|NAMESPACE) ... IS ...
COMMENT ON TABLE ... IS ...
```
Details about the `comment` syntaxes:
As the new design of catalog v2, some properties become reserved, e.g. `location`, `comment`. We are going to disable setting reserved properties by dbproperties or tblproperites directly to avoid confliction with their related subClause or specific commands.

They are the best practices from PostgreSQL and presto.

https://www.postgresql.org/docs/12/sql-comment.html
https://prestosql.io/docs/current/sql/comment.html

Mostly, the basic thoughts of the new framework came from the discussions bellow with cloud-fan,  https://github.com/apache/spark/pull/26847#issuecomment-564510061,

### Why are the changes needed?
To make it easier to add new v2 commands, and easier to unify the table relation behavior.

### Does this PR introduce any user-facing change?
yes, add new syntax

### How was this patch tested?

add uts.

Closes #26847 from yaooqinn/SPARK-30214.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 16:09:06 +08:00
Maxim Gekk 51373467cc [SPARK-30412][SQL][TESTS] Eliminate warnings in Java tests regarding to deprecated Spark SQL API
### What changes were proposed in this pull request?
In the PR, I propose to add the `SuppressWarnings("deprecation")` annotation to Java tests for deprecated Spark SQL APIs.

### Why are the changes needed?
This eliminates the following warnings:
```
sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java
    Warning:Warning:line (32)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (91)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (100)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (109)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (118)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated

sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java
    Warning:Warning:line (28)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (37)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (46)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (55)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (64)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated

sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
    Warning:Warning:line (478)java: json(org.apache.spark.api.java.JavaRDD<java.lang.String>) in org.apache.spark.sql.DataFrameReader has been deprecated
```
and highlights warnings about real problems.

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

### How was this patch tested?
By existing test suites `Java8DatasetAggregatorSuite.java`, `JavaDataFrameSuite.java` and `JavaDatasetAggregatorSuite.java`.

Closes #27081 from MaxGekk/eliminate-warnings-part2.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-03 13:26:48 +09:00
Maxim Gekk a469976e6e [SPARK-29930][SQL][FOLLOW-UP] Allow only default value to be set for removed SQL configs
### What changes were proposed in this pull request?
In the PR, I propose to throw `AnalysisException` when a removed SQL config is set to non-default value. The following SQL configs removed by #26559 are marked as removed:
1. `spark.sql.fromJsonForceNullableSchema`
2. `spark.sql.legacy.compareDateTimestampInTimestamp`
3. `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation`

### Why are the changes needed?
To improve user experience with Spark SQL by notifying of removed SQL configs used by users.

### Does this PR introduce any user-facing change?
Yes, before the `set` command was silently ignored:
```sql
spark-sql> set spark.sql.fromJsonForceNullableSchema=false;
spark.sql.fromJsonForceNullableSchema	false
```
after the exception should be raised:
```sql
spark-sql> set spark.sql.fromJsonForceNullableSchema=false;
Error in query: The SQL config 'spark.sql.fromJsonForceNullableSchema' was removed in the version 3.0.0. It was removed to prevent errors like SPARK-23173 for non-default value.;
```

### How was this patch tested?
Added new tests into `SQLConfSuite` for both cases when removed SQL configs are set to default and non-default values.

Closes #27057 from MaxGekk/remove-sql-configs-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-03 10:41:30 +09:00
Kent Yao e04309cb1f [SPARK-30341][SQL] Overflow check for interval arithmetic operations
### What changes were proposed in this pull request?

1. For the interval arithmetic functions, e.g. `add`/`subtract`/`negative`/`multiply`/`divide`, enable overflow check when `ANSI` is on.

2. For `multiply`/`divide`,  throw an exception when an overflow happens in spite of `ANSI` is on/off.

3. `add`/`subtract`/`negative` stay the same for backward compatibility.

4. `divide` by 0 throws ArithmeticException whether `ANSI` or not as same as numerics.

5. These behaviors fit the numeric type operations fully when ANSI is on.

6. These behaviors fit the numeric type operations fully when ANSI is off, except 2 and 4.

### Why are the changes needed?

1. bug fix
2. `ANSI` support

### Does this PR introduce any user-facing change?
When `ANSI` is on, interval `add`/`subtract`/`negative`/`multiply`/`divide` will overflow if any field overflows

### How was this patch tested?

add unit tests

Closes #26995 from yaooqinn/SPARK-30341.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 02:04:20 +08:00
Wenchen Fan 1743d5be7f [SPARK-30284][SQL] CREATE VIEW should keep the current catalog and namespace
### What changes were proposed in this pull request?

Update CREATE VIEW command to store the current catalog and namespace instead of current database in view metadata. Also update analyzer to leverage the catalog and namespace in view metastore to resolve relations inside views.

Note that, this PR still keeps the way we resolve views, by recursively calling Analyzer. This is necessary because view text may contain CTE, window spec, etc. which needs rules outside of the main resolution batch (e.g. `CTESubstitution`)

### Why are the changes needed?

To resolve relations inside view correctly.

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

Yes, fix a bug. Now tables referred by a view can be resolved correctly even if the current catalog/namespace has been updated.

### How was this patch tested?

a new test

Closes #26923 from cloud-fan/view.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 01:41:32 +08:00
jiake 6bd5494f34 [SPARK-30403][SQL] fix the NoSuchElementException when enable AQE with InSubquery expression
### What changes were proposed in this pull request?
This PR aim to fix the NoSuchElementException  exception when enable AQE with insubquery expression.

### Why are the changes needed?
Fix exception

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

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

Closes #27068 from JkSelf/fixSubqueryIssue.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-02 23:11:56 +08:00
jiake 05f7b57ddc [SPARK-30407][SQL] fix the reset metric issue when enable AQE
### What changes were proposed in this pull request?
When working on [PR#26813](https://github.com/apache/spark/pull/26813), we encounter the exception in [here(the number of metrics(1) is 2 not 1 )](5d870ef0bc/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala (L120)). This PR fix the above exception.

### Why are the changes needed?
Fix exception

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

### How was this patch tested?
[this test with enable AQE](5d870ef0bc/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala (L120))

Closes #27074 from JkSelf/resetMetricsIssue.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-02 21:55:36 +08:00
Maxim Gekk b316d37365 [SPARK-30401][SQL] Call requireNonStaticConf() only once in set()
### What changes were proposed in this pull request?
Calls of `requireNonStaticConf()` are removed from the `set()` methods in RuntimeConfig because those methods invoke `def set(key: String, value: String): Unit` where `requireNonStaticConf()` is called as well.

### Why are the changes needed?
To avoid unnecessary calls of `requireNonStaticConf()`.

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

### How was this patch tested?
By existing tests from `SQLConfSuite`

Closes #27062 from MaxGekk/call-requireNonStaticConf-once.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-02 09:56:50 +09:00
Jungtaek Lim (HeartSaVioR) e054a0af6f [SPARK-29348][SQL][FOLLOWUP] Fix slight bug on streaming example for Dataset.observe
### What changes were proposed in this pull request?

This patch fixes a small bug in the example of streaming query, as the type of observable metrics is Java Map instead of Scala Map, so to use foreach it should be converted first.

### Why are the changes needed?

Described above.

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

No.

### How was this patch tested?

Ran below query via `spark-shell`:

**Streaming**

```scala
import scala.collection.JavaConverters._
import scala.util.Random
import org.apache.spark.sql.streaming.StreamingQueryListener
import org.apache.spark.sql.streaming.StreamingQueryListener._

spark.streams.addListener(new StreamingQueryListener() {
  override def onQueryProgress(event: QueryProgressEvent): Unit = {
    event.progress.observedMetrics.asScala.get("my_event").foreach { row =>
      // Trigger if the number of errors exceeds 5 percent
      val num_rows = row.getAs[Long]("rc")
      val num_error_rows = row.getAs[Long]("erc")
      val ratio = num_error_rows.toDouble / num_rows
      if (ratio > 0.05) {
        // Trigger alert
        println(s"alert! error ratio: $ratio")
      }
    }
  }

  def onQueryStarted(event: QueryStartedEvent): Unit = {}
  def onQueryTerminated(event: QueryTerminatedEvent): Unit = {}
})

val rates = spark
  .readStream
  .format("rate")
  .option("rowsPerSecond", 10)
  .load

val rand = new Random()
val df = rates.map { row => (row.getLong(1), if (row.getLong(1) % 2 == 0) "error" else null) }.toDF
val ds = df.selectExpr("_1 AS id", "_2 AS error")
// Observe row count (rc) and error row count (erc) in the batch Dataset
val observed_ds = ds.observe("my_event", count(lit(1)).as("rc"), count($"error").as("erc"))
observed_ds.writeStream.format("console").start()
```

Closes #27046 from HeartSaVioR/SPARK-29348-FOLLOWUP.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-31 01:08:25 +09:00
HyukjinKwon 7079e871a7 [SPARK-30185][SQL] Implement Dataset.tail API
### What changes were proposed in this pull request?

This PR proposes a `tail` API.

Namely, as below:

```scala
scala> spark.range(10).head(5)
res1: Array[Long] = Array(0, 1, 2, 3, 4)
scala> spark.range(10).tail(5)
res2: Array[Long] = Array(5, 6, 7, 8, 9)
```

Implementation details will be similar with `head` but it will be reversed:

1. Run the job against the last partition and collect rows. If this is enough, return as is.
2. If this is not enough, calculate the number of partitions to select more based upon
 `spark.sql.limit.scaleUpFactor`
3. Run more jobs against more partitions (in a reversed order compared to head) as many as the number calculated from 2.
4. Go to 2.

**Note that**, we don't guarantee the natural order in DataFrame in general - there are cases when it's deterministic and when it's not. We probably should write down this as a caveat separately.

### Why are the changes needed?

Many other systems support the way to take data from the end, for instance, pandas[1] and
 Python[2][3]. Scala collections APIs also have head and tail

On the other hand, in Spark, we only provide a way to take data from the start
 (e.g., DataFrame.head).

This has been requested multiple times here and there in Spark user mailing list[4], StackOverFlow[5][6], JIRA[7] and other third party projects such as
 Koalas[8]. In addition, this missing API seems explicitly mentioned in comparison to another system[9] time to time.

It seems we're missing non-trivial use case in Spark and this motivated me to propose this API.

[1] https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.tail.html?highlight=tail#pandas.DataFrame.tail
[2] https://stackoverflow.com/questions/10532473/head-and-tail-in-one-line
[3] https://stackoverflow.com/questions/646644/how-to-get-last-items-of-a-list-in-python
[4] http://apache-spark-user-list.1001560.n3.nabble.com/RDD-tail-td4217.html
[5] https://stackoverflow.com/questions/39544796/how-to-select-last-row-and-also-how-to-access-pyspark-dataframe-by-index
[6] https://stackoverflow.com/questions/45406762/how-to-get-the-last-row-from-dataframe
[7] https://issues.apache.org/jira/browse/SPARK-26433
[8] https://github.com/databricks/koalas/issues/343
[9] https://medium.com/chris_bour/6-differences-between-pandas-and-spark-dataframes-1380cec394d2

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

No, (new API)

### How was this patch tested?

Unit tests were added and manually tested.

Closes #26809 from HyukjinKwon/wip-tail.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-31 01:07:09 +09:00
Xiao Li 919d551ddb Revert "[SPARK-29390][SQL] Add the justify_days(), justify_hours() and justif_interval() functions"
This reverts commit f926809a1f.

Closes #27032 from gatorsmile/revertSPARK-29390.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-12-29 15:25:14 -08:00
sandeep katta 16e5e79877 [SPARK-28670][SQL] create function should thrown Exception if the resource is not found
## What changes were proposed in this pull request?

Create temporary or permanent function it should throw AnalysisException if the resource is not found. Need to keep behavior consistent across permanent and temporary functions.

## How was this patch tested?

Added UT and also tested manually

**Before Fix**
If the UDF resource is not present then on creation of temporary function it throws AnalysisException where as for permanent function it does not throw. Permanent funtcion  throws AnalysisException only after select operation is performed.

**After Fix**

For temporary and permanent function check for the resource, if the UDF resource is not found then throw AnalysisException

![rt](https://user-images.githubusercontent.com/35216143/62781519-d1131580-bad5-11e9-9d58-69e65be86c03.png)

Closes #25399 from sandeep-katta/funcIssue.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-28 14:35:33 +09:00