Commit graph

10957 commits

Author SHA1 Message Date
Wenchen Fan 0d5d248bdc [SPARK-34508][SQL][TEST] Skip HiveExternalCatalogVersionsSuite if network is down
### What changes were proposed in this pull request?

It's possible that the network is down when running Spark tests, and it's annoying to see `HiveExternalCatalogVersionsSuite` keep failing.

This PR proposes to skip this test suite if we can't get the latest Spark version from the Apache website.

### Why are the changes needed?

Make the Spark tests more robust.

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

no

### How was this patch tested?

N/A

Closes #31627 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-23 13:35:29 -08:00
Huaxin Gao 443139b601 [SPARK-34502][SQL] Remove unused parameters in join methods
### What changes were proposed in this pull request?

Remove unused parameters in `CoalesceBucketsInJoin`, `UnsafeCartesianRDD` and `ShuffledHashJoinExec`.

### Why are the changes needed?
Clean up

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

### How was this patch tested?
Existing tests

Closes #31617 from huaxingao/join-minor.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-02-23 12:18:43 -08:00
Wenchen Fan 429f8af9b6 Revert "[SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command"
This reverts commit 9a566f83a0.
2021-02-24 02:38:22 +08:00
Max Gekk 8f994cbb4a [SPARK-34475][SQL] Rename logical nodes of v2 ALTER commands
### What changes were proposed in this pull request?
In the PR, I propose to rename logical nodes of v2 commands in the form: `<verb> + <object>` like:
- AlterTableAddPartition -> AddPartition
- AlterTableSetLocation -> SetTableLocation

### Why are the changes needed?
1. For simplicity and readability of logical plans
2. For consistency with other logical nodes. For example, the logical node `RenameTable` for `ALTER TABLE .. RENAME TO` was added before `AlterTableRenamePartition`.

### Does this PR introduce _any_ user-facing change?
Should not since this is non-public APIs.

### How was this patch tested?
1. Check scala style: `./dev/scalastyle`
2. Affected test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite"
```

Closes #31596 from MaxGekk/rename-alter-table-logic-nodes.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-23 12:04:31 +00:00
Linhong Liu be675a052c [SPARK-34490][SQL] Analysis should fail if the view refers a dropped table
### What changes were proposed in this pull request?
When resolving a view, we use the captured view name in `AnalysisContext` to
distinguish whether a relation name is a view or a table. But if the resolution failed,
other rules (e.g. `ResolveTables`) will try to resolve the relation again but without
`AnalysisContext`. So, in this case, the resolution may be incorrect. For example,
if the view refers to a dropped table while a view with the same name exists, the
dropped table will be resolved as a view rather than an unresolved exception.

### Why are the changes needed?
bugfix

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

### How was this patch tested?
newly added test cases

Closes #31606 from linhongliu-db/fix-temp-view-master.

Lead-authored-by: Linhong Liu <linhong.liu@databricks.com>
Co-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-23 15:51:02 +08:00
Kousuke Saruta 612d52315b [SPARK-34500][DOCS][EXAMPLES] Replace symbol literals with $"" in examples and documents
### What changes were proposed in this pull request?

This PR replaces all the occurrences of symbol literals (`'name`) with string interpolation (`$"name"`) in examples and documents.

### Why are the changes needed?

Symbol literals are used to represent columns in Spark SQL but the Scala community seems to remove `Symbol` completely.
As we discussed in #31569, first we should replacing symbol literals with `$"name"` in user facing examples and documents.

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

No.

### How was this patch tested?

Build docs.

Closes #31615 from sarutak/replace-symbol-literals-in-doc-and-examples.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-23 11:22:02 +09:00
Max Gekk 7df4fed420 [MINOR][SQL] Fix the comment for CalendarIntervalType about comparability
### What changes were proposed in this pull request?
In the PR, I propose to revert https://github.com/apache/spark/pull/26659 partially regarding to comparability of interval values. The comment became incorrect after https://github.com/apache/spark/pull/27262.

### Why are the changes needed?
The comment is incorrect, and it might confuse Spark's devs/users.

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

### How was this patch tested?
By checking scala coding style `./dev/scalastyle`.

Closes #31610 from MaxGekk/doc-interval-not-comparable.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-22 21:29:14 +08:00
Wenchen Fan 02c784ca68 [SPARK-34473][SQL] Avoid NPE in DataFrameReader.schema(StructType)
### What changes were proposed in this pull request?

This fixes a regression in `DataFrameReader.schema(StructType)`, to avoid NPE if the given `StructType` is null. Note that, passing null to Spark public APIs leads to undefined behavior. There is no document mentioning the null behavior, and it's just an accident that `DataFrameReader.schema(StructType)` worked before. So I think this is not a 3.1 blocker.

### Why are the changes needed?

It fixes a 3.1 regression

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

yea, now `df.read.schema(null: StructType)` is a noop as before, while in the current branch-3.1 it throws NPE.

### How was this patch tested?

It's undefined behavior and is very obvious, so I didn't add a test. We should add tests when we clearly define and fix the null behavior for all public APIs.

Closes #31593 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-22 21:11:21 +08:00
kevincmchen 9767041153 [SPARK-34432][SQL][TESTS] Add JavaSimpleWritableDataSource
### What changes were proposed in this pull request?

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

In #19269 , there is only a scala implementation of simple writable data source in `DataSourceV2Suite`.

This PR adds a java implementation of it.

### Why are the changes needed?

To improve test coverage.

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

No

### How was this patch tested?

existing testsuites

Closes #31560 from kevincmchen/SPARK-34432.

Lead-authored-by: kevincmchen <kevincmchen@tencent.com>
Co-authored-by: Kevin Pis <68981916+kevincmchen@users.noreply.github.com>
Co-authored-by: Kevin Pis <kc4163568@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-22 09:38:13 +00:00
Max Gekk 23a5996a46 [SPARK-34450][SQL][TESTS] Unify v1 and v2 ALTER TABLE .. RENAME tests
### What changes were proposed in this pull request?
1. Move parser tests from `DDLParserSuite` to `AlterTableRenameParserSuite`.
2. Port DS v1 tests from `DDLSuite` and other test suites to `v1.AlterTableRenameBase` and to `v1.AlterTableRenameSuite`.
3. Add a test for DSv2 `ALTER TABLE .. RENAME` to `v2.AlterTableRenameSuite`.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenameSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenameParserSuite"
```

Closes #31575 from MaxGekk/unify-rename-table-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-22 08:36:16 +00:00
Dongjoon Hyun 2fb5f21b1e [SPARK-34495][TESTS] Add DedicatedJVMTest test tag
### What changes were proposed in this pull request?

This PR aims to add a test tag, `DedicatedJVMTest`, and replace `SecurityTest` with this.

### Why are the changes needed?

To have a reusable general test tag.

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

No.

### How was this patch tested?

Pass the CIs.

Closes #31607 from dongjoon-hyun/SPARK-34495.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-22 16:00:48 +09:00
Raza Jafri 38fbe560fd [SPARK-34167][SQL] Reading parquet with IntDecimal written as a LongDecimal blows up
### What changes were proposed in this pull request?
If an IntDecimal type was written as a LongDecimal in a parquet file. Spark should read it as a long from `VectorizedValuesReader` but write it to the `WritableColumnVector` as an int by down-casting it and calling the appropriate method. `readLongs` has been modified to take in a boolean flag that tells it if the number would fit in a 32-bit Decimal and subsequently downsized.

### Why are the changes needed?
If a Parquet file writes an IntDecimal as LongDecimal, which is supported by the parquet spec, Spark will not be able to read it and will throw an exception.  The reason this happens is because method `readLong` tries to write the long to a `WritableColumnVector` which has been initialized to accept only Ints which leads to a `NullPointerException`.

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

### How was this patch tested?
manually tested and added unit-test

Closes #31284 from razajafri/decimal_fix.

Authored-by: Raza Jafri <rjafri@nvidia.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-22 04:48:56 +00:00
Max Gekk a22d20a6ca [SPARK-34468][SQL] Rename v2 table in place if new name has single part
### What changes were proposed in this pull request?
If new table name consists of single part (no namespaces), the v2 `ALTER TABLE .. RENAME TO` command renames the table while keeping it in the same namespace. For example:
```sql
ALTER TABLE catalog_name.ns1.ns2.ns3.ns4.ns5.tbl RENAME TO new_table
```
the command should rename the source table to `catalog_name.ns1.ns2.ns3.ns4.ns5.new_table`. Before the changes, the command moves the table to the "root" name space i.e. `catalog_name.new_table`.

### Why are the changes needed?
To have the same behavior as v1 implementation of `ALTER TABLE .. RENAME TO`, and other DBMSs.

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

### How was this patch tested?
By running new test:
```
$ build/sbt "sql/test:testOnly *DataSourceV2SQLSuite"
```

Closes #31594 from MaxGekk/rename-table-single-part.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-22 04:43:19 +00:00
Max Gekk 6ea4b5fda7 [SPARK-34401][SQL][DOCS] Update docs about altering cached tables/views
### What changes were proposed in this pull request?
Update public docs of SQL commands about altering cached tables/views. For instance:
<img width="869" alt="Screenshot 2021-02-08 at 15 11 48" src="https://user-images.githubusercontent.com/1580697/107217940-fd3b8980-6a1f-11eb-98b9-9b2e3fe7f4ef.png">

### Why are the changes needed?
To inform users about commands behavior in altering cached tables or views.

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

### How was this patch tested?
By running the command below and manually checking the docs:
```
$ SKIP_API=1 SKIP_SCALADOC=1 SKIP_PYTHONDOC=1 SKIP_RDOC=1 jekyll serve --watch
```

Closes #31524 from MaxGekk/doc-cmd-caching.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-22 04:32:09 +00:00
Dongjoon Hyun 03f4cf5845 [SPARK-34029][SQL][TESTS] Add OrcEncryptionSuite and FakeKeyProvider
### What changes were proposed in this pull request?

This is a retry of #31065 . Last time, the newly add test cases passed in Jenkins and individually, but it's reverted because they fail when `GitHub Action` runs with  `SERIAL_SBT_TESTS=1`.

In this PR, `SecurityTest` tag is used to isolate `KeyProvider`.

This PR aims to add a basis for columnar encryption test framework by add `OrcEncryptionSuite` and `FakeKeyProvider`.

Please note that we will improve more in both Apache Spark and Apache ORC in Apache Spark 3.2.0 timeframe.

### Why are the changes needed?

Apache ORC 1.6 supports columnar encryption.

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

No. This is for a test case.

### How was this patch tested?

Pass the newly added test suite.

Closes #31603 from dongjoon-hyun/SPARK-34486-RETRY.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-21 15:05:29 -08:00
Yuming Wang 94f9617cb4 [SPARK-34129][SQL] Add table name to LogicalRelation.simpleString
### What changes were proposed in this pull request?

This pr add table name to `LogicalRelation.simpleString`.

### Why are the changes needed?

Make optimized logical plan more readable.

Before this pr:
```
== Optimized Logical Plan ==
Project [i_item_sk#7 AS ss_item_sk#162], Statistics(sizeInBytes=8.07E+27 B)
+- Join Inner, (((i_brand_id#14 = brand_id#159) AND (i_class_id#16 = class_id#160)) AND (i_category_id#18 = category_id#161)), Statistics(sizeInBytes=2.42E+28 B)
   :- Project [i_item_sk#7, i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=8.5 MiB, rowCount=3.69E+5)
   :  +- Filter ((isnotnull(i_brand_id#14) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#18)), Statistics(sizeInBytes=150.0 MiB, rowCount=3.69E+5)
   :     +- Relation[i_item_sk#7,i_item_id#8,i_rec_start_date#9,i_rec_end_date#10,i_item_desc#11,i_current_price#12,i_wholesale_cost#13,i_brand_id#14,i_brand#15,i_class_id#16,i_class#17,i_category_id#18,i_category#19,i_manufact_id#20,i_manufact#21,i_size#22,i_formulation#23,i_color#24,i_units#25,i_container#26,i_manager_id#27,i_product_name#28] parquet, Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
   +- Aggregate [brand_id#159, class_id#160, category_id#161], [brand_id#159, class_id#160, category_id#161], Statistics(sizeInBytes=2.73E+21 B)
      +- Aggregate [brand_id#159, class_id#160, category_id#161], [brand_id#159, class_id#160, category_id#161], Statistics(sizeInBytes=2.73E+21 B)
         +- Join LeftSemi, (((brand_id#159 <=> i_brand_id#14) AND (class_id#160 <=> i_class_id#16)) AND (category_id#161 <=> i_category_id#18)), Statistics(sizeInBytes=2.73E+21 B)
            :- Join LeftSemi, (((brand_id#159 <=> i_brand_id#14) AND (class_id#160 <=> i_class_id#16)) AND (category_id#161 <=> i_category_id#18)), Statistics(sizeInBytes=2.73E+21 B)
            :  :- Project [i_brand_id#14 AS brand_id#159, i_class_id#16 AS class_id#160, i_category_id#18 AS category_id#161], Statistics(sizeInBytes=2.73E+21 B)
            :  :  +- Join Inner, (ss_sold_date_sk#51 = d_date_sk#52), Statistics(sizeInBytes=3.83E+21 B)
            :  :     :- Project [ss_sold_date_sk#51, i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=387.3 PiB)
            :  :     :  +- Join Inner, (ss_item_sk#30 = i_item_sk#7), Statistics(sizeInBytes=516.5 PiB)
            :  :     :     :- Project [ss_item_sk#30, ss_sold_date_sk#51], Statistics(sizeInBytes=61.1 GiB)
            :  :     :     :  +- Filter ((isnotnull(ss_item_sk#30) AND isnotnull(ss_sold_date_sk#51)) AND dynamicpruning#168 [ss_sold_date_sk#51]), Statistics(sizeInBytes=580.6 GiB)
            :  :     :     :     :  +- Project [d_date_sk#52], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
            :  :     :     :     :     +- Filter ((((d_year#58 >= 1999) AND (d_year#58 <= 2001)) AND isnotnull(d_year#58)) AND isnotnull(d_date_sk#52)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
            :  :     :     :     :        +- Relation[d_date_sk#52,d_date_id#53,d_date#54,d_month_seq#55,d_week_seq#56,d_quarter_seq#57,d_year#58,d_dow#59,d_moy#60,d_dom#61,d_qoy#62,d_fy_year#63,d_fy_quarter_seq#64,d_fy_week_seq#65,d_day_name#66,d_quarter_name#67,d_holiday#68,d_weekend#69,d_following_holiday#70,d_first_dom#71,d_last_dom#72,d_same_day_ly#73,d_same_day_lq#74,d_current_day#75,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
            :  :     :     :     +- Relation[ss_sold_time_sk#29,ss_item_sk#30,ss_customer_sk#31,ss_cdemo_sk#32,ss_hdemo_sk#33,ss_addr_sk#34,ss_store_sk#35,ss_promo_sk#36,ss_ticket_number#37L,ss_quantity#38,ss_wholesale_cost#39,ss_list_price#40,ss_sales_price#41,ss_ext_discount_amt#42,ss_ext_sales_price#43,ss_ext_wholesale_cost#44,ss_ext_list_price#45,ss_ext_tax#46,ss_coupon_amt#47,ss_net_paid#48,ss_net_paid_inc_tax#49,ss_net_profit#50,ss_sold_date_sk#51] parquet, Statistics(sizeInBytes=580.6 GiB)
            :  :     :     +- Project [i_item_sk#7, i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=8.5 MiB, rowCount=3.69E+5)
            :  :     :        +- Filter (((isnotnull(i_brand_id#14) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#18)) AND isnotnull(i_item_sk#7)), Statistics(sizeInBytes=150.0 MiB, rowCount=3.69E+5)
            :  :     :           +- Relation[i_item_sk#7,i_item_id#8,i_rec_start_date#9,i_rec_end_date#10,i_item_desc#11,i_current_price#12,i_wholesale_cost#13,i_brand_id#14,i_brand#15,i_class_id#16,i_class#17,i_category_id#18,i_category#19,i_manufact_id#20,i_manufact#21,i_size#22,i_formulation#23,i_color#24,i_units#25,i_container#26,i_manager_id#27,i_product_name#28] parquet, Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
            :  :     +- Project [d_date_sk#52], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
            :  :        +- Filter ((((d_year#58 >= 1999) AND (d_year#58 <= 2001)) AND isnotnull(d_year#58)) AND isnotnull(d_date_sk#52)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
            :  :           +- Relation[d_date_sk#52,d_date_id#53,d_date#54,d_month_seq#55,d_week_seq#56,d_quarter_seq#57,d_year#58,d_dow#59,d_moy#60,d_dom#61,d_qoy#62,d_fy_year#63,d_fy_quarter_seq#64,d_fy_week_seq#65,d_day_name#66,d_quarter_name#67,d_holiday#68,d_weekend#69,d_following_holiday#70,d_first_dom#71,d_last_dom#72,d_same_day_ly#73,d_same_day_lq#74,d_current_day#75,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
            :  +- Aggregate [i_brand_id#14, i_class_id#16, i_category_id#18], [i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=1414.2 EiB)
            :     +- Project [i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=1414.2 EiB)
            :        +- Join Inner, (cs_sold_date_sk#113 = d_date_sk#52), Statistics(sizeInBytes=1979.9 EiB)
            :           :- Project [cs_sold_date_sk#113, i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=231.1 PiB)
            :           :  +- Join Inner, (cs_item_sk#94 = i_item_sk#7), Statistics(sizeInBytes=308.2 PiB)
            :           :     :- Project [cs_item_sk#94, cs_sold_date_sk#113], Statistics(sizeInBytes=36.2 GiB)
            :           :     :  +- Filter ((isnotnull(cs_item_sk#94) AND isnotnull(cs_sold_date_sk#113)) AND dynamicpruning#169 [cs_sold_date_sk#113]), Statistics(sizeInBytes=470.5 GiB)
            :           :     :     :  +- Project [d_date_sk#52], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
            :           :     :     :     +- Filter ((((d_year#58 >= 1999) AND (d_year#58 <= 2001)) AND isnotnull(d_year#58)) AND isnotnull(d_date_sk#52)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
            :           :     :     :        +- Relation[d_date_sk#52,d_date_id#53,d_date#54,d_month_seq#55,d_week_seq#56,d_quarter_seq#57,d_year#58,d_dow#59,d_moy#60,d_dom#61,d_qoy#62,d_fy_year#63,d_fy_quarter_seq#64,d_fy_week_seq#65,d_day_name#66,d_quarter_name#67,d_holiday#68,d_weekend#69,d_following_holiday#70,d_first_dom#71,d_last_dom#72,d_same_day_ly#73,d_same_day_lq#74,d_current_day#75,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
            :           :     :     +- Relation[cs_sold_time_sk#80,cs_ship_date_sk#81,cs_bill_customer_sk#82,cs_bill_cdemo_sk#83,cs_bill_hdemo_sk#84,cs_bill_addr_sk#85,cs_ship_customer_sk#86,cs_ship_cdemo_sk#87,cs_ship_hdemo_sk#88,cs_ship_addr_sk#89,cs_call_center_sk#90,cs_catalog_page_sk#91,cs_ship_mode_sk#92,cs_warehouse_sk#93,cs_item_sk#94,cs_promo_sk#95,cs_order_number#96L,cs_quantity#97,cs_wholesale_cost#98,cs_list_price#99,cs_sales_price#100,cs_ext_discount_amt#101,cs_ext_sales_price#102,cs_ext_wholesale_cost#103,... 10 more fields] parquet, Statistics(sizeInBytes=470.5 GiB)
            :           :     +- Project [i_item_sk#7, i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=8.5 MiB, rowCount=3.72E+5)
            :           :        +- Filter isnotnull(i_item_sk#7), Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
            :           :           +- Relation[i_item_sk#7,i_item_id#8,i_rec_start_date#9,i_rec_end_date#10,i_item_desc#11,i_current_price#12,i_wholesale_cost#13,i_brand_id#14,i_brand#15,i_class_id#16,i_class#17,i_category_id#18,i_category#19,i_manufact_id#20,i_manufact#21,i_size#22,i_formulation#23,i_color#24,i_units#25,i_container#26,i_manager_id#27,i_product_name#28] parquet, Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
            :           +- Project [d_date_sk#52], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
            :              +- Filter ((((d_year#58 >= 1999) AND (d_year#58 <= 2001)) AND isnotnull(d_year#58)) AND isnotnull(d_date_sk#52)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
            :                 +- Relation[d_date_sk#52,d_date_id#53,d_date#54,d_month_seq#55,d_week_seq#56,d_quarter_seq#57,d_year#58,d_dow#59,d_moy#60,d_dom#61,d_qoy#62,d_fy_year#63,d_fy_quarter_seq#64,d_fy_week_seq#65,d_day_name#66,d_quarter_name#67,d_holiday#68,d_weekend#69,d_following_holiday#70,d_first_dom#71,d_last_dom#72,d_same_day_ly#73,d_same_day_lq#74,d_current_day#75,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
            +- Aggregate [i_brand_id#14, i_class_id#16, i_category_id#18], [i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=650.5 EiB)
               +- Project [i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=650.5 EiB)
                  +- Join Inner, (ws_sold_date_sk#147 = d_date_sk#52), Statistics(sizeInBytes=910.6 EiB)
                     :- Project [ws_sold_date_sk#147, i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=106.3 PiB)
                     :  +- Join Inner, (ws_item_sk#116 = i_item_sk#7), Statistics(sizeInBytes=141.7 PiB)
                     :     :- Project [ws_item_sk#116, ws_sold_date_sk#147], Statistics(sizeInBytes=16.6 GiB)
                     :     :  +- Filter ((isnotnull(ws_item_sk#116) AND isnotnull(ws_sold_date_sk#147)) AND dynamicpruning#170 [ws_sold_date_sk#147]), Statistics(sizeInBytes=216.4 GiB)
                     :     :     :  +- Project [d_date_sk#52], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
                     :     :     :     +- Filter ((((d_year#58 >= 1999) AND (d_year#58 <= 2001)) AND isnotnull(d_year#58)) AND isnotnull(d_date_sk#52)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
                     :     :     :        +- Relation[d_date_sk#52,d_date_id#53,d_date#54,d_month_seq#55,d_week_seq#56,d_quarter_seq#57,d_year#58,d_dow#59,d_moy#60,d_dom#61,d_qoy#62,d_fy_year#63,d_fy_quarter_seq#64,d_fy_week_seq#65,d_day_name#66,d_quarter_name#67,d_holiday#68,d_weekend#69,d_following_holiday#70,d_first_dom#71,d_last_dom#72,d_same_day_ly#73,d_same_day_lq#74,d_current_day#75,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
                     :     :     +- Relation[ws_sold_time_sk#114,ws_ship_date_sk#115,ws_item_sk#116,ws_bill_customer_sk#117,ws_bill_cdemo_sk#118,ws_bill_hdemo_sk#119,ws_bill_addr_sk#120,ws_ship_customer_sk#121,ws_ship_cdemo_sk#122,ws_ship_hdemo_sk#123,ws_ship_addr_sk#124,ws_web_page_sk#125,ws_web_site_sk#126,ws_ship_mode_sk#127,ws_warehouse_sk#128,ws_promo_sk#129,ws_order_number#130L,ws_quantity#131,ws_wholesale_cost#132,ws_list_price#133,ws_sales_price#134,ws_ext_discount_amt#135,ws_ext_sales_price#136,ws_ext_wholesale_cost#137,... 10 more fields] parquet, Statistics(sizeInBytes=216.4 GiB)
                     :     +- Project [i_item_sk#7, i_brand_id#14, i_class_id#16, i_category_id#18], Statistics(sizeInBytes=8.5 MiB, rowCount=3.72E+5)
                     :        +- Filter isnotnull(i_item_sk#7), Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
                     :           +- Relation[i_item_sk#7,i_item_id#8,i_rec_start_date#9,i_rec_end_date#10,i_item_desc#11,i_current_price#12,i_wholesale_cost#13,i_brand_id#14,i_brand#15,i_class_id#16,i_class#17,i_category_id#18,i_category#19,i_manufact_id#20,i_manufact#21,i_size#22,i_formulation#23,i_color#24,i_units#25,i_container#26,i_manager_id#27,i_product_name#28] parquet, Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
                     +- Project [d_date_sk#52], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
                        +- Filter ((((d_year#58 >= 1999) AND (d_year#58 <= 2001)) AND isnotnull(d_year#58)) AND isnotnull(d_date_sk#52)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
                           +- Relation[d_date_sk#52,d_date_id#53,d_date#54,d_month_seq#55,d_week_seq#56,d_quarter_seq#57,d_year#58,d_dow#59,d_moy#60,d_dom#61,d_qoy#62,d_fy_year#63,d_fy_quarter_seq#64,d_fy_week_seq#65,d_day_name#66,d_quarter_name#67,d_holiday#68,d_weekend#69,d_following_holiday#70,d_first_dom#71,d_last_dom#72,d_same_day_ly#73,d_same_day_lq#74,d_current_day#75,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
```

After this pr:
```
== Optimized Logical Plan ==
Project [i_item_sk#9 AS ss_item_sk#3], Statistics(sizeInBytes=8.07E+27 B)
+- Join Inner, (((i_brand_id#16 = brand_id#0) AND (i_class_id#18 = class_id#1)) AND (i_category_id#20 = category_id#2)), Statistics(sizeInBytes=2.42E+28 B)
   :- Project [i_item_sk#9, i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=8.5 MiB, rowCount=3.69E+5)
   :  +- Filter ((isnotnull(i_brand_id#16) AND isnotnull(i_class_id#18)) AND isnotnull(i_category_id#20)), Statistics(sizeInBytes=150.0 MiB, rowCount=3.69E+5)
   :     +- Relation tpcds5t.item[i_item_sk#9,i_item_id#10,i_rec_start_date#11,i_rec_end_date#12,i_item_desc#13,i_current_price#14,i_wholesale_cost#15,i_brand_id#16,i_brand#17,i_class_id#18,i_class#19,i_category_id#20,i_category#21,i_manufact_id#22,i_manufact#23,i_size#24,i_formulation#25,i_color#26,i_units#27,i_container#28,i_manager_id#29,i_product_name#30] parquet, Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
   +- Aggregate [brand_id#0, class_id#1, category_id#2], [brand_id#0, class_id#1, category_id#2], Statistics(sizeInBytes=2.73E+21 B)
      +- Aggregate [brand_id#0, class_id#1, category_id#2], [brand_id#0, class_id#1, category_id#2], Statistics(sizeInBytes=2.73E+21 B)
         +- Join LeftSemi, (((brand_id#0 <=> i_brand_id#16) AND (class_id#1 <=> i_class_id#18)) AND (category_id#2 <=> i_category_id#20)), Statistics(sizeInBytes=2.73E+21 B)
            :- Join LeftSemi, (((brand_id#0 <=> i_brand_id#16) AND (class_id#1 <=> i_class_id#18)) AND (category_id#2 <=> i_category_id#20)), Statistics(sizeInBytes=2.73E+21 B)
            :  :- Project [i_brand_id#16 AS brand_id#0, i_class_id#18 AS class_id#1, i_category_id#20 AS category_id#2], Statistics(sizeInBytes=2.73E+21 B)
            :  :  +- Join Inner, (ss_sold_date_sk#53 = d_date_sk#54), Statistics(sizeInBytes=3.83E+21 B)
            :  :     :- Project [ss_sold_date_sk#53, i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=387.3 PiB)
            :  :     :  +- Join Inner, (ss_item_sk#32 = i_item_sk#9), Statistics(sizeInBytes=516.5 PiB)
            :  :     :     :- Project [ss_item_sk#32, ss_sold_date_sk#53], Statistics(sizeInBytes=61.1 GiB)
            :  :     :     :  +- Filter ((isnotnull(ss_item_sk#32) AND isnotnull(ss_sold_date_sk#53)) AND dynamicpruning#150 [ss_sold_date_sk#53]), Statistics(sizeInBytes=580.6 GiB)
            :  :     :     :     :  +- Project [d_date_sk#54], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
            :  :     :     :     :     +- Filter ((((d_year#60 >= 1999) AND (d_year#60 <= 2001)) AND isnotnull(d_year#60)) AND isnotnull(d_date_sk#54)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
            :  :     :     :     :        +- Relation tpcds5t.date_dim[d_date_sk#54,d_date_id#55,d_date#56,d_month_seq#57,d_week_seq#58,d_quarter_seq#59,d_year#60,d_dow#61,d_moy#62,d_dom#63,d_qoy#64,d_fy_year#65,d_fy_quarter_seq#66,d_fy_week_seq#67,d_day_name#68,d_quarter_name#69,d_holiday#70,d_weekend#71,d_following_holiday#72,d_first_dom#73,d_last_dom#74,d_same_day_ly#75,d_same_day_lq#76,d_current_day#77,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
            :  :     :     :     +- Relation tpcds5t.store_sales[ss_sold_time_sk#31,ss_item_sk#32,ss_customer_sk#33,ss_cdemo_sk#34,ss_hdemo_sk#35,ss_addr_sk#36,ss_store_sk#37,ss_promo_sk#38,ss_ticket_number#39L,ss_quantity#40,ss_wholesale_cost#41,ss_list_price#42,ss_sales_price#43,ss_ext_discount_amt#44,ss_ext_sales_price#45,ss_ext_wholesale_cost#46,ss_ext_list_price#47,ss_ext_tax#48,ss_coupon_amt#49,ss_net_paid#50,ss_net_paid_inc_tax#51,ss_net_profit#52,ss_sold_date_sk#53] parquet, Statistics(sizeInBytes=580.6 GiB)
            :  :     :     +- Project [i_item_sk#9, i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=8.5 MiB, rowCount=3.69E+5)
            :  :     :        +- Filter (((isnotnull(i_brand_id#16) AND isnotnull(i_class_id#18)) AND isnotnull(i_category_id#20)) AND isnotnull(i_item_sk#9)), Statistics(sizeInBytes=150.0 MiB, rowCount=3.69E+5)
            :  :     :           +- Relation tpcds5t.item[i_item_sk#9,i_item_id#10,i_rec_start_date#11,i_rec_end_date#12,i_item_desc#13,i_current_price#14,i_wholesale_cost#15,i_brand_id#16,i_brand#17,i_class_id#18,i_class#19,i_category_id#20,i_category#21,i_manufact_id#22,i_manufact#23,i_size#24,i_formulation#25,i_color#26,i_units#27,i_container#28,i_manager_id#29,i_product_name#30] parquet, Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
            :  :     +- Project [d_date_sk#54], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
            :  :        +- Filter ((((d_year#60 >= 1999) AND (d_year#60 <= 2001)) AND isnotnull(d_year#60)) AND isnotnull(d_date_sk#54)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
            :  :           +- Relation tpcds5t.date_dim[d_date_sk#54,d_date_id#55,d_date#56,d_month_seq#57,d_week_seq#58,d_quarter_seq#59,d_year#60,d_dow#61,d_moy#62,d_dom#63,d_qoy#64,d_fy_year#65,d_fy_quarter_seq#66,d_fy_week_seq#67,d_day_name#68,d_quarter_name#69,d_holiday#70,d_weekend#71,d_following_holiday#72,d_first_dom#73,d_last_dom#74,d_same_day_ly#75,d_same_day_lq#76,d_current_day#77,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
            :  +- Aggregate [i_brand_id#16, i_class_id#18, i_category_id#20], [i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=1414.2 EiB)
            :     +- Project [i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=1414.2 EiB)
            :        +- Join Inner, (cs_sold_date_sk#115 = d_date_sk#54), Statistics(sizeInBytes=1979.9 EiB)
            :           :- Project [cs_sold_date_sk#115, i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=231.1 PiB)
            :           :  +- Join Inner, (cs_item_sk#96 = i_item_sk#9), Statistics(sizeInBytes=308.2 PiB)
            :           :     :- Project [cs_item_sk#96, cs_sold_date_sk#115], Statistics(sizeInBytes=36.2 GiB)
            :           :     :  +- Filter ((isnotnull(cs_item_sk#96) AND isnotnull(cs_sold_date_sk#115)) AND dynamicpruning#151 [cs_sold_date_sk#115]), Statistics(sizeInBytes=470.5 GiB)
            :           :     :     :  +- Project [d_date_sk#54], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
            :           :     :     :     +- Filter ((((d_year#60 >= 1999) AND (d_year#60 <= 2001)) AND isnotnull(d_year#60)) AND isnotnull(d_date_sk#54)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
            :           :     :     :        +- Relation tpcds5t.date_dim[d_date_sk#54,d_date_id#55,d_date#56,d_month_seq#57,d_week_seq#58,d_quarter_seq#59,d_year#60,d_dow#61,d_moy#62,d_dom#63,d_qoy#64,d_fy_year#65,d_fy_quarter_seq#66,d_fy_week_seq#67,d_day_name#68,d_quarter_name#69,d_holiday#70,d_weekend#71,d_following_holiday#72,d_first_dom#73,d_last_dom#74,d_same_day_ly#75,d_same_day_lq#76,d_current_day#77,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
            :           :     :     +- Relation tpcds5t.catalog_sales[cs_sold_time_sk#82,cs_ship_date_sk#83,cs_bill_customer_sk#84,cs_bill_cdemo_sk#85,cs_bill_hdemo_sk#86,cs_bill_addr_sk#87,cs_ship_customer_sk#88,cs_ship_cdemo_sk#89,cs_ship_hdemo_sk#90,cs_ship_addr_sk#91,cs_call_center_sk#92,cs_catalog_page_sk#93,cs_ship_mode_sk#94,cs_warehouse_sk#95,cs_item_sk#96,cs_promo_sk#97,cs_order_number#98L,cs_quantity#99,cs_wholesale_cost#100,cs_list_price#101,cs_sales_price#102,cs_ext_discount_amt#103,cs_ext_sales_price#104,cs_ext_wholesale_cost#105,... 10 more fields] parquet, Statistics(sizeInBytes=470.5 GiB)
            :           :     +- Project [i_item_sk#9, i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=8.5 MiB, rowCount=3.72E+5)
            :           :        +- Filter isnotnull(i_item_sk#9), Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
            :           :           +- Relation tpcds5t.item[i_item_sk#9,i_item_id#10,i_rec_start_date#11,i_rec_end_date#12,i_item_desc#13,i_current_price#14,i_wholesale_cost#15,i_brand_id#16,i_brand#17,i_class_id#18,i_class#19,i_category_id#20,i_category#21,i_manufact_id#22,i_manufact#23,i_size#24,i_formulation#25,i_color#26,i_units#27,i_container#28,i_manager_id#29,i_product_name#30] parquet, Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
            :           +- Project [d_date_sk#54], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
            :              +- Filter ((((d_year#60 >= 1999) AND (d_year#60 <= 2001)) AND isnotnull(d_year#60)) AND isnotnull(d_date_sk#54)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
            :                 +- Relation tpcds5t.date_dim[d_date_sk#54,d_date_id#55,d_date#56,d_month_seq#57,d_week_seq#58,d_quarter_seq#59,d_year#60,d_dow#61,d_moy#62,d_dom#63,d_qoy#64,d_fy_year#65,d_fy_quarter_seq#66,d_fy_week_seq#67,d_day_name#68,d_quarter_name#69,d_holiday#70,d_weekend#71,d_following_holiday#72,d_first_dom#73,d_last_dom#74,d_same_day_ly#75,d_same_day_lq#76,d_current_day#77,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
            +- Aggregate [i_brand_id#16, i_class_id#18, i_category_id#20], [i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=650.5 EiB)
               +- Project [i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=650.5 EiB)
                  +- Join Inner, (ws_sold_date_sk#149 = d_date_sk#54), Statistics(sizeInBytes=910.6 EiB)
                     :- Project [ws_sold_date_sk#149, i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=106.3 PiB)
                     :  +- Join Inner, (ws_item_sk#118 = i_item_sk#9), Statistics(sizeInBytes=141.7 PiB)
                     :     :- Project [ws_item_sk#118, ws_sold_date_sk#149], Statistics(sizeInBytes=16.6 GiB)
                     :     :  +- Filter ((isnotnull(ws_item_sk#118) AND isnotnull(ws_sold_date_sk#149)) AND dynamicpruning#152 [ws_sold_date_sk#149]), Statistics(sizeInBytes=216.4 GiB)
                     :     :     :  +- Project [d_date_sk#54], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
                     :     :     :     +- Filter ((((d_year#60 >= 1999) AND (d_year#60 <= 2001)) AND isnotnull(d_year#60)) AND isnotnull(d_date_sk#54)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
                     :     :     :        +- Relation tpcds5t.date_dim[d_date_sk#54,d_date_id#55,d_date#56,d_month_seq#57,d_week_seq#58,d_quarter_seq#59,d_year#60,d_dow#61,d_moy#62,d_dom#63,d_qoy#64,d_fy_year#65,d_fy_quarter_seq#66,d_fy_week_seq#67,d_day_name#68,d_quarter_name#69,d_holiday#70,d_weekend#71,d_following_holiday#72,d_first_dom#73,d_last_dom#74,d_same_day_ly#75,d_same_day_lq#76,d_current_day#77,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
                     :     :     +- Relation tpcds5t.web_sales[ws_sold_time_sk#116,ws_ship_date_sk#117,ws_item_sk#118,ws_bill_customer_sk#119,ws_bill_cdemo_sk#120,ws_bill_hdemo_sk#121,ws_bill_addr_sk#122,ws_ship_customer_sk#123,ws_ship_cdemo_sk#124,ws_ship_hdemo_sk#125,ws_ship_addr_sk#126,ws_web_page_sk#127,ws_web_site_sk#128,ws_ship_mode_sk#129,ws_warehouse_sk#130,ws_promo_sk#131,ws_order_number#132L,ws_quantity#133,ws_wholesale_cost#134,ws_list_price#135,ws_sales_price#136,ws_ext_discount_amt#137,ws_ext_sales_price#138,ws_ext_wholesale_cost#139,... 10 more fields] parquet, Statistics(sizeInBytes=216.4 GiB)
                     :     +- Project [i_item_sk#9, i_brand_id#16, i_class_id#18, i_category_id#20], Statistics(sizeInBytes=8.5 MiB, rowCount=3.72E+5)
                     :        +- Filter isnotnull(i_item_sk#9), Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
                     :           +- Relation tpcds5t.item[i_item_sk#9,i_item_id#10,i_rec_start_date#11,i_rec_end_date#12,i_item_desc#13,i_current_price#14,i_wholesale_cost#15,i_brand_id#16,i_brand#17,i_class_id#18,i_class#19,i_category_id#20,i_category#21,i_manufact_id#22,i_manufact#23,i_size#24,i_formulation#25,i_color#26,i_units#27,i_container#28,i_manager_id#29,i_product_name#30] parquet, Statistics(sizeInBytes=151.1 MiB, rowCount=3.72E+5)
                     +- Project [d_date_sk#54], Statistics(sizeInBytes=8.6 KiB, rowCount=731)
                        +- Filter ((((d_year#60 >= 1999) AND (d_year#60 <= 2001)) AND isnotnull(d_year#60)) AND isnotnull(d_date_sk#54)), Statistics(sizeInBytes=175.6 KiB, rowCount=731)
                           +- Relation tpcds5t.date_dim[d_date_sk#54,d_date_id#55,d_date#56,d_month_seq#57,d_week_seq#58,d_quarter_seq#59,d_year#60,d_dow#61,d_moy#62,d_dom#63,d_qoy#64,d_fy_year#65,d_fy_quarter_seq#66,d_fy_week_seq#67,d_day_name#68,d_quarter_name#69,d_holiday#70,d_weekend#71,d_following_holiday#72,d_first_dom#73,d_last_dom#74,d_same_day_ly#75,d_same_day_lq#76,d_current_day#77,... 4 more fields] parquet, Statistics(sizeInBytes=17.1 MiB, rowCount=7.30E+4)
```

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

No.

### How was this patch tested?

Unit test.

Closes #31196 from wangyum/SPARK-34129.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-21 12:04:49 -06:00
Max Gekk 04c3125dcf [SPARK-34360][SQL] Support truncation of v2 tables
### What changes were proposed in this pull request?
1. Add new interface `TruncatableTable` which represents tables that allow atomic truncation.
2. Implement new method in `InMemoryTable` and in `InMemoryPartitionTable`.

### Why are the changes needed?
To support `TRUNCATE TABLE` for v2 tables.

### Does this PR introduce _any_ user-facing change?
Should not.

### How was this patch tested?
Added new tests to `TableCatalogSuite` that check truncation of non-partitioned and partitioned tables:
```
$ build/sbt "test:testOnly *TableCatalogSuite"
```

Closes #31475 from MaxGekk/dsv2-truncate-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-21 17:50:38 +09:00
Kent Yao 1fac706db5 [SPARK-34373][SQL] HiveThriftServer2 startWithContext may hang with a race issue
### What changes were proposed in this pull request?

fix a race issue by interrupting the thread

### Why are the changes needed?

```
21:43:26.809 WARN org.apache.thrift.server.TThreadPoolServer: Transport error occurred during acceptance of message.
org.apache.thrift.transport.TTransportException: No underlying server socket.
at org.apache.thrift.transport.TServerSocket.acceptImpl(TServerSocket.java:126)
at org.apache.thrift.transport.TServerSocket.acceptImpl(TServerSocket.java:35)
at org.apache.thrift.transport.TServerTransport.acceException in thread "Thread-15" java.io.IOException: Stream closed
at java.io.BufferedInputStream.getBufIfOpen(BufferedInputStream.java:170)
at java.io.BufferedInputStream.read(BufferedInputStream.java:336)
at java.io.FilterInputStream.read(FilterInputStream.java:107)
at scala.sys.process.BasicIO$.loop$1(BasicIO.scala:238)
at scala.sys.process.BasicIO$.transferFullyImpl(BasicIO.scala:246)
at scala.sys.process.BasicIO$.transferFully(BasicIO.scala:227)
at scala.sys.process.BasicIO$.$anonfun$toStdOut$1(BasicIO.scala:221)
```
when the TServer try to `serve` after `stop`, it hangs with the log above forever
### Does this PR introduce _any_ user-facing change?

no
### How was this patch tested?

passing ci

Closes #31479 from yaooqinn/SPARK-34373.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-21 17:37:12 +09:00
Yuchen Huo 7de49a8fc0 [SPARK-34481][SQL] Refactor dataframe reader/writer optionsWithPath logic
### What changes were proposed in this pull request?

Extract optionsWithPath logic into their own function.

### Why are the changes needed?

Reduce the code duplication and improve modularity.

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

### How was this patch tested?
Just some refactoring. Existing tests.

Closes #31599 from yuchenhuo/SPARK-34481.

Authored-by: Yuchen Huo <yuchen.huo@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-20 17:57:43 -08:00
Kousuke Saruta 82b33a3041 [SPARK-34379][SQL] Map JDBC RowID to StringType rather than LongType
### What changes were proposed in this pull request?

This PR fix an issue that `java.sql.RowId` is mapped to `LongType` and prefer `StringType`.

In the current implementation, JDBC RowID type is mapped to `LongType` except for `OracleDialect`, but there is no guarantee to be able to convert RowID to long.
`java.sql.RowId` declares `toString` and the specification of `java.sql.RowId` says

> _all methods on the RowId interface must be fully implemented if the JDBC driver supports the data type_
(https://docs.oracle.com/javase/8/docs/api/java/sql/RowId.html)

So, we should prefer StringType to LongType.

### Why are the changes needed?

This seems to be a potential bug.

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

Yes. RowID is mapped to StringType rather than LongType.

### How was this patch tested?

New test and  the existing test case `SPARK-32992: map Oracle's ROWID type to StringType` in `OracleIntegrationSuite` passes.

Closes #31491 from sarutak/rowid-type.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-02-20 23:45:56 +09:00
Sean Owen f78466dca6 [SPARK-7768][CORE][SQL] Open UserDefinedType as a Developer API
### What changes were proposed in this pull request?

UserDefinedType and UDTRegistration become public Developer APIs, not package-private to Spark.

### Why are the changes needed?

This proposes to simply open up the UserDefinedType class as a developer API. It was public in 1.x, but closed in 2.x for some possible redesign that does not seem to have happened.

Other libraries have managed to define UDTs anyway by inserting shims into the Spark namespace, and this evidently has worked OK. But package isolation in Java 9+ breaks this.

The logic here is mostly: this is de facto a stable API, so can at least be open to developers with the usual caveats about developer APIs.

Open questions:

- Is there in fact some important redesign that's needed before opening it? The comment to this effect is from 2016
- Is this all that needs to be opened up? Like PythonUserDefinedType?
- Should any of this be kept package-private?

This was first proposed in https://github.com/apache/spark/pull/16478 though it was a larger change, but, the other API issues it was fixing seem to have been addressed already (e.g. no need to return internal Spark types). It was never really reviewed.

My hunch is that there isn't much downside, and some upside, to just opening this as-is now.

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

UserDefinedType becomes visible to developers to subclass.

### How was this patch tested?

Existing tests; there is no change to the existing logic.

Closes #31461 from srowen/SPARK-7768.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-20 07:32:06 -06:00
Zhichao Zhang 96bcb4bbe4 [SPARK-34283][SQL] Combines all adjacent 'Union' operators into a single 'Union' when using 'Dataset.union.distinct.union.distinct'
### What changes were proposed in this pull request?

Handled 'Deduplicate(Keys, Union)' operation in rule 'CombineUnions' to combine adjacent 'Union' operators  into a single 'Union' if necessary when using 'Dataset.union.distinct.union.distinct'.
Currently only handle distinct-like 'Deduplicate', where the keys == output, for example:
```
val df1 = Seq((1, 2, 3)).toDF("a", "b", "c")
val df2 = Seq((6, 2, 5)).toDF("a", "b", "c")
val df3 = Seq((2, 4, 3)).toDF("c", "a", "b")
val df4 = Seq((1, 4, 5)).toDF("b", "a", "c")
val unionDF1 = df1.unionByName(df2).dropDuplicates(Seq("b", "a", "c"))
      .unionByName(df3).dropDuplicates().unionByName(df4)
      .dropDuplicates("a")
```
In this case, **all Union operators will be combined**.
but,
```
val df1 = Seq((1, 2, 3)).toDF("a", "b", "c")
val df2 = Seq((6, 2, 5)).toDF("a", "b", "c")
val df3 = Seq((2, 4, 3)).toDF("c", "a", "b")
val df4 = Seq((1, 4, 5)).toDF("b", "a", "c")
val unionDF = df1.unionByName(df2).dropDuplicates(Seq("a"))
      .unionByName(df3).dropDuplicates("c").unionByName(df4)
      .dropDuplicates("b")
```
In this case, **all unions will not be combined, because the Deduplicate.keys doesn't equal to Union.output**.

### Why are the changes needed?

When using 'Dataset.union.distinct.union.distinct', the operator is  'Deduplicate(Keys, Union)', but AstBuilder transform sql-style 'Union' to operator 'Distinct(Union)', the rule 'CombineUnions' in Optimizer only handle 'Distinct(Union)' operator but not Deduplicate(Keys, Union).
Please see the detailed  description in [SPARK-34283](https://issues.apache.org/jira/browse/SPARK-34283).

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

No

### How was this patch tested?

Unit tests.

Closes #31404 from zzcclp/SPARK-34283.

Authored-by: Zhichao Zhang <441586683@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-19 15:19:13 +00:00
gengjiaan 06df1210d4 [SPARK-28123][SQL] String Functions: support btrim
### What changes were proposed in this pull request?
Spark support `trim`/`ltrim`/`rtrim` now. The function `btrim` is an alternate form of `TRIM(BOTH <chars> FROM <expr>)`.
`btrim` removes the longest string consisting only of specified characters from the start and end of a string.

The mainstream database support this feature show below:

**Postgresql**
https://www.postgresql.org/docs/11/functions-binarystring.html

**Vertica**
https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/Functions/String/BTRIM.htm?tocpath=SQL%20Reference%20Manual%7CSQL%20Functions%7CString%20Functions%7C_____5

**Redshift**
https://docs.aws.amazon.com/redshift/latest/dg/r_BTRIM.html

**Druid**
https://druid.apache.org/docs/latest/querying/sql.html#string-functions

**Greenplum**
http://docs.greenplum.org/6-8/ref_guide/function-summary.html

### Why are the changes needed?
btrim is very useful.

### Does this PR introduce _any_ user-facing change?
Yes. btrim is a new function

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

Closes #31390 from beliefer/SPARK-28123-support-btrim.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-19 13:28:49 +00:00
Peter Toth 27abb6ab56 [SPARK-34421][SQL] Resolve temporary functions and views in views with CTEs
### What changes were proposed in this pull request?
This PR:
- Fixes a bug that prevents analysis of:
  ```
  CREATE TEMPORARY VIEW temp_view AS WITH cte AS (SELECT temp_func(0)) SELECT * FROM cte;
  SELECT * FROM temp_view
  ```
  by throwing:
  ```
  Undefined function: 'temp_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.
  ```
- and doesn't report analysis error when it should:
  ```
  CREATE TEMPORARY VIEW temp_view AS SELECT 0;
  CREATE VIEW view_on_temp_view AS WITH cte AS (SELECT * FROM temp_view) SELECT * FROM cte
  ```
  by properly collecting temporary objects from VIEW definitions with CTEs.

- Minor refactor to make the affected code more readable.

### Why are the changes needed?
To fix a bug introduced with https://github.com/apache/spark/pull/30567

### Does this PR introduce _any_ user-facing change?
Yes, the query works again.

### How was this patch tested?
Added new UT + existing ones.

Closes #31550 from peter-toth/SPARK-34421-temp-functions-in-views-with-cte.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-19 18:14:49 +08:00
Max Gekk b26e7b510b [SPARK-34314][SQL] Fix partitions schema inference
### What changes were proposed in this pull request?
Infer the partitions schema by:
1. interring the common type over all partition part values, and
2. casting those values to the common type

Before the changes:
1. Spark creates a literal with most appropriate type for concrete partition value i.e. `part0=-0` -> `Literal(0, IntegerType)`, `part0=abc` -> `Literal(UTF8String.fromString("abc"), StringType)`.
2. Finds the common type for all literals of a partition column. For the example above, it is `StringType`.
3. Casts those literal to the desired type:
  - `Cast(Literal(0, IntegerType), StringType)` -> `UTF8String.fromString("0")`
  - `Cast(Literal(UTF8String.fromString("abc", StringType), StringType)` -> `UTF8String.fromString("abc")`

In the example, we get a partition part value "0" which is different from the original one "-0". Spark shouldn't modify partition part values of the string type because it can influence on query results.

Closes #31423

### Why are the changes needed?
The changes fix the bug demonstrated by the example:
1. There are partitioned parquet files (file format doesn't matter):
```
/private/var/folders/p3/dfs6mf655d7fnjrsjvldh0tc0000gn/T/spark-e09eae99-7ecf-4ab2-b99b-f63f8dea658d
├── _SUCCESS
├── part=-0
│   └── part-00001-02144398-2896-4d21-9628-a8743d098cb4.c000.snappy.parquet
└── part=AA
    └── part-00000-02144398-2896-4d21-9628-a8743d098cb4.c000.snappy.parquet
```
placed to two partitions "AA" and **"-0"**.

2. When reading them w/o specified schema:
```
val df = spark.read.parquet(path)
df.printSchema()
root
 |-- id: integer (nullable = true)
 |-- part: string (nullable = true)
```
the inferred type of the partition column `part` is the **string** type.
3. The expected values in the column `part` are "AA" and "-0" but we get:
```
df.show(false)
+---+----+
|id |part|
+---+----+
|0  |AA  |
|1  |0   |
+---+----+
```
So, Spark returns **"0"** instead of **"-0"**.

### Does this PR introduce _any_ user-facing change?
This PR can change query results.

### How was this patch tested?
By running new test and existing test suites:
```
$ build/sbt "test:testOnly *FileIndexSuite"
$ build/sbt "test:testOnly *ParquetV1PartitionDiscoverySuite"
$ build/sbt "test:testOnly *ParquetV2PartitionDiscoverySuite"
```

Closes #31549 from MaxGekk/fix-partition-file-index-2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-19 08:36:13 +00:00
yzjg 26548edfa2 [MINOR][SQL][DOCS] Fix the comments in the example at window function
### What changes were proposed in this pull request?

`functions.scala` window function has an comment error in the field name. The column should be `time` per `timestamp:TimestampType`.

### Why are the changes needed?

To deliver the correct documentation and examples.

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

Yes, it fixes the user-facing docs.

### How was this patch tested?

CI builds in this PR should test the documentation build.

Closes #31582 from yzjg/yzjg-patch-1.

Authored-by: yzjg <785246661@qq.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-19 10:45:21 +09:00
Max Gekk cad469d47a [SPARK-34465][SQL] Rename v2 alter table exec nodes
### What changes were proposed in this pull request?
Rename the following v2 exec nodes:
- AlterTableAddPartitionExec -> AddPartitionExec
- AlterTableRenamePartitionExec -> RenamePartitionExec
- AlterTableDropPartitionExec -> DropPartitionExec

### Why are the changes needed?
- To be consistent with v2 exec node added before: ALTER TABLE .. RENAME TO` -> RenameTableExec.
- For simplicity and readability of the execution plans.

### Does this PR introduce _any_ user-facing change?
Should not since this is internal API.

### How was this patch tested?
By running the existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite"
```

Closes #31584 from MaxGekk/rename-alter-table-exec-nodes.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-18 14:33:26 -08:00
Max Gekk 8f7ec4b28e [SPARK-34454][SQL] Mark legacy SQL configs as internal
### What changes were proposed in this pull request?
1. Make the following SQL configs as internal:
    - spark.sql.legacy.allowHashOnMapType
    - spark.sql.legacy.sessionInitWithConfigDefaults
2. Add a test to check that all SQL configs from the `legacy` namespace are marked as internal configs.

### Why are the changes needed?
Assuming that legacy SQL configs shouldn't be set by users in common cases. The purpose of such configs is to allow switching to old behavior in corner cases. So, the configs should be marked as internals.

### Does this PR introduce _any_ user-facing change?
Should not.

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *SQLConfSuite"
```

Closes #31577 from MaxGekk/mark-legacy-configs-as-internal.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-18 10:39:51 -08:00
Chao Sun 27873280ff [SPARK-32703][SQL] Replace deprecated API calls from SpecificParquetRecordReaderBase
### What changes were proposed in this pull request?

Currently in `SpecificParquetRecordReaderBase` we use deprecated APIs in a few places from Parquet, such as `readFooter`, `ParquetInputSplit`, `new ParquetFileReader`, `filterRowGroups`, etc. This replaces these with the newer APIs. In specific this:
- Replaces `ParquetInputSplit` with `FileSplit`. We never use specific things in the former such as `rowGroupOffsets` so the swap is pretty simple.
- Removes `readFooter` calls by using `ParquetFileReader.open`
- Replace deprecated `ParquetFileReader` ctor with the newer API which takes `ParquetReadOptions`.
- Removes the unnecessary handling of case when `rowGroupOffsets` is not null. It seems this never happens.

### Why are the changes needed?

The aforementioned APIs were deprecated and is going to be removed at some point in future. This is to ensure better supportability.

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

No

### How was this patch tested?

This is a cleanup and relies on existing tests on the relevant code paths.

Closes #29542 from sunchao/SPARK-32703.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-18 10:18:14 -06:00
Steve Loughran ff5115c3ac [SPARK-33739][SQL] Jobs committed through the S3A Magic committer don't track bytes
BasicWriteStatsTracker to probe for a custom Xattr if the size of
the generated file is 0 bytes; if found and parseable use that as
the declared length of the output.

The matching Hadoop patch in HADOOP-17414:

* Returns all S3 object headers as XAttr attributes prefixed "header."
* Sets the custom header x-hadoop-s3a-magic-data-length to the length of
  the data in the marker file.

As a result, spark job tracking will correctly report the amount of data uploaded
and yet to materialize.

### Why are the changes needed?

Now that S3 is consistent, it's a lot easier to use the S3A "magic" committer
which redirects a file written to `dest/__magic/job_0011/task_1245/__base/year=2020/output.avro`
to its final destination `dest/year=2020/output.avro` , adding a zero byte marker file at
the end and a json file `dest/__magic/job_0011/task_1245/__base/year=2020/output.avro.pending`
containing all the information for the job committer to complete the upload.

But: the write tracker statictics don't show progress as they measure the length of the
created file, find the marker file and report 0 bytes.
By probing for a specific HTTP header in the marker file and parsing that if
retrieved, the real progress can be reported.

There's a matching change in Hadoop [https://github.com/apache/hadoop/pull/2530](https://github.com/apache/hadoop/pull/2530)
which adds getXAttr API support to the S3A connector and returns the headers; the magic
committer adds the relevant attributes.

If the FS being probed doesn't support the XAttr API, the header is missing
or the value not a positive long then the size of 0 is returned.

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

No

### How was this patch tested?

New tests in BasicWriteTaskStatsTrackerSuite which use a filter FS to
implement getXAttr on top of LocalFS; this is used to explore the set of
options:
* no XAttr API implementation (existing tests; what callers would see with
  most filesystems)
* no attribute found (HDFS, ABFS without the attribute)
* invalid data of different forms

All of these return Some(0) as file length.

The Hadoop PR verifies XAttr implementation in S3A and that
the commit protocol attaches the header to the files.

External downstream testing has done the full hadoop+spark end
to end operation, with manual review of logs to verify that the
data was successfully collected from the attribute.

Closes #30714 from steveloughran/cdpd/SPARK-33739-magic-commit-tracking-master.

Authored-by: Steve Loughran <stevel@cloudera.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-02-18 08:43:18 -06:00
gengjiaan edccf96cad [SPARK-34394][SQL] Unify output of SHOW FUNCTIONS and pass output attributes properly
### What changes were proposed in this pull request?
The current implement of some DDL not unify the output and not pass the output properly to physical command.
Such as: The output attributes of `ShowFunctions` does't pass to `ShowFunctionsCommand` properly.

As the query plan, this PR pass the output attributes from `ShowFunctions` to `ShowFunctionsCommand`.

### Why are the changes needed?
This PR pass the output attributes could keep the expr ID unchanged, so that avoid bugs when we apply more operators above the command output dataframe.

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

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

Closes #31519 from beliefer/SPARK-34394.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-18 12:50:50 +00:00
gengjiaan c925e4d0fd [SPARK-34393][SQL] Unify output of SHOW VIEWS and pass output attributes properly
### What changes were proposed in this pull request?
The current implement of some DDL not unify the output and not pass the output properly to physical command.
Such as: The output attributes of `ShowViews` does't pass to `ShowViewsCommand` properly.

As the query plan, this PR pass the output attributes from `ShowViews` to `ShowViewsCommand`.

### Why are the changes needed?
This PR pass the output attributes could keep the expr ID unchanged, so that avoid bugs when we apply more operators above the command output dataframe.

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

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

Closes #31508 from beliefer/SPARK-34393.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-18 12:48:39 +00:00
Max Gekk 7b549c3e53 [SPARK-34455][SQL] Deprecate spark.sql.legacy.replaceDatabricksSparkAvro.enabled
### What changes were proposed in this pull request?
1. Put the SQL config `spark.sql.legacy.replaceDatabricksSparkAvro.enabled` to the list of deprecated configs `deprecatedSQLConfigs`
2. Update docs for the Avro datasource
<img width="982" alt="Screenshot 2021-02-17 at 21 04 26" src="https://user-images.githubusercontent.com/1580697/108249890-abed7180-7166-11eb-8cb7-0c246d2a34fc.png">

### Why are the changes needed?
The config exists for enough time. We can deprecate it, and recommend users to use `.format("avro")` instead.

### Does this PR introduce _any_ user-facing change?
Should not except of the warning with the recommendation to use the `avro` format.

### How was this patch tested?
1. By generating docs via:
```
$ SKIP_API=1 SKIP_SCALADOC=1 SKIP_PYTHONDOC=1 SKIP_RDOC=1 jekyll serve --watch
```
2. Manually checking the warning:
```
scala> spark.conf.set("spark.sql.legacy.replaceDatabricksSparkAvro.enabled", false)
21/02/17 21:20:18 WARN SQLConf: The SQL config 'spark.sql.legacy.replaceDatabricksSparkAvro.enabled' has been deprecated in Spark v3.2 and may be removed in the future. Use `.format("avro")` in `DataFrameWriter` or `DataFrameReader` instead.
```

Closes #31578 from MaxGekk/deprecate-replaceDatabricksSparkAvro.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-17 21:54:20 -08:00
Anton Okolnychyi 1ad343238c [SPARK-33736][SQL] Handle MERGE in ReplaceNullWithFalseInPredicate
### What changes were proposed in this pull request?

This PR handles merge operations in `ReplaceNullWithFalseInPredicate`.

### Why are the changes needed?

These changes are needed to match what we already do for delete and update operations.

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

No.

### How was this patch tested?

This PR extends existing tests to cover merge operations.

Closes #31579 from aokolnychyi/spark-33736.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-17 17:27:21 -08:00
Anton Okolnychyi 44a9aed0d7 [SPARK-34456][SQL] Remove unused write options from BatchWriteHelper
### What changes were proposed in this pull request?

This PR removes dead code from `BatchWriteHelper` after SPARK-33808.

### Why are the changes needed?

These changes simplify `BatchWriteHelper` by removing write options that are no longer needed as we build `Write` earlier.

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

No.

### How was this patch tested?

Existing tests.

Closes #31581 from aokolnychyi/simplify-batch-write-helper.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-17 17:25:26 -08:00
Max Gekk 5957bc18a1 [SPARK-34451][SQL] Add alternatives for datetime rebasing SQL configs and deprecate legacy configs
### What changes were proposed in this pull request?
Move the datetime rebase SQL configs from the `legacy` namespace by:
1. Renaming of the existing rebase configs like `spark.sql.legacy.parquet.datetimeRebaseModeInRead` -> `spark.sql.parquet.datetimeRebaseModeInRead`.
2. Add the legacy configs as alternatives
3. Deprecate the legacy rebase configs.

### Why are the changes needed?
The rebasing SQL configs like `spark.sql.legacy.parquet.datetimeRebaseModeInRead` can be used not only for migration from previous Spark versions but also to read/write datatime columns saved by other systems/frameworks/libs. So, the configs shouldn't be considered as legacy configs.

### Does this PR introduce _any_ user-facing change?
Should not. Users will see a warning if they still use one of the legacy configs.

### How was this patch tested?
1. Manually checking new configs:
```scala
scala> spark.conf.get("spark.sql.parquet.datetimeRebaseModeInRead")
res0: String = EXCEPTION

scala> spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "LEGACY")
21/02/17 14:57:10 WARN SQLConf: The SQL config 'spark.sql.legacy.parquet.datetimeRebaseModeInRead' has been deprecated in Spark v3.2 and may be removed in the future. Use 'spark.sql.parquet.datetimeRebaseModeInRead' instead.

scala> spark.conf.get("spark.sql.parquet.datetimeRebaseModeInRead")
res2: String = LEGACY
```
2. By running a datetime rebasing test suite:
```
$ build/sbt "test:testOnly *ParquetRebaseDatetimeV1Suite"
```

Closes #31576 from MaxGekk/rebase-confs-alternatives.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-17 14:04:47 +00:00
Kousuke Saruta dd6383f0a3 [SPARK-34333][SQL] Fix PostgresDialect to handle money types properly
### What changes were proposed in this pull request?

This PR changes the type mapping for `money` and `money[]`  types for PostgreSQL.
Currently, those types are tried to convert to `DoubleType` and `ArrayType` of `double` respectively.
But the JDBC driver seems not to be able to handle those types properly.

https://github.com/pgjdbc/pgjdbc/issues/100
https://github.com/pgjdbc/pgjdbc/issues/1405

Due to these issue, we can get the error like as follows.

money type.
```
[info]   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0) (192.168.1.204 executor driver): org.postgresql.util.PSQLException: Bad value for type double : 1,000.00
[info] 	at org.postgresql.jdbc.PgResultSet.toDouble(PgResultSet.java:3104)
[info] 	at org.postgresql.jdbc.PgResultSet.getDouble(PgResultSet.java:2432)
[info] 	at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$makeGetter$5(JdbcUtils.scala:418)
```

money[] type.
```
[info]   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0) (192.168.1.204 executor driver): org.postgresql.util.PSQLException: Bad value for type double : $2,000.00
[info] 	at org.postgresql.jdbc.PgResultSet.toDouble(PgResultSet.java:3104)
[info] 	at org.postgresql.jdbc.ArrayDecoding$5.parseValue(ArrayDecoding.java:235)
[info] 	at org.postgresql.jdbc.ArrayDecoding$AbstractObjectStringArrayDecoder.populateFromString(ArrayDecoding.java:122)
[info] 	at org.postgresql.jdbc.ArrayDecoding.readStringArray(ArrayDecoding.java:764)
[info] 	at org.postgresql.jdbc.PgArray.buildArray(PgArray.java:310)
[info] 	at org.postgresql.jdbc.PgArray.getArrayImpl(PgArray.java:171)
[info] 	at org.postgresql.jdbc.PgArray.getArray(PgArray.java:111)
```

For money type, a known workaround is to treat it as string so this PR do it.
For money[], however, there is no reasonable workaround so this PR remove the support.

### Why are the changes needed?

This is a bug.

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

Yes. As of this PR merged, money type is mapped to `StringType` rather than `DoubleType` and the support for money[] is stopped.
For money type, if the value is less than one thousand,  `$100.00` for instance, it works without this change so I also updated the migration guide because it's a behavior change for such small values.
On the other hand, money[] seems not to work with any value but mentioned in the migration guide just in case.

### How was this patch tested?

New test.

Closes #31442 from sarutak/fix-for-money-type.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-02-17 10:50:06 +09:00
Max Gekk 1a11fe5501 [SPARK-33210][SQL][DOCS][FOLLOWUP] Fix descriptions of the SQL configs for the parquet INT96 rebase modes
### What changes were proposed in this pull request?
Fix descriptions of the SQL configs `spark.sql.legacy.parquet.int96RebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInWrite`, and mention `EXCEPTION` as the default value.

### Why are the changes needed?
This fixes incorrect descriptions that can mislead users.

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

### How was this patch tested?
By running `./dev/scalastyle`.

Closes #31557 from MaxGekk/int96-exception-by-default-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-16 11:55:53 +09:00
Max Gekk 03161055de [SPARK-34424][SQL][TESTS] Fix failures of HiveOrcHadoopFsRelationSuite
### What changes were proposed in this pull request?
Modify `RandomDataGenerator.forType()` to allow generation of dates/timestamps that are valid in both Julian and Proleptic Gregorian calendars. Currently, the function can produce a date (for example `1582-10-06`) which is valid in the Proleptic Gregorian calendar. Though it cannot be saved to ORC files AS IS since ORC format (ORC libs in fact) assumes Julian calendar. So, Spark shifts `1582-10-06` to the next valid date `1582-10-15` while saving it to ORC files. And as a consequence of that, the test fails because it compares original date `1582-10-06` and the date `1582-10-15` loaded back from the ORC files.

In this PR, I propose to generate valid dates/timestamps in both calendars for ORC datasource till SPARK-34440 is resolved.

### Why are the changes needed?
The changes fix failures of `HiveOrcHadoopFsRelationSuite`. For instance, the test "test all data types" fails with the seed **610710213676**:
```
== Results ==
!== Correct Answer - 20 ==    == Spark Answer - 20 ==
 struct<index:int,col:date>   struct<index:int,col:date>
...
![9,1582-10-06]               [9,1582-10-15]
```

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

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *HiveOrcHadoopFsRelationSuite"
```

Closes #31552 from MaxGekk/fix-HiveOrcHadoopFsRelationSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-16 11:53:26 +09:00
Max Gekk aca6db1868 [SPARK-34434][SQL] Mention DS rebase options in SparkUpgradeException
### What changes were proposed in this pull request?
Mention the DS options introduced by https://github.com/apache/spark/pull/31529 and by https://github.com/apache/spark/pull/31489 in `SparkUpgradeException`.

### Why are the changes needed?
To improve user experience with Spark SQL. Before the changes, the error message recommends to set SQL configs but the configs cannot help in the some situations (see the PRs for more details).

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the error message is:

_org.apache.spark.SparkUpgradeException: You may get a different result due to the upgrading of Spark 3.0: reading dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z from Parquet files can be ambiguous, as the files may be written by Spark 2.x or legacy versions of Hive, which uses a legacy hybrid calendar that is different from Spark 3.0+'s Proleptic Gregorian calendar. See more details in SPARK-31404. You can set the SQL config 'spark.sql.legacy.parquet.datetimeRebaseModeInRead' or the datasource option 'datetimeRebaseMode' to 'LEGACY' to rebase the datetime values w.r.t. the calendar difference during reading. To read the datetime values as it is, set the SQL config 'spark.sql.legacy.parquet.datetimeRebaseModeInRead' or the datasource option 'datetimeRebaseMode' to 'CORRECTED'._

### How was this patch tested?
1. By checking coding style: `./dev/scalastyle`
2. By running the related test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ParquetRebaseDatetimeV1Suite"
```

Closes #31562 from MaxGekk/rebase-upgrade-exception.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-14 17:42:15 -08:00
Terry Kim 9a566f83a0 [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command
### What changes were proposed in this pull request?

This PR proposes to support `ifExists` flag for v2 `ALTER TABLE ... UNSET TBLPROPERTIES` command. Currently, the flag is not respected and the command behaves as `ifExists = true` where the command always succeeds when the properties do not exist.

### Why are the changes needed?

To support `ifExists` flag and align with v1 command behavior.

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

Yes, now if the property does not exist and `IF EXISTS` is not specified, the command will fail:
```
ALTER TABLE t UNSET TBLPROPERTIES ('unknown') // Fails with "Attempted to unset non-existent property 'unknown'"
ALTER TABLE t UNSET TBLPROPERTIES IF EXISTS ('unknown') // OK
```

### How was this patch tested?

Added new test

Closes #31494 from imback82/AlterTableUnsetPropertiesIfExists.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-12 17:42:43 -08:00
Max Gekk 91be583fb8 [SPARK-34418][SQL][TESTS] Check partitions existence after v1 TRUNCATE TABLE
### What changes were proposed in this pull request?
Add a test and modify an existing one to check that partitions still exist after v1 `TRUNCATE TABLE`.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running new test:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *TruncateTableSuite"
```

Closes #31544 from MaxGekk/test-truncate-partitioned-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-12 15:57:42 -08:00
Liang-Chi Hsieh e0053853c9 [SPARK-34420][SQL] Throw exception if non-streaming Deduplicate is not replaced by aggregate
### What changes were proposed in this pull request?

This patch proposes to throw exception if non-streaming `Deduplicate` is not replaced by aggregate in query planner.

### Why are the changes needed?

We replace some operations in the query optimizer. For them we throw some exceptions accordingly in query planner if these logical nodes are not replaced. But `Deduplicate` is missing and it opens a possible hole. For code consistency and to prevent possible unexpected query planning error, we should add similar exception case to query planner.

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

No

### How was this patch tested?

Unit test.

Closes #31547 from viirya/minor-deduplicate.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-02-10 22:40:51 -08:00
Chao Sun cd38287ce2 [SPARK-34419][SQL] Move PartitionTransforms.scala to scala directory
### What changes were proposed in this pull request?

Move `PartitionTransforms.scala` from `sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions` to `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions`.

### Why are the changes needed?

We should put java/scala files to their corresponding directories.

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

No.

### How was this patch tested?

N/A

Closes #31546 from sunchao/SPARK-34419.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-02-10 17:08:50 -08:00
David Li 9b875ceada [SPARK-32953][PYTHON][SQL] Add Arrow self_destruct support to toPandas
### What changes were proposed in this pull request?

Creating a Pandas dataframe via Apache Arrow currently can use twice as much memory as the final result, because during the conversion, both Pandas and Arrow retain a copy of the data. Arrow has a "self-destruct" mode now (Arrow >= 0.16) to avoid this, by freeing each column after conversion. This PR integrates support for this in toPandas, handling a couple of edge cases:

self_destruct has no effect unless the memory is allocated appropriately, which is handled in the Arrow serializer here. Essentially, the issue is that self_destruct frees memory column-wise, but Arrow record batches are oriented row-wise:

```
Record batch 0: allocation 0: column 0 chunk 0, column 1 chunk 0, ...
Record batch 1: allocation 1: column 0 chunk 1, column 1 chunk 1, ...
```

In this scenario, Arrow will drop references to all of column 0's chunks, but no memory will actually be freed, as the chunks were just slices of an underlying allocation. The PR copies each column into its own allocation so that memory is instead arranged as so:

```
Record batch 0: allocation 0 column 0 chunk 0, allocation 1 column 1 chunk 0, ...
Record batch 1: allocation 2 column 0 chunk 1, allocation 3 column 1 chunk 1, ...
```

The optimization is disabled by default, and can be enabled with the Spark SQL conf "spark.sql.execution.arrow.pyspark.selfDestruct.enabled" set to "true". We can't always apply this optimization because it's more likely to generate a dataframe with immutable buffers, which Pandas doesn't always handle well, and because it is slower overall (since it only converts one column at a time instead of in parallel).

### Why are the changes needed?

This lets us load larger datasets - in particular, with N bytes of memory, before we could never load a dataset bigger than N/2 bytes; now the overhead is more like N/1.25 or so.

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

Yes - it adds a new SQL conf "spark.sql.execution.arrow.pyspark.selfDestruct.enabled"

### How was this patch tested?

See the [mailing list](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Reducing-memory-usage-of-toPandas-with-Arrow-quot-self-destruct-quot-option-td30149.html) - it was tested with Python memory_profiler. Unit tests added to check memory within certain bounds and correctness with the option enabled.

Closes #29818 from lidavidm/spark-32953.

Authored-by: David Li <li.davidm96@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2021-02-10 09:58:46 -08:00
gengjiaan 32a523b56f [SPARK-34234][SQL] Remove TreeNodeException that didn't work
### What changes were proposed in this pull request?
`TreeNodeException` causes the error msg not clear and it didn't work well.
Because the `TreeNodeException` looks redundancy, we could remove it.

There are show a case:
```
val df = Seq(("1", 1), ("1", 2), ("2", 3), ("2", 4)).toDF("x", "y")
val hashAggDF = df.groupBy("x").agg(c, sum("y"))
```
The above code will use `HashAggregateExec`. In order to ensure that an exception will be thrown when executing `HashAggregateExec`, I added `throw new RuntimeException("calculate error")` into 72b7f8abfb/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala (L85)

So, if the above code is executed, `RuntimeException("calculate error")` will be thrown.
Before this PR, the error is:
```
execute, tree:
ShuffleQueryStage 0
+- Exchange hashpartitioning(x#105, 5), ENSURE_REQUIREMENTS, [id=#168]
   +- HashAggregate(keys=[x#105], functions=[partial_sum(y#106)], output=[x#105, sum#118L])
      +- Project [_1#100 AS x#105, _2#101 AS y#106]
         +- LocalTableScan [_1#100, _2#101]

org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
ShuffleQueryStage 0
+- Exchange hashpartitioning(x#105, 5), ENSURE_REQUIREMENTS, [id=#168]
   +- HashAggregate(keys=[x#105], functions=[partial_sum(y#106)], output=[x#105, sum#118L])
      +- Project [_1#100 AS x#105, _2#101 AS y#106]
         +- LocalTableScan [_1#100, _2#101]

	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
	at org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.doMaterialize(QueryStageExec.scala:163)
	at org.apache.spark.sql.execution.adaptive.QueryStageExec.$anonfun$materialize$1(QueryStageExec.scala:81)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:218)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:215)
	at org.apache.spark.sql.execution.adaptive.QueryStageExec.materialize(QueryStageExec.scala:79)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5(AdaptiveSparkPlanExec.scala:207)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5$adapted(AdaptiveSparkPlanExec.scala:205)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$1(AdaptiveSparkPlanExec.scala:205)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.getFinalPhysicalPlan(AdaptiveSparkPlanExec.scala:179)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.executeCollect(AdaptiveSparkPlanExec.scala:289)
	at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3708)
	at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:2977)
	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3699)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3697)
	at org.apache.spark.sql.Dataset.collect(Dataset.scala:2977)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$3(DataFrameAggregateSuite.scala:665)
	at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
	at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38)
	at org.apache.spark.sql.DataFrameAggregateSuite.org$apache$spark$sql$test$SQLTestUtilsBase$$super$withSQLConf(DataFrameAggregateSuite.scala:37)
	at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf(SQLTestUtils.scala:246)
	at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf$(SQLTestUtils.scala:244)
	at org.apache.spark.sql.DataFrameAggregateSuite.withSQLConf(DataFrameAggregateSuite.scala:37)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$2(DataFrameAggregateSuite.scala:659)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$2$adapted(DataFrameAggregateSuite.scala:655)
	at scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:877)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:876)
	at org.apache.spark.sql.DataFrameAggregateSuite.assertNoExceptions(DataFrameAggregateSuite.scala:655)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$126(DataFrameAggregateSuite.scala:695)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$126$adapted(DataFrameAggregateSuite.scala:695)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$125(DataFrameAggregateSuite.scala:695)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:190)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:176)
	at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:188)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:200)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:200)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:182)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:61)
	at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
	at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
	at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:61)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:233)
	at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:233)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:232)
	at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1563)
	at org.scalatest.Suite.run(Suite.scala:1112)
	at org.scalatest.Suite.run$(Suite.scala:1094)
	at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1563)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:237)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
	at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:237)
	at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:236)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:61)
	at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
	at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
	at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
	at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:61)
	at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1320)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1314)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1314)
	at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:993)
	at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:971)
	at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1480)
	at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:971)
	at org.scalatest.tools.Runner$.run(Runner.scala:798)
	at org.scalatest.tools.Runner.run(Runner.scala)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2(ScalaTestRunner.java:131)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:28)
Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
HashAggregate(keys=[x#105], functions=[partial_sum(y#106)], output=[x#105, sum#118L])
+- Project [_1#100 AS x#105, _2#101 AS y#106]
   +- LocalTableScan [_1#100, _2#101]

	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
	at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doExecute(HashAggregateExec.scala:84)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:180)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:218)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:215)
	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:176)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD$lzycompute(ShuffleExchangeExec.scala:118)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD(ShuffleExchangeExec.scala:118)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture$lzycompute(ShuffleExchangeExec.scala:122)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture(ShuffleExchangeExec.scala:121)
	at org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.$anonfun$doMaterialize$1(QueryStageExec.scala:163)
	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
	... 91 more
Caused by: java.lang.RuntimeException: calculate error
	at org.apache.spark.sql.execution.aggregate.HashAggregateExec.$anonfun$doExecute$1(HashAggregateExec.scala:85)
	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
	... 103 more
```

After this PR, the error is:
```
calculate error
java.lang.RuntimeException: calculate error
	at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doExecute(HashAggregateExec.scala:84)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:180)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:218)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:215)
	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:176)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD$lzycompute(ShuffleExchangeExec.scala:117)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD(ShuffleExchangeExec.scala:117)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture$lzycompute(ShuffleExchangeExec.scala:121)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture(ShuffleExchangeExec.scala:120)
	at org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.doMaterialize(QueryStageExec.scala:161)
	at org.apache.spark.sql.execution.adaptive.QueryStageExec.$anonfun$materialize$1(QueryStageExec.scala:80)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:218)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:215)
	at org.apache.spark.sql.execution.adaptive.QueryStageExec.materialize(QueryStageExec.scala:78)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5(AdaptiveSparkPlanExec.scala:207)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5$adapted(AdaptiveSparkPlanExec.scala:205)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$1(AdaptiveSparkPlanExec.scala:205)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.getFinalPhysicalPlan(AdaptiveSparkPlanExec.scala:179)
	at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.executeCollect(AdaptiveSparkPlanExec.scala:289)
	at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3708)
	at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:2977)
	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3699)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3697)
	at org.apache.spark.sql.Dataset.collect(Dataset.scala:2977)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$3(DataFrameAggregateSuite.scala:665)
	at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
	at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38)
	at org.apache.spark.sql.DataFrameAggregateSuite.org$apache$spark$sql$test$SQLTestUtilsBase$$super$withSQLConf(DataFrameAggregateSuite.scala:37)
	at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf(SQLTestUtils.scala:246)
	at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf$(SQLTestUtils.scala:244)
	at org.apache.spark.sql.DataFrameAggregateSuite.withSQLConf(DataFrameAggregateSuite.scala:37)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$2(DataFrameAggregateSuite.scala:659)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$2$adapted(DataFrameAggregateSuite.scala:655)
	at scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:877)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:876)
	at org.apache.spark.sql.DataFrameAggregateSuite.assertNoExceptions(DataFrameAggregateSuite.scala:655)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$126(DataFrameAggregateSuite.scala:695)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$126$adapted(DataFrameAggregateSuite.scala:695)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$125(DataFrameAggregateSuite.scala:695)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:190)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:176)
	at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:188)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:200)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:200)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:182)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:61)
	at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
	at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
	at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:61)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:233)
	at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:233)
	at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:232)
	at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1563)
	at org.scalatest.Suite.run(Suite.scala:1112)
	at org.scalatest.Suite.run$(Suite.scala:1094)
	at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1563)
	at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:237)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
	at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:237)
	at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:236)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:61)
	at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
	at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
	at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
	at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:61)
	at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1320)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1314)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1314)
	at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:993)
	at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:971)
	at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1480)
	at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:971)
	at org.scalatest.tools.Runner$.run(Runner.scala:798)
	at org.scalatest.tools.Runner.run(Runner.scala)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2(ScalaTestRunner.java:131)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:28)
```

### Why are the changes needed?
`TreeNodeException` didn't work well.

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

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

Closes #31337 from beliefer/SPARK-34234.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-10 06:25:33 +00:00
Chao Sun 0986f16c8d [SPARK-34347][SQL] CatalogImpl.uncacheTable should invalidate in cascade for temp views
### What changes were proposed in this pull request?

This PR includes the following changes:
1. in `CatalogImpl.uncacheTable`, invalidate caches in cascade when the target table is
 a temp view, and `spark.sql.legacy.storeAnalyzedPlanForView` is false (default value).
2. make `SessionCatalog.lookupTempView` public and return processed temp view plan (i.e., with `View` op).

### Why are the changes needed?

Following [SPARK-34052](https://issues.apache.org/jira/browse/SPARK-34052) (#31107), we should invalidate in cascade for `CatalogImpl.uncacheTable` when the table is a temp view, so that the behavior is consistent.

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

Yes, now `SQLContext.uncacheTable` will drop temp view in cascade by default.

### How was this patch tested?

Added a UT

Closes #31462 from sunchao/SPARK-34347.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-02-09 20:48:58 -08:00
Gabor Somogyi 0a37a95224 [SPARK-31816][SQL][DOCS] Added high level description about JDBC connection providers for users/developers
### What changes were proposed in this pull request?
JDBC connection provider API and embedded connection providers already added to the code but no in-depth description about the internals. In this PR I've added both user and developer documentation and additionally added an example custom JDBC connection provider.

### Why are the changes needed?
No documentation and example custom JDBC provider.

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

### How was this patch tested?
```
cd docs/
SKIP_API=1 jekyll build
```
<img width="793" alt="Screenshot 2021-02-02 at 16 35 43" src="https://user-images.githubusercontent.com/18561820/106623428-e48d2880-6574-11eb-8d14-e5c2aa7c37f1.png">

Closes #31384 from gaborgsomogyi/SPARK-31816.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-02-10 12:28:28 +09:00
MrPowers e6753c9402 [SPARK-33995][SQL] Expose make_interval as a Scala function
### What changes were proposed in this pull request?

This pull request exposes the `make_interval` function, [as suggested here](https://github.com/apache/spark/pull/31000#pullrequestreview-560812433), and as agreed to [here](https://github.com/apache/spark/pull/31000#issuecomment-754856820) and [here](https://github.com/apache/spark/pull/31000#issuecomment-755040234).

This powerful little function allows for idiomatic datetime arithmetic via the Scala API:

```scala
// add two hours
df.withColumn("plus_2_hours", col("first_datetime") + make_interval(hours = lit(2)))

// subtract one week and 30 seconds
col("d") - make_interval(weeks = lit(1), secs = lit(30))
```

The `make_interval` [SQL function](https://github.com/apache/spark/pull/26446) already exists.

Here is [the JIRA ticket](https://issues.apache.org/jira/browse/SPARK-33995) for this PR.

### Why are the changes needed?

The Spark API makes it easy to perform datetime addition / subtraction with months (`add_months`) and days (`date_add`).  Users need to write code like this to perform datetime addition with years, weeks, hours, minutes, or seconds:

```scala
df.withColumn("plus_2_hours", expr("first_datetime + INTERVAL 2 hours"))
```

We don't want to force users to manipulate SQL strings when they're using the Scala API.

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

Yes, this PR adds `make_interval` to the `org.apache.spark.sql.functions` API.

This single function will benefit a lot of users.  It's a small increase in the surface of the API for a big gain.

### How was this patch tested?

This was tested via unit tests.

cc: MaxGekk

Closes #31073 from MrPowers/SPARK-33995.

Authored-by: MrPowers <matthewkevinpowers@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-10 03:27:41 +00:00
Angerszhuuuu 2f387b41e8 [SPARK-34137][SQL] Update suquery's stats when build LogicalPlan's stats
### What changes were proposed in this pull request?
When explain SQL with cost, treeString about subquery won't show it's statistics:

How to reproduce:
```
spark.sql("create table t1 using parquet as select id as a, id as b from range(1000)")
spark.sql("create table t2 using parquet as select id as c, id as d from range(2000)")

spark.sql("ANALYZE TABLE t1 COMPUTE STATISTICS FOR ALL COLUMNS")
spark.sql("ANALYZE TABLE t2 COMPUTE STATISTICS FOR ALL COLUMNS")
spark.sql("set spark.sql.cbo.enabled=true")

spark.sql(
  """
    |WITH max_store_sales AS
    |  (SELECT max(csales) tpcds_cmax
    |  FROM (SELECT
    |    sum(b) csales
    |  FROM t1 WHERE a < 100 ) x),
    |best_ss_customer AS
    |  (SELECT
    |    c
    |  FROM t2
    |  WHERE d > (SELECT * FROM max_store_sales))
    |
    |SELECT c FROM best_ss_customer
    |""".stripMargin).explain("cost")
```
Before this PR's output:
```
== Optimized Logical Plan ==
Project [c#4263L], Statistics(sizeInBytes=31.3 KiB, rowCount=2.00E+3)
+- Filter (isnotnull(d#4264L) AND (d#4264L > scalar-subquery#4262 [])), Statistics(sizeInBytes=46.9 KiB, rowCount=2.00E+3)
   :  +- Aggregate [max(csales#4260L) AS tpcds_cmax#4261L]
   :     +- Aggregate [sum(b#4266L) AS csales#4260L]
   :        +- Project [b#4266L]
   :           +- Filter ((a#4265L < 100) AND isnotnull(a#4265L))
   :              +- Relation default.t1[a#4265L,b#4266L] parquet, Statistics(sizeInBytes=23.4 KiB, rowCount=1.00E+3)
   +- Relation default.t2[c#4263L,d#4264L] parquet, Statistics(sizeInBytes=46.9 KiB, rowCount=2.00E+3)
```

After this pr:
```
== Optimized Logical Plan ==
Project [c#4481L], Statistics(sizeInBytes=31.3 KiB, rowCount=2.00E+3)
+- Filter (isnotnull(d#4482L) AND (d#4482L > scalar-subquery#4480 [])), Statistics(sizeInBytes=46.9 KiB, rowCount=2.00E+3)
   :  +- Aggregate [max(csales#4478L) AS tpcds_cmax#4479L], Statistics(sizeInBytes=16.0 B, rowCount=1)
   :     +- Aggregate [sum(b#4484L) AS csales#4478L], Statistics(sizeInBytes=16.0 B, rowCount=1)
   :        +- Project [b#4484L], Statistics(sizeInBytes=1616.0 B, rowCount=101)
   :           +- Filter (isnotnull(a#4483L) AND (a#4483L < 100)), Statistics(sizeInBytes=2.4 KiB, rowCount=101)
   :              +- Relation[a#4483L,b#4484L] parquet, Statistics(sizeInBytes=23.4 KiB, rowCount=1.00E+3)
   +- Relation[c#4481L,d#4482L] parquet, Statistics(sizeInBytes=46.9 KiB, rowCount=2.00E+3)

```

### Why are the changes needed?
Complete explain treeString's statistics

### Does this PR introduce _any_ user-facing change?
When user use explain with cost mode, user can see subquery's statistic too.

### How was this patch tested?
Added UT

Closes #31485 from AngersZhuuuu/SPARK-34137.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-10 03:21:45 +00:00
Angerszhuuuu 123365e05c [SPARK-34240][SQL] Unify output of SHOW TBLPROPERTIES clause's output attribute's schema and ExprID
### What changes were proposed in this pull request?
Passing around the output attributes should have more benefits like keeping the exprID unchanged to avoid bugs when we apply more operators above the command output DataFrame.

This PR did 2 things :

1. After this pr, a `SHOW TBLPROPERTIES` clause's output shows `key` and `value` columns whether you specify the table property `key`. Before this pr, a `SHOW TBLPROPERTIES` clause's output only show a `value` column when you specify the table property `key`..
2. Keep `SHOW TBLPROPERTIES` command's output attribute exprId unchanged.

### Why are the changes needed?
 1. Keep `SHOW TBLPROPERTIES`'s output schema consistence
 2. Keep `SHOW TBLPROPERTIES` command's output attribute exprId unchanged.

### Does this PR introduce _any_ user-facing change?
After this pr, a `SHOW TBLPROPERTIES` clause's output shows `key` and `value` columns whether you specify the table property `key`. Before this pr, a `SHOW TBLPROPERTIES` clause's output only show a `value` column when you specify the table property `key`.

Before this PR:
```
sql > SHOW TBLPROPERTIES tabe_name('key')
value
value_of_key
```

After this PR
```
sql > SHOW TBLPROPERTIES tabe_name('key')
key value
key value_of_key
```

### How was this patch tested?
Added UT

Closes #31378 from AngersZhuuuu/SPARK-34240.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-10 03:19:52 +00:00
Kousuke Saruta f79305a402 [SPARK-34311][SQL] PostgresDialect can't treat arrays of some types
### What changes were proposed in this pull request?

This PR fixes the issue that `PostgresDialect` can't treat arrays of some types.
Though PostgreSQL supports wide range of types (https://www.postgresql.org/docs/13/datatype.html),  the current `PostgresDialect` can't treat arrays of the following types.

* xml
* tsvector
* tsquery
* macaddr
* macaddr8
* txid_snapshot
* pg_snapshot
* point
* line
* lseg
* box
* path
* polygon
* circle
* pg_lsn
* bit varying
* interval

NOTE: PostgreSQL doesn't implement arrays of serial types so this PR doesn't care about them.

### Why are the changes needed?

To provide better support with PostgreSQL.

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

Yes. PostgresDialect can handle arrays of types shown above.

### How was this patch tested?

New test.

Closes #31419 from sarutak/postgres-array-types.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-02-10 11:29:14 +09:00
Angerszhuuuu 3e12e9d2ee [SPARK-34238][SQL][FOLLOW_UP] SHOW PARTITIONS Keep consistence with other SHOW command
### What changes were proposed in this pull request?
Keep consistence with other `SHOW` command according to  https://github.com/apache/spark/pull/31341#issuecomment-774613080

### Why are the changes needed?
Keep consistence

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

### How was this patch tested?
Not need

Closes #31516 from AngersZhuuuu/SPARK-34238-follow-up.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-10 02:28:05 +00:00
Holden Karau cf7a13c363 [SPARK-34209][SQL] Delegate table name validation to the session catalog
### What changes were proposed in this pull request?

Delegate table name validation to the session catalog

### Why are the changes needed?

Queerying of tables with nested namespaces.

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

SQL queries of nested namespace queries

### How was this patch tested?

Unit tests updated.

Closes #31427 from holdenk/SPARK-34209-delegate-table-name-validation-to-the-catalog.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2021-02-09 10:15:16 -08:00
Angerszhuuuu 7ea3a336b9 [SPARK-34355][CORE][SQL][FOLLOWUP] Log commit time in all File Writer
### What changes were proposed in this pull request?
When doing https://issues.apache.org/jira/browse/SPARK-34399 based  on https://github.com/apache/spark/pull/31471
Found FileBatchWrite will use `FileFormatWrite.processStates()` too. We need log commit duration  in other writer too.
In this pr:

1. Extract a commit job method in SparkHadoopWriter
2. address other commit writer

### Why are the changes needed?

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

### How was this patch tested?
No

Closes #31520 from AngersZhuuuu/SPARK-34355-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2021-02-09 16:05:39 +09:00
yikf 37fe8c6d3c [SPARK-34395][SQL] Clean up unused code for code simplifications
### What changes were proposed in this pull request?
Currently, we pass the default value `EmptyRow` to method `checkEvaluation` in the `StringExpressionsSuite`, but the default value of the 'checkEvaluation' method parameter is the `emptyRow`.

We can clean the parameter for Code Simplifications.

### Why are the changes needed?
for Code Simplifications

**before**:
```
def testConcat(inputs: String*): Unit = {
  val expected = if (inputs.contains(null)) null else inputs.mkString
  checkEvaluation(Concat(inputs.map(Literal.create(_, StringType))), expected, EmptyRow)
}
```
**after**:
```
def testConcat(inputs: String*): Unit = {
  val expected = if (inputs.contains(null)) null else inputs.mkString
  checkEvaluation(Concat(inputs.map(Literal.create(_, StringType))), expected)
}
```

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

### How was this patch tested?
Pass the Jenkins or Github action.

Closes #31510 from yikf/master.

Authored-by: yikf <13468507104@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-08 20:37:23 -06:00
gengjiaan e65b28cf7d [SPARK-34352][SQL] Improve SQLQueryTestSuite so as could run on windows system
### What changes were proposed in this pull request?
The current implement of `SQLQueryTestSuite` cannot run on windows system.
Becasue the code below will fail on windows system:
`assume(TestUtils.testCommandAvailable("/bin/bash"))`

For operation system that cannot support `/bin/bash`, we just skip some tests.

### Why are the changes needed?
SQLQueryTestSuite has a bug on windows system.

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

### How was this patch tested?
Jenkins test

Closes #31466 from beliefer/SPARK-34352.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-09 10:58:58 +09:00
yangjie01 777d51e7e3 [SPARK-34374][SQL][DSTREAM] Use standard methods to extract keys or values from a Map
### What changes were proposed in this pull request?
Use standard methods to extract `keys` or `values` from a `Map`, it's semantically consistent and  use the `DefaultKeySet` and `DefaultValuesIterable` instead of a manual loop.

**Before**
```
map.map(_._1)
map.map(_._2)
```

**After**
```
map.keys
map.values
```

### Why are the changes needed?
Code Simpilefications.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31484 from LuciferYang/keys-and-values.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-08 15:42:55 -06:00
jiake 3b26bc2536 [SPARK-34168][SQL] Support DPP in AQE when the join is Broadcast hash join at the beginning
### What changes were proposed in this pull request?
This PR is to enable AQE and DPP when the join is broadcast hash join at the beginning, which can benefit the performance improvement from DPP and AQE at the same time. This PR will make use of the result of build side and then insert the DPP filter into the probe side.

### Why are the changes needed?
Improve performance

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

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

Closes #31258 from JkSelf/supportDPP1.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 16:42:52 +00:00
Terry Kim c92e408aa1 [SPARK-34388][SQL] Propagate the registered UDF name to ScalaUDF, ScalaUDAF and ScalaAggregator
### What changes were proposed in this pull request?

This PR proposes to propagate the name used for registering UDFs to `ScalaUDF`, `ScalaUDAF` and `ScaalAggregator`.

Note that `PythonUDF` gets the name correctly: 466c045bfa/python/pyspark/sql/udf.py (L358-L359)
, and same for Hive UDFs:
466c045bfa/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala (L67)
### Why are the changes needed?

This PR can help in the following scenarios:
1) Better EXPLAIN output
2) By adding  `def name: String` to `UserDefinedExpression`, we can match an expression by `UserDefinedExpression` and look up the catalog, an use case needed for #31273.

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

The EXPLAIN output involving udfs will be changed to use the name used for UDF registration.

For example, for the following:
```
sql("CREATE TEMPORARY FUNCTION test_udf AS 'org.apache.spark.examples.sql.Spark33084'")
sql("SELECT test_udf(col1) FROM VALUES (1), (2), (3)").explain(true)
```
The output of the optimized plan will change from:
```
Aggregate [spark33084(cast(col1#223 as bigint), org.apache.spark.examples.sql.Spark330846906be0f, 1, 1) AS spark33084(col1)#237]
+- LocalRelation [col1#223]
```
to
```
Aggregate [test_udf(cast(col1#223 as bigint), org.apache.spark.examples.sql.Spark330847a62d697, 1, 1, Some(test_udf)) AS test_udf(col1)#237]
+- LocalRelation [col1#223]
```

### How was this patch tested?

Added new tests.

Closes #31500 from imback82/udaf_name.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 16:02:07 +00:00
yliou d1131bc850 [MINOR][SQL][FOLLOW-UP] Add assertion to FixedLengthRowBasedKeyValueBatch
### What changes were proposed in this pull request?
Adds an assert to `FixedLengthRowBasedKeyValueBatch#appendRow` method to check the incoming vlen and klen by comparing them with the lengths stored as member variables as followup to https://github.com/apache/spark/pull/30788

### Why are the changes needed?
Add assert statement to catch similar bugs in future.

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

### How was this patch tested?
Ran some tests locally, though not easy to test.

Closes #31447 from yliou/SPARK-33726-Assert.

Authored-by: yliou <yliou@berkeley.edu>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-08 08:46:01 -06:00
Linhong Liu 037bfb2dbc [SPARK-33438][SQL] Eagerly init objects with defined SQL Confs for command set -v
### What changes were proposed in this pull request?
In Spark, `set -v` is defined as "Queries all properties that are defined in the SQLConf of the sparkSession".
But there are other external modules that also define properties and register them to SQLConf. In this case,
it can't be displayed by `set -v` until the conf object is initiated (i.e. calling the object at least once).

In this PR, I propose to eagerly initiate all the objects registered to SQLConf, so that `set -v` will always output
the completed properties.

### Why are the changes needed?
Improve the `set -v` command to produces completed and  deterministic results

### Does this PR introduce _any_ user-facing change?
`set -v` command will dump more configs

### How was this patch tested?
existing tests

Closes #30363 from linhongliu-db/set-v.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-08 22:48:28 +09:00
Max Gekk a85490659f [SPARK-34377][SQL] Add new parquet datasource options to control datetime rebasing in read
### What changes were proposed in this pull request?
In the PR, I propose new options for the Parquet datasource:
1. `datetimeRebaseMode`
2. `int96RebaseMode`

Both options influence on loading ancient dates and timestamps column values from parquet files. The `datetimeRebaseMode` option impacts on loading values of the `DATE`, `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` types, `int96RebaseMode` impacts on loading of `INT96` timestamps.

The options support the same values as the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInRead` namely;
- `"LEGACY"`, when an option is set to this value, Spark rebases dates/timestamps from the legacy hybrid calendar (Julian + Gregorian) to the Proleptic Gregorian calendar.
- `"CORRECTED"`, dates/timestamps are read AS IS from parquet files.
- `"EXCEPTION"`, when it is set as an option value, Spark will fail the reading if it sees ancient dates/timestamps that are ambiguous between the two calendars.

### Why are the changes needed?
1. New options will allow to load parquet files from at least two sources in different rebasing modes in the same query. For instance:
```scala
val df1 = spark.read.option("datetimeRebaseMode", "legacy").parquet(folder1)
val df2 = spark.read.option("datetimeRebaseMode", "corrected").parquet(folder2)
df1.join(df2, ...)
```
Before the changes, it is impossible because the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead`  influences on both reads.

2. Mixing of Dataset/DataFrame and RDD APIs should become possible. Since SQL configs are not propagated through RDDs, the following code fails on ancient timestamps:
```scala
spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "legacy")
spark.read.parquet(folder).distinct.rdd.collect()
```

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

### How was this patch tested?
By running the modified test suites:
```
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite"
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite"
```

Closes #31489 from MaxGekk/parquet-rebase-options.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 13:28:40 +00:00
HyukjinKwon 70ef196d59 [SPARK-34157][BUILD][FOLLOW-UP] Fix Scala 2.13 compilation error via using Array.deep
### What changes were proposed in this pull request?

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

```
[error] /home/runner/work/spark/spark/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:112:53: value deep is not a member of Array[String]
[error]         assert(sql("show tables").schema.fieldNames.deep ==
[error]                                                     ^
[error] /home/runner/work/spark/spark/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:115:72: value deep is not a member of Array[String]
[error]         assert(sql("show table extended like 'tbl'").schema.fieldNames.deep ==
[error]                                                                        ^
[error] /home/runner/work/spark/spark/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:121:55: value deep is not a member of Array[String]
[error]           assert(sql("show tables").schema.fieldNames.deep ==
[error]                                                       ^
[error] /home/runner/work/spark/spark/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:124:74: value deep is not a member of Array[String]
[error]           assert(sql("show table extended like 'tbl'").schema.fieldNames.deep ==
[error]                                                                          ^
```

It broke Scala 2.13 build. This PR works around by using ScalaTests' `===` that can compare `Array`s safely.

### Why are the changes needed?

To fix the build.

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

No, dev-only.

### How was this patch tested?

CI in this PR should test it out.

Closes #31526 from HyukjinKwon/SPARK-34157.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-08 22:25:59 +09:00
Angerszhuuuu 70a79e920a [SPARK-34239][SQL][FOLLOW_UP] SHOW COLUMNS Keep consistence with other SHOW command
### What changes were proposed in this pull request?
Keep consistence with other `SHOW` command according to  https://github.com/apache/spark/pull/31341#issuecomment-774613080

### Why are the changes needed?
Keep consistence

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

### How was this patch tested?
Not need

Closes #31518 from AngersZhuuuu/SPARK-34239-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 11:39:59 +00:00
gengjiaan 2c243c93d9 [SPARK-34157][SQL] Unify output of SHOW TABLES and pass output attributes properly
### What changes were proposed in this pull request?
The current implement of some DDL not unify the output and not pass the output properly to physical command.
Such as: The `ShowTables` output attributes `namespace`, but `ShowTablesCommand` output attributes `database`.

As the query plan, this PR pass the output attributes from `ShowTables` to `ShowTablesCommand`, `ShowTableExtended ` to `ShowTablesCommand`.

Take `show tables` and `show table extended like 'tbl'` as example.
The output before this PR:
`show tables`
|database|tableName|isTemporary|
-- | -- | --
| default|      tbl|      false|

If catalog is v2 session catalog, the output before this PR:
|namespace|tableName|
-- | --
| default|      tbl

`show table extended like 'tbl'`
|database|tableName|isTemporary|         information|
-- | -- | -- | --
| default|      tbl|      false|Database: default...|

The output after this PR:
`show tables`
|namespace|tableName|isTemporary|
-- | -- | --
|  default|      tbl|      false|

`show table extended like 'tbl'`
|namespace|tableName|isTemporary|         information|
-- | -- | -- | --
|  default|      tbl|      false|Database: default...|

### Why are the changes needed?
This PR have benefits as follows:
First, Unify schema for the output of SHOW TABLES.
Second, pass the output attributes could keep the expr ID unchanged, so that avoid bugs when we apply more operators above the command output dataframe.

### Does this PR introduce _any_ user-facing change?
Yes.
The output schema of `SHOW TABLES` replace `database` by `namespace`.

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

Closes #31245 from beliefer/SPARK-34157.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 08:39:58 +00:00
ulysses-you 9270238473 [SPARK-34355][SQL] Add log and time cost for commit job
### What changes were proposed in this pull request?

Add some info log around commit log.

### Why are the changes needed?

Th commit job is a heavy option and we have seen many times Spark block at this code place due to the slow rpc with namenode or other.

It's better to record the time that commit job cost.

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

Yes, more info log.

### How was this patch tested?

Not need.

Closes #31471 from ulysses-you/add-commit-log.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2021-02-08 16:44:59 +09:00
Yuming Wang 6e05e99143 [SPARK-34342][SQL] Format DateLiteral and TimestampLiteral toString
### What changes were proposed in this pull request?

This pr format DateLiteral and TimestampLiteral toString. For example:
```sql
SELECT * FROM date_dim WHERE d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days)
```
Before this pr:
```
Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10997)) AND (d_date#18 <= 11057))
```
After this pr:
```
Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-02-10)) AND (d_date#14 <= 2000-04-10))
```

### Why are the changes needed?

Make the plan more readable.

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

No.

### How was this patch tested?

Unit test.

Closes #31455 from wangyum/SPARK-34342.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-06 19:49:38 -08:00
“attilapiros” cc508d17c7 [SPARK-34370][SQL] Support Avro schema evolution for partitioned Hive tables using "avro.schema.url"
### What changes were proposed in this pull request?

With https://github.com/apache/spark/pull/31133 Avro schema evolution is introduce for partitioned hive tables where the schema is given by `avro.schema.literal`.
Here that functionality is extended to support schema evolution where the schema is defined via `avro.schema.url`.

### Why are the changes needed?

Without this PR the problem described in https://github.com/apache/spark/pull/31133 can be reproduced by tables where `avro.schema.url` is used. As in this case always the property value given at partition level is used for the `avro.schema.url`.

So for example when a new column (with a default value) is added to the table then one the following problem happens:
-  when the new field is added after the last one the cell values will be null values instead of the default value
-  when the schema is extended somewhere before the last field then values will be listed for the wrong column positions

Similar error will happen when one of the field is removed from the schema.

For details please check the attached unit tests where both cases are checked.

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

Fixes the potential value error.

### How was this patch tested?

The existing unit tests for schema evolution is generalized and reused.
New tests:
- `SPARK-34370: support Avro schema evolution (add column with avro.schema.url)`
- `SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)`

Closes #31501 from attilapiros/SPARK-34370.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-06 17:25:39 -08:00
Max Gekk 6845d26057 [SPARK-34385][SQL] Unwrap SparkUpgradeException in v2 Parquet datasource
### What changes were proposed in this pull request?
Unwrap `SparkUpgradeException` from `ParquetDecodingException` in v2 `FilePartitionReader` in the same way as v1 implementation does: 3a299aa648/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala (L180-L183)

### Why are the changes needed?
1. To be compatible with v1 implementation of the Parquet datasource.
2. To improve UX with Spark SQL by making `SparkUpgradeException` more visible.

### Does this PR introduce _any_ user-facing change?
Yes, it can.

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite"
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite"
```

Closes #31497 from MaxGekk/parquet-spark-upgrade-exception.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-06 16:49:15 -08:00
tanel.kiis@gmail.com c73f70bb0d [SPARK-34141][SQL] Remove side effect from ExtractGenerator
### What changes were proposed in this pull request?

Rewrote one `ExtractGenerator` case such that it would not rely on a side effect of the flatmap function.

### Why are the changes needed?

With the dataframe api it is possible to have a lazy sequence as the `output` of a `LogicalPlan`. When exploding a column on this dataframe using the `withColumn("newName", explode(col("name")))` method, the `ExtractGenerator` does not extract the generator and `CheckAnalysis` would throw an exception.

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

Bugfix
Before this, the work around was to put `.select("*")` before the explode.

### How was this patch tested?

UT

Closes #31213 from tanelk/SPARK-34141_extract_generator.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-06 13:27:07 -06:00
“attilapiros” e614f34c7a [SPARK-26836][SQL] Supporting Avro schema evolution for partitioned Hive tables with "avro.schema.literal"
### What changes were proposed in this pull request?

Before this PR for a partitioned Avro Hive table when the SerDe is configured to read the partition data
the table level properties were overwritten by the partition level properties.

This PR changes this ordering by giving table level properties higher precedence  thus when a new evolved schema
is set for the table this new schema will be used to read the partition data and not the original schema which was used for writing the data.

This new behavior is consistent with Apache Hive.
See the example used in the unit test `SPARK-26836: support Avro schema evolution`, in Hive this results in:

```
0: jdbc:hive2://<IP>:10000> select * from t;
INFO  : Compiling command(queryId=hive_20210111141102_7a6349d0-f9ed-4aad-ac07-b94b44de2394): select * from t
INFO  : Semantic Analysis Completed
INFO  : Returning Hive schema: Schema(fieldSchemas:[FieldSchema(name:t.col1, type:string, comment:null), FieldSchema(name:t.col2, type:string, comment:null), FieldSchema(name:t.ds, type:string, comment:null)], properties:null)
INFO  : Completed compiling command(queryId=hive_20210111141102_7a6349d0-f9ed-4aad-ac07-b94b44de2394); Time taken: 0.098 seconds
INFO  : Executing command(queryId=hive_20210111141102_7a6349d0-f9ed-4aad-ac07-b94b44de2394): select * from t
INFO  : Completed executing command(queryId=hive_20210111141102_7a6349d0-f9ed-4aad-ac07-b94b44de2394); Time taken: 0.013 seconds
INFO  : OK
+---------------+-------------+-------------+
|    t.col1     |   t.col2    |    t.ds     |
+---------------+-------------+-------------+
| col1_default  | col2_value  | 1981-01-07  |
| col1_value    | col2_value  | 1983-04-27  |
+---------------+-------------+-------------+
2 rows selected (0.159 seconds)
```

### Why are the changes needed?

Without this change the old schema would be used. This can use a correctness issue when the new schema introduces
a new field with a default value (following the rules of schema evolution) before an existing field.
In this case the rows coming from the partition where the old schema was used will **contain values in wrong column positions**.

For example check the attached unit test `SPARK-26836: support Avro schema evolution`

Without this fix the result of the select on the table would be:

```
+----------+----------+----------+
|      col1|      col2|        ds|
+----------+----------+----------+
|col2_value|      null|1981-01-07|
|col1_value|col2_value|1983-04-27|
+----------+----------+----------+

```

With this fix:

```
+------------+----------+----------+
|        col1|      col2|        ds|
+------------+----------+----------+
|col1_default|col2_value|1981-01-07|
|  col1_value|col2_value|1983-04-27|
+------------+----------+----------+
```

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

Just fixes the value errors.
When a new column is introduced even to the last position then instead of 'null' the given default will be used.

### How was this patch tested?

This was tested with the unit tested included to the PR.
And manually on Apache Spark / Hive.

Closes #31133 from attilapiros/SPARK-26836.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-05 10:56:25 -08:00
Cheng Su 76baaf7465 [SPARK-32985][SQL] Decouple bucket scan and bucket filter pruning for data source v1
### What changes were proposed in this pull request?

As a followup from discussion in https://github.com/apache/spark/pull/29804#discussion_r493100510 . Currently in data source v1 file scan `FileSourceScanExec`, [bucket filter pruning will only take effect with bucket table scan](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L542 ). However this is unnecessary, as bucket filter pruning can also happen if we disable bucketed table scan. Read files with bucket hash partitioning, and bucket filter pruning are two orthogonal features, and do not need to couple together.

### Why are the changes needed?

This help query leverage the benefit from bucket filter pruning to save CPU/IO to not read unnecessary bucket files, and do not bound by bucket table scan when the parallelism of tasks is a concern.

In addition, this also resolves the issue to reduce number of tasks launched for simple query with bucket column filter - SPARK-33207, because with bucket scan, we launch # of tasks to equal to # of buckets, and this is unnecessary.

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

Users will notice query to start pruning irrelevant files for reading bucketed table, when disabling bucketing. If the input data does not follow spark data source bucketing convention, by default exception will be thrown and query will be failed. The exception can be bypassed with setting config `spark.sql.files.ignoreCorruptFiles` to true.

### How was this patch tested?

Added unit test in `BucketedReadSuite.scala` to make all existing unit tests for bucket filter work with this PR.

Closes #31413 from c21/bucket-pruning.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-05 13:00:06 +00:00
Wenchen Fan 989eb6884d [SPARK-34331][SQL] Speed up DS v2 metadata col resolution
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/28027

https://github.com/apache/spark/pull/28027 added a DS v2 API that allows data sources to produce metadata/hidden columns that can only be seen when it's explicitly selected. The way we integrate this API into Spark is:
1. The v2 relation gets normal output and metadata output from the data source, and the metadata output is excluded from the plan output by default.
2. column resolution can resolve `UnresolvedAttribute` with metadata columns, even if the child plan doesn't output metadata columns.
3. An analyzer rule searches the query plan, trying to find a node that has missing inputs. If such node is found, transform the sub-plan of this node, and update the v2 relation to include the metadata output.

The analyzer rule in step 3 brings a perf regression, for queries that do not read v2 tables at all. This rule will calculate `QueryPlan.inputSet` (which builds an `AttributeSet` from outputs of all children) and `QueryPlan.missingInput` (which does a set exclusion and creates a new `AttributeSet`) for every plan node in the query plan. In our benchmark, the TPCDS query compilation time gets increased by more than 10%

This PR proposes a simple way to improve it: we add a special metadata entry to the metadata attribute, which allows us to quickly check if a plan needs to add metadata columns: we just check all the references of this plan, and see if the attribute contains the special metadata entry, instead of calculating `QueryPlan.missingInput`.

This PR also fixes one bug: we should not change the final output schema of the plan, if we only use metadata columns in operators like filter, sort, etc.

### Why are the changes needed?

Fix perf regression in SQL query compilation, and fix a bug.

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

No

### How was this patch tested?

Run `org.apache.spark.sql.TPCDSQuerySuite`, before this PR, `AddMetadataColumns` is the top 4 rule ranked by running time
```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 407641
Total time: 47.257239779 seconds

Rule                                  Effective Time / Total Time                     Effective Runs / Total Runs

OptimizeSubqueries                      4157690003 / 8485444626                         49 / 2778
Analyzer$ResolveAggregateFunctions      1238968711 / 3369351761                         49 / 2141
ColumnPruning                           660038236 / 2924755292                          338 / 6391
Analyzer$AddMetadataColumns             0 / 2918352992                                  0 / 2151
```
after this PR:
```
Analyzer$AddMetadataColumns             0 / 122885629                                   0 / 2151
```
This rule is 20 times faster and is negligible to the total compilation time.

This PR also add new tests to verify the bug fix.

Closes #31440 from cloud-fan/metadata-col.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-05 16:37:29 +08:00
Max Gekk ee11a8f407 [SPARK-34371][SQL][TESTS] Run the datetime rebasing tests for Parquet datasource v1 and v2
### What changes were proposed in this pull request?
Extract the date/timestamps rebasing tests from `ParquetIOSuite` to `ParquetRebaseDatetimeSuite` to run them for both DSv1 and DSv2 implementations of Parquet datasource.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running new test suites:
```
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite"
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite"
$ build/sbt "sql/test:testOnly *ParquetIOSuite"
```

Closes #31478 from MaxGekk/rebase-tests-dsv1-and-dsv2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-05 07:10:37 +00:00
Wenchen Fan 361d702f8d [SPARK-34359][SQL] Add a legacy config to restore the output schema of SHOW DATABASES
### What changes were proposed in this pull request?

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

In #26006 , we merged the v1 and v2 SHOW DATABASES/NAMESPACES commands, but we missed a behavior change that the output schema of SHOW DATABASES becomes different.

This PR adds a legacy config to restore the old schema, with a migration guide item to mention this behavior change.

### Why are the changes needed?

Improve backward compatibility

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

No (the legacy config is false by default)

### How was this patch tested?

a new test

Closes #31474 from cloud-fan/command-schema.

Lead-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-05 04:57:51 +00:00
Kent Yao 961c85166a [SPARK-34346][CORE][SQL] io.file.buffer.size set by spark.buffer.size will override by loading hive-site.xml accidentally may cause perf regression
### What changes were proposed in this pull request?

In many real-world cases, when interacting with hive catalog through Spark SQL, users may just share the `hive-site.xml` for their hive jobs and make a copy to `SPARK_HOME`/conf w/o modification. In Spark, when we generate Hadoop configurations, we will use `spark.buffer.size(65536)` to reset `io.file.buffer.size(4096)`. But when we load the hive-site.xml, we may ignore this behavior and reset `io.file.buffer.size` again according to `hive-site.xml`.

1. The configuration priority for setting Hadoop and Hive config here is not right, while literally, the order should be `spark > spark.hive > spark.hadoop > hive > hadoop`

2. This breaks `spark.buffer.size` congfig's behavior for tuning the IO performance w/ HDFS if there is an existing `io.file.buffer.size` in hive-site.xml

### Why are the changes needed?

bugfix for configuration behavior and fix performance regression by that behavior change

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

this pr restores silent user face change

### How was this patch tested?

new tests

Closes #31460 from yaooqinn/SPARK-34346.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-05 10:13:19 +09:00
Jungtaek Lim (HeartSaVioR) fbe726f5b1 [SPARK-34339][CORE][SQL] Expose the number of total paths in Utils.buildLocationMetadata()
### What changes were proposed in this pull request?

This PR proposes to expose the number of total paths in Utils.buildLocationMetadata(), with relaxing space usage a bit (around 10+ chars).

Suppose the first 2 of 5 paths are only fit to the threshold, the outputs between the twos are below:

* before the change: `[path1, path2]`
* after the change: `(5 paths)[path1, path2, ...]`

### Why are the changes needed?

SPARK-31793 silently truncates the paths hence end users can't indicate how many paths are truncated, and even more, whether paths are truncated or not.

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

Yes, the location metadata will also show how many paths are truncated (not shown), instead of silently truncated.

### How was this patch tested?

Modified UTs

Closes #31464 from HeartSaVioR/SPARK-34339.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-05 09:37:38 +09:00
Hoa 7675582dab [SPARK-34357][SQL] Map JDBC SQL TIME type to TimestampType with time portion fixed regardless of timezone
### What changes were proposed in this pull request?

Due to user-experience (confusing to Spark users - java.sql.Time using milliseconds vs Spark using microseconds; and user losing useful functions like hour(), minute(), etc on the column), we have decided to revert back to use TimestampType but this time we will enforce the hour to be consistently across system timezone (via offset manipulation) and date part fixed to zero epoch.

Full Discussion with Wenchen Fan Wenchen Fan regarding this ticket is here https://github.com/apache/spark/pull/30902#discussion_r569186823

### Why are the changes needed?

Revert and improvement to sql.Time handling

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

No

### How was this patch tested?

Unit tests and integration tests

Closes #31473 from saikocat/SPARK-34357.

Authored-by: Hoa <hoameomu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-04 17:16:39 +00:00
Jungtaek Lim (HeartSaVioR) 44dcf0062c [SPARK-34326][CORE][SQL] Fix UTs added in SPARK-31793 depending on the length of temp path
### What changes were proposed in this pull request?

This PR proposes to fix the UTs being added in SPARK-31793, so that all things contributing the length limit are properly accounted.

### Why are the changes needed?

The test `DataSourceScanExecRedactionSuite.SPARK-31793: FileSourceScanExec metadata should contain limited file paths` is failing conditionally, depending on the length of the temp directory.

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

No.

### How was this patch tested?

Modified UTs explain the missing points, which also do the test.

Closes #31449 from HeartSaVioR/SPARK-34326-v2.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-02-04 08:46:11 +09:00
Terry Kim 3d7e1397d6 [SPARK-34317][SQL][FOLLOW-UP] Use relationTypeMismatchHint when UnresolvedTable is resolved to a temp view
### What changes were proposed in this pull request?

This is a follow up to #31424, and proposes to use `UnresolvedTable.relationTypeMismatchHint` when `UnresolvedTable` is resolved to a temp view.

### Why are the changes needed?

This change utilizes the type mismatch hint when a relation is resolved to a temp view when a table is expected.

For example, `ALTER TABLE tmpView SET TBLPROPERTIES ('p' = 'an')` will now include `Please use ALTER VIEW instead.` in the exception message: `tmpView is a temp view. 'ALTER TABLE ... SET TBLPROPERTIES' expects a table. Please use ALTER VIEW instead.`

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

Yes, adds the hint in the exception message.

### How was this patch tested?

Update existing tests to include the hint.

Closes #31452 from imback82/followup_SPARK-34317.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-03 16:12:27 +00:00
Max Gekk 7bfb4a4642 [SPARK-34304][SQL] Remove view checks in v1 alter table commands
### What changes were proposed in this pull request?
Remove the check `verifyAlterTableType()` from the following v1 commands:
- AlterTableAddPartitionCommand
- AlterTableDropPartitionCommand
- AlterTableRenamePartitionCommand
- AlterTableRecoverPartitionsCommand
- AlterTableSerDePropertiesCommand
- AlterTableSetLocationCommand

The check is not needed any more after migration on new resolution framework, see SPARK-29900.

Also new tests were added to:
- AlterTableAddPartitionSuiteBase
- AlterTableDropPartitionSuiteBase
- AlterTableRenamePartitionSuiteBase
- v1/AlterTableRecoverPartitionsSuite

and removed duplicate tests from `SQLViewSuite` and `HiveDDLSuite`.

The tests for `AlterTableSerDePropertiesCommand`/`AlterTableSetLocationCommand` exist in SQLViewSuite` and `HiveDDLSuite`, and they can be ported to unified tests after SPARK-34305 and SPARK-34332.

The `ALTER TABLE .. CHANGE COLUMN` command accepts only tables too, so, the check can be removed after migration on new resolution framework, SPARK-34302.

### Why are the changes needed?
To improve code maintenance by removing dead code.

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

### How was this patch tested?
1. Added new tests to unified test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsSuite"
```
2. Run the modified test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *SQLViewSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *HiveDDLSuite"
```

Closes #31405 from MaxGekk/remove-view-check-in-alter-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-03 16:11:36 +00:00
allisonwang-db 76a7fca4e1 [SPARK-34335][SQL] Support referencing subquery with column aliases by table alias
### What changes were proposed in this pull request?
This PR adds support for referencing subquery with column aliases by its table alias.

Before
```sql
-- AnalysisException: cannot resolve '`t.c1`' given input columns: [c1, c2];
SELECT t.c1, t.c2 FROM (SELECT 1 AS a, 1 AS b) t(c1, c2)
```

After:
```sql
-- [(1, 1)]
SELECT t.c1, t.c2 FROM (SELECT 1 AS a, 1 AS b) t(c1, c2)
```

### Why are the changes needed?
To allow users to reference subquery with column aliases by its table alias.

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

### How was this patch tested?
Added parser tests and SQL query tests.

Closes #31444 from allisonwang-db/spark-34335.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-03 08:51:28 +00:00
Terry Kim a1d4bb3300 [SPARK-34313][SQL] Migrate ALTER TABLE SET/UNSET TBLPROPERTIES commands to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `ALTER TABLE ... SET/UNSET TBLPROPERTIES` to use `UnresolvedTable` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

### 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?

After this PR, `ALTER TABLE SET/UNSET TBLPROPERTIES` will have a consistent resolution behavior.

### How was this patch tested?

Updated existing tests / added new tests.

Closes #31422 from imback82/v2_alter_table_set_unset_properties.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-03 05:44:58 +00:00
Ruifeng Zheng fc80a5b877 [SPARK-34307][SQL] TakeOrderedAndProjectExec avoid shuffle if input rdd has single partition
### What changes were proposed in this pull request?
when the child rdd has only one partition, skip the shuffle

### Why are the changes needed?
avoid unnecessary shuffle

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

### How was this patch tested?
existing testsuites

Closes #31409 from zhengruifeng/limit_with_single_partition.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-03 04:49:08 +00:00
HyukjinKwon e927bf90e0 Revert "[SPARK-34326][CORE][SQL] Fix UTs added in SPARK-31793 depending on the length of temp path"
This reverts commit 63866025d2.
2021-02-03 12:32:39 +09:00
Kousuke Saruta 603a7fd7b6 [SPARK-34308][SQL] Escape meta-characters in printSchema
### What changes were proposed in this pull request?

Similar to SPARK-33690, this PR improves the output layout of `printSchema` for the case column names contain meta characters.
Here is an example.

Before:
```
scala> val df1 = spark.sql("SELECT 'aaa\nbbb\tccc\rddd\feee\bfff\u000Bggg\u0007hhh'")
scala> df1.printSchema
root
 |-- aaa
ddd	ccc
   eefff
        ggghhh: string (nullable = false)
```

After:
```
scala> df1.printSchema
root
 |-- aaa\nbbb\tccc\rddd\feee\bfff\vggg\ahhh: string (nullable = false)
```

### Why are the changes needed?

To avoid breaking the layout of `Dataset#printSchema`

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

No.

### How was this patch tested?

New test.

Closes #31412 from sarutak/escape-meta-printSchema.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-03 11:06:41 +09:00
Wenchen Fan 00120ea537 [SPARK-34212][SQL][FOLLOWUP] Parquet vectorized reader can read decimal fields with a larger precision
### What changes were proposed in this pull request?

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

#31357 added a very strong restriction to the vectorized parquet reader, that the spark data type must exactly match the physical parquet type, when reading decimal fields. This restriction is actually not necessary, as we can safely read parquet decimals with a larger precision. This PR releases this restriction a little bit.

### Why are the changes needed?

To not fail queries unnecessarily.

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

Yes, now users can read parquet decimals with mismatched `DecimalType` as long as the scale is the same and precision is larger.

### How was this patch tested?

updated test.

Closes #31443 from cloud-fan/improve.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-03 09:26:36 +09:00
Jungtaek Lim (HeartSaVioR) 63866025d2 [SPARK-34326][CORE][SQL] Fix UTs added in SPARK-31793 depending on the length of temp path
### What changes were proposed in this pull request?

This PR proposes to fix the UTs being added in SPARK-31793, so that all things contributing the length limit are properly accounted.

### Why are the changes needed?

The test `DataSourceScanExecRedactionSuite.SPARK-31793: FileSourceScanExec metadata should contain limited file paths` is failing conditionally, depending on the length of the temp directory.

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

No.

### How was this patch tested?

Modified UTs explain the missing points, which also do the test.

Closes #31435 from HeartSaVioR/SPARK-34326.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-02-03 07:35:22 +09:00
Liang-Chi Hsieh cadca8d352 [SPARK-34324][SQL] FileTable should not list TRUNCATE in capabilities by default
### What changes were proposed in this pull request?

This patch proposes to remove `TRUNCATE` from the default `capabilities` list from `FileTable`.

### Why are the changes needed?

The abstract class `FileTable` now lists `TRUNCATE` in its `capabilities`, but `FileTable` does not know if an implementation really supports truncation or not. Specifically, we can check existing `FileTable` implementations including `AvroTable`, `CSVTable`, `JsonTable`, etc. No one implementation really implements `SupportsTruncate` in its writer builder.

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

No, because seems to me `FileTable` is not of user-facing public API.

### How was this patch tested?

Existing unit tests.

Closes #31432 from viirya/SPARK-34324.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-02-02 11:41:05 -08:00
Kousuke Saruta d308794adb [SPARK-34263][SQL] Simplify the code for treating unicode/octal/escaped characters in string literals
### What changes were proposed in this pull request?

In the current master, the code for treating unicode/octal/escaped characters in string literals is a little bit complex so let's simplify it.

### Why are the changes needed?

To keep it easy to maintain.

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

No.

### How was this patch tested?

`ParserUtilsSuite` passes.

Closes #31362 from sarutak/refactor-unicode-escapes.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-02-03 01:07:12 +09:00
Max Gekk 79515b82f1 [SPARK-34282][SQL][TESTS] Unify v1 and v2 TRUNCATE TABLE tests
### What changes were proposed in this pull request?
1. Move parser tests from `DDLParserSuite` to `TruncateTableParserSuite`.
2. Port DS v1 tests from `DDLSuite` and other test suites to `v1.TruncateTableSuiteBase` and to `v1.TruncateTableSuite`.
3. Add a test for DSv2 `TRUNCATE TABLE` to `v2.TruncateTableSuite`.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *TruncateTableSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31387 from MaxGekk/unify-truncate-table-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-02 14:32:35 +00:00
Gengliang Wang ff1b6ecc37 [SPARK-33591][SQL][FOLLOW-UP] Revise the version and doc of spark.sql.legacy.parseNullPartitionSpecAsStringLiteral
### What changes were proposed in this pull request?

Correct the version of SQL configuration `spark.sql.legacy.parseNullPartitionSpecAsStringLiteral` from 3.2.0 to 3.0.2.
Also, revise the documentation and test case.

### Why are the changes needed?

The release version in https://github.com/apache/spark/pull/31421 was wrong.

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

No

### How was this patch tested?

Unit tests

Closes #31434 from gengliangwang/reviseVersion.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-02 13:51:20 +00:00
gengjiaan 5b2ad59f64 [SPARK-33599][SQL] Restore the assert-like in catalyst/analysis
### What changes were proposed in this pull request?
There exists some `Exception` for assert in fact. Such as:
`throw new IllegalStateException("[BUG] unexpected plan returned by `lookupV2Relation`: " + other)`

This kind `Exception` seems should not put in single dedicated files.

### Why are the changes needed?
Reduce the workload of auditing.

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

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

Closes #31395 from beliefer/SPARK-33599-restore-assert.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-02 13:28:28 +00:00
Kousuke Saruta 66f3480f2b [SPARK-34318][SQL] Dataset.colRegex should work with column names and qualifiers which contain newlines
### What changes were proposed in this pull request?

This PR fixes an issue that `Dataset.colRegex` doesn't work with column names or qualifiers which contain newlines.
In the current master, if column names or qualifiers passed to `colRegex` contain newlines, it throws exception.
```
val df = Seq(1, 2, 3).toDF("test\n_column").as("test\n_table")
val col1 = df.colRegex("`tes.*\n.*mn`")
org.apache.spark.sql.AnalysisException: Cannot resolve column name "`tes.*
.*mn`" among (test
_column)
  at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$resolveException(Dataset.scala:272)
  at org.apache.spark.sql.Dataset.$anonfun$resolve$1(Dataset.scala:263)
  at scala.Option.getOrElse(Option.scala:189)
  at org.apache.spark.sql.Dataset.resolve(Dataset.scala:263)
  at org.apache.spark.sql.Dataset.colRegex(Dataset.scala:1407)
  ... 47 elided

val col2 = df.colRegex("test\n_table.`tes.*\n.*mn`")
org.apache.spark.sql.AnalysisException: Cannot resolve column name "test
_table.`tes.*
.*mn`" among (test
_column)
  at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$resolveException(Dataset.scala:272)
  at org.apache.spark.sql.Dataset.$anonfun$resolve$1(Dataset.scala:263)
  at scala.Option.getOrElse(Option.scala:189)
  at org.apache.spark.sql.Dataset.resolve(Dataset.scala:263)
  at org.apache.spark.sql.Dataset.colRegex(Dataset.scala:1407)
  ... 47 elided
```

### Why are the changes needed?

Column names and qualifiers can contain newlines but `colRegex` can't work with them, so it's a bug.

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

Yes. users can pass column names and qualifiers even though they contain newlines.

### How was this patch tested?

New test.

Closes #31426 from sarutak/fix-colRegex.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-02-02 21:47:11 +09:00
Max Gekk 6d3674bb62 [SPARK-34312][SQL] Support partition(s) truncation by Supports(Atomic)PartitionManagement
### What changes were proposed in this pull request?
1. Add new method `truncatePartition()` to the `SupportsPartitionManagement` interface.
2. Add new method `truncatePartitions()` to the `SupportsAtomicPartitionManagement` interface.
3. Default implementation of new methods in `InMemoryPartitionTable`/`InMemoryAtomicPartitionTable`.

### Why are the changes needed?
This is the first step in supporting of v2 `TRUNCATE TABLE .. PARTITION`.

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *SupportsPartitionManagementSuite"
$ build/sbt "test:testOnly *SupportsAtomicPartitionManagementSuite"
```

Closes #31420 from MaxGekk/dsv2-truncate-table-partitions.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-02 08:25:59 +00:00
Terry Kim f024d3051c [SPARK-34317][SQL] Introduce relationTypeMismatchHint to UnresolvedTable for a better error message
### What changes were proposed in this pull request?

This PR proposes to add `relationTypeMismatchHint` to `UnresolvedTable` so that if a relation is resolved to a view when a table is expected, a hint message can be included as a part of the analysis exception message. Note that the same feature is already introduced to `UnresolvedView` in #30636.

This mostly affects `ALTER TABLE` commands where the analysis exception message will now contain `Please use ALTER VIEW as instead`.

### Why are the changes needed?

To give a better error message. (The hint used to exist but got removed for commands that migrated to the new resolution framework)

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

Yes, now `ALTER TABLE` commands include a hint to use `ALTER VIEW` instead.
```
sql("ALTER TABLE v SET SERDE 'whatever'")
```
Before:
```
"v is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table.
```
After this PR:
```
"v is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table. Please use ALTER VIEW instead.
```

### How was this patch tested?

Updated existing test cases to include the hint.

Closes #31424 from imback82/better_error.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-02 08:24:44 +00:00
Linhong Liu bb9bf66bb6 [SPARK-34199][SQL] Block table.* inside function to follow ANSI standard and other SQL engines
### What changes were proposed in this pull request?
In spark, the `count(table.*)` may cause very weird result, for example:
```
select count(*) from (select 1 as a, null as b) t;
output: 1
select count(t.*) from (select 1 as a, null as b) t;
output: 0
```
 This is because spark expands `t.*` while converts `*` to count(1), this will confuse
users. After checking the ANSI standard, `count(*)` should always be `count(1)` while `count(t.*)`
is not allowed. What's more, this is also not allowed by common databases, e.g. MySQL, Oracle.

So, this PR proposes to block the ambiguous behavior and print a clear error message for users.

### Why are the changes needed?
to avoid ambiguous behavior and follow ANSI standard and other SQL engines

### Does this PR introduce _any_ user-facing change?
Yes, `count(table.*)` behavior will be blocked and output an error message.

### How was this patch tested?
newly added and existing tests

Closes #31286 from linhongliu-db/fix-table-star.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-02 07:49:50 +00:00
yi.wu e9362c2571 [SPARK-34319][SQL] Resolve duplicate attributes for FlatMapCoGroupsInPandas/MapInPandas
### What changes were proposed in this pull request?

Resolve duplicate attributes for `FlatMapCoGroupsInPandas`.

### Why are the changes needed?

When performing self-join on top of `FlatMapCoGroupsInPandas`, analysis can fail because of conflicting attributes. For example,

```scala
df = spark.createDataFrame([(1, 1)], ("column", "value"))
row = df.groupby("ColUmn").cogroup(
    df.groupby("COLUMN")
).applyInPandas(lambda r, l: r + l, "column long, value long")
row.join(row).show()
```
error:

```scala
...
Conflicting attributes: column#163321L,value#163322L
;;
’Join Inner
:- FlatMapCoGroupsInPandas [ColUmn#163312L], [COLUMN#163312L], <lambda>(column#163312L, value#163313L, column#163312L, value#163313L), [column#163321L, value#163322L]
:  :- Project [ColUmn#163312L, column#163312L, value#163313L]
:  :  +- LogicalRDD [column#163312L, value#163313L], false
:  +- Project [COLUMN#163312L, column#163312L, value#163313L]
:     +- LogicalRDD [column#163312L, value#163313L], false
+- FlatMapCoGroupsInPandas [ColUmn#163312L], [COLUMN#163312L], <lambda>(column#163312L, value#163313L, column#163312L, value#163313L), [column#163321L, value#163322L]
   :- Project [ColUmn#163312L, column#163312L, value#163313L]
   :  +- LogicalRDD [column#163312L, value#163313L], false
   +- Project [COLUMN#163312L, column#163312L, value#163313L]
      +- LogicalRDD [column#163312L, value#163313L], false
...
```

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

yes, the query like the above example won't fail.

### How was this patch tested?

Adde unit tests.

Closes #31429 from Ngone51/fix-conflcting-attrs-of-FlatMapCoGroupsInPandas.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-02 16:25:32 +09:00
Gengliang Wang 521397f2f9 [SPARK-33591][SQL][FOLLOWUP] Add legacy config for recognizing null partition spec values
### What changes were proposed in this pull request?

This is a follow up for https://github.com/apache/spark/pull/30538.
It adds a legacy conf `spark.sql.legacy.parseNullPartitionSpecAsStringLiteral` in case users wants the legacy behavior.
It also adds document for the behavior change.

### Why are the changes needed?

In case users want the legacy behavior, they can set `spark.sql.legacy.parseNullPartitionSpecAsStringLiteral` as true.

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

Yes, adding a legacy configuration to restore the old behavior.

### How was this patch tested?

Unit test.

Closes #31421 from gengliangwang/legacyNullStringConstant.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-02 16:13:40 +09:00
HyukjinKwon 30468a9015 [SPARK-34306][SQL][PYTHON][R] Use Snake naming rule across the function APIs
### What changes were proposed in this pull request?

This PR completes snake_case rule at functions APIs across the languages, see also SPARK-10621.

In more details, this PR:
- Adds `count_distinct` in Scala Python, and R, and document that `count_distinct` is encouraged. This was not deprecated because `countDistinct` is pretty commonly used. We could deprecate in the future releases.
- (Scala-specific) adds `typedlit` but doesn't deprecate `typedLit` which is arguably commonly used. Likewise, we could deprecate in the future releases.
- Deprecates and renames:
  - `sumDistinct` -> `sum_distinct`
  - `bitwiseNOT` -> `bitwise_not`
  - `shiftLeft` -> `shiftleft` (matched with SQL name in `FunctionRegistry`)
  - `shiftRight` -> `shiftright` (matched with SQL name in `FunctionRegistry`)
  - `shiftRightUnsigned` -> `shiftrightunsigned` (matched with SQL name in `FunctionRegistry`)
  - (Scala-specific) `callUDF` -> `call_udf`

### Why are the changes needed?

To keep the consistent naming in APIs.

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

Yes, it deprecates some APIs and add new renamed APIs as described above.

### How was this patch tested?

Unittests were added.

Closes #31408 from HyukjinKwon/SPARK-34306.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-02 09:29:40 +09:00
yangjie01 9db566a882 [SPARK-34310][CORE][SQL] Replaces map and flatten with flatMap
### What changes were proposed in this pull request?
Replaces `collection.map(f1).flatten(f2)` with `collection.flatMap` if possible. it's semantically consistent, but looks simpler.

### Why are the changes needed?
Code Simpilefications.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31416 from LuciferYang/SPARK-34310.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-01 08:21:35 -06:00
Angerszhuuuu 74116b6b25 [SPARK-34239][SQL] Unify output of SHOW COLUMNS pass output attributes properly
### What changes were proposed in this pull request?
Passing around the output attributes should have more benefits like keeping the expr ID unchanged to avoid bugs when we apply more operators above the command output dataframe.

This PR keep SHOW COLUMNS command's output attribute exprId unchanged.

### Why are the changes needed?
 Keep SHOW PARTITIONS command's output attribute exprid unchanged.

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

### How was this patch tested?
Added UT

Closes #31377 from AngersZhuuuu/SPARK-34239.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-01 14:16:03 +00:00
Max Gekk 2b76e6d15c [SPARK-34301][SQL] Use logical plan of alter table in CatalogImpl.recoverPartitions()
### What changes were proposed in this pull request?
Replace v1 exec node `AlterTableRecoverPartitionsCommand` by the logical node `AlterTableRecoverPartitions` in `CatalogImpl.recoverPartitions()`.

### Why are the changes needed?
1. Print user friendly error message for views:
```
my_temp_table is a temp view. 'recoverPartitions()' expects a table
```
Before the changes:
```
Table or view 'my_temp_table' not found in database 'default'
```

2. To not bind to v1 `ALTER TABLE .. RECOVER PARTITIONS`, and to support v2 tables potentially as well.

### Does this PR introduce _any_ user-facing change?
Yes, it can.

### How was this patch tested?
By running new test in `CatalogSuite`:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly org.apache.spark.sql.internal.CatalogSuite"
```

Closes #31403 from MaxGekk/catalogimpl-recoverPartitions.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-01 14:09:40 +00:00
Max Gekk 0837c1aa3d [SPARK-34303][SQL] Migrate ALTER TABLE .. SET LOCATION to new resolution framework
### What changes were proposed in this pull request?
1. Remove old statement `AlterTableSetLocationStatement`
2. Introduce new command `AlterTableSetLocation` for  `ALTER TABLE .. SET LOCATION`.

### Why are the changes needed?
This is a part of effort to make the relation lookup behavior consistent: SPARK-29900.

### Does this PR introduce _any_ user-facing change?
It can change the error message for views.

### How was this patch tested?
By running `ALTER TABLE .. SET LOCATION` tests:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *DataSourceV2SQLSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31414 from MaxGekk/migrate-set-location-resolv-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-01 13:41:15 +00:00
Max Gekk 95302756f1 [SPARK-34266][SQL][DOCS] Update comments for SessionCatalog.refreshTable() and CatalogImpl.refreshTable()
### What changes were proposed in this pull request?
Describe `SessionCatalog.refreshTable()` and `CatalogImpl.refreshTable()`. what they do and when they are supposed to be used.

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

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

### How was this patch tested?
By running `./dev/scalastyle`

Closes #31364 from MaxGekk/doc-refreshTable.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-01 13:07:05 +00:00
HyukjinKwon 4e7e7ee6e5 [SPARK-33245][SQL][FOLLOW-UP] Remove bitwiseGet in Scala functions API
### What changes were proposed in this pull request?

This PR is a followup that removes `bitwiseGet` in functions API. This is mainly for SQL compliance, and arguably not very much commonly used.

### Why are the changes needed?

See https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L41-L59

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

No, it's a change in unreleased branches.

### How was this patch tested?

Existing tests should cover.

Closes #31410 from HyukjinKwon/SPARK-33245.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-01 18:21:27 +09:00
Terry Kim a8eb443bf8 [SPARK-34299][SQL] Clean up ResolveSessionCatalog's isTempView and isTempFunction
### What changes were proposed in this pull request?

`ResolveSessionCatalog`'s `isTempView` and `isTempFunction` are not being used anymore since the resolution of temp view/function has moved to `Analyzer`.

This PR proposes to remove `isTempView` and `isTempFunction` from `ResolveSessionCatalog`.

### Why are the changes needed?

To clean up unused variables.

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

No

### How was this patch tested?

Existing tests should cover as this PR just removes the unused variables.

Closes #31400 from imback82/cleanup_resolve_session_catalog.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-31 13:03:30 +09:00
Terry Kim bec88a66bd [SPARK-34269][SQL][TESTS][FOLLOWUP] Test a subquery with view in aggregate's grouping expression
### What changes were proposed in this pull request?

This PR is a follow-up to #31368 to add a test case that has a subquery with "view" in aggregate's grouping expression. The existing test tests a subquery with dataframe's temp view, so it doesn't contain a `View` node.

### Why are the changes needed?

To increase the test coverage.

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

No

### How was this patch tested?

Added a new test.

Closes #31352 from imback82/grouping_expr.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-30 17:07:40 -08:00
Chao Sun 463d4ec350 [SPARK-34269][SQL][TESTS][FOLLOWUP] Add test cases for cache lookup and project removal
### What changes were proposed in this pull request?

This adds a few test cases for looking up cached temporary/permanent view created using clauses such as `ORDER BY` or `LIMIT`.

### Why are the changes needed?

Due to `EliminateView` and how canonization is done for `View`, which inserts an extra project operator, cache lookup could fail in the following simple example:
```sql
> CREATE TABLE t (key bigint, value string) USING parquet
> CACHE TABLE v1 AS SELECT * FROM t ORDER BY key
> SELECT * FROM t ORDER BY key
```

#31368 addresses this issue by removing the project operator if `canRemoveProject` check is successful. This PR adds a few tests.

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

NO

### How was this patch tested?

This PR just adds unit tests.

Closes #31182 from sunchao/SPARK-34108.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-01-30 12:31:57 -08:00
Liang-Chi Hsieh 50d14c98c3 [SPARK-34270][SS] Combine StateStoreMetrics should not override StateStoreCustomMetric
### What changes were proposed in this pull request?

This patch proposes to sum up custom metric values instead of taking arbitrary one when combining `StateStoreMetrics`.

### Why are the changes needed?

For stateful join in structured streaming, we need to combine `StateStoreMetrics` from both left and right side. Currently we simply take arbitrary one from custom metrics with same name from left and right. By doing this we miss half of metric number.

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

Yes, this corrects metrics collected for stateful join.

### How was this patch tested?

Unit test.

Closes #31369 from viirya/SPARK-34270.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-29 20:50:39 -08:00
Yuming Wang f2b22d1487 [SPARK-34289][SQL] Parquet vectorized reader support column index
### What changes were proposed in this pull request?

This pr make parquet vectorized reader support [column index](https://issues.apache.org/jira/browse/PARQUET-1201).

### Why are the changes needed?

Improve filter performance. for example: `id = 1`, we only need to read `page-0` in `block 1`:

```
block 1:
                     null count  min                                       max
page-0                         0  0                                         99
page-1                         0  100                                       199
page-2                         0  200                                       299
page-3                         0  300                                       399
page-4                         0  400                                       449

block 2:
                     null count  min                                       max
page-0                         0  450                                       549
page-1                         0  550                                       649
page-2                         0  650                                       749
page-3                         0  750                                       849
page-4                         0  850                                       899
```

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

No.

### How was this patch tested?

Unit test and benchmark: https://github.com/apache/spark/pull/31393#issuecomment-769767724

Closes #31393 from wangyum/SPARK-34289.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-29 09:53:46 -08:00
Max Gekk 588ddcdf22 [SPARK-33163][SQL][TESTS][FOLLOWUP] Fix the test for the parquet metadata key 'org.apache.spark.legacyDateTime'
### What changes were proposed in this pull request?
1. Test both date and timestamp column types
2. Write the timestamp as the `TIMESTAMP_MICROS` logical type
3. Change the timestamp value to `'1000-01-01 01:02:03'` to check exception throwing.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt "testOnly org.apache.spark.sql.execution.datasources.parquet.ParquetIOSuite"
```

Closes #31396 from MaxGekk/parquet-test-metakey-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-29 22:25:01 +09:00
beliefer 0f7a4977c9 [SPARK-33601][SQL] Group exception messages in catalyst/parser
### What changes were proposed in this pull request?
This PR group exception messages in `/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31293 from beliefer/SPARK-33601.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-29 08:57:58 +00:00
Chircu 520e5d2ab8 [SPARK-34144][SQL] Exception thrown when trying to write LocalDate and Instant values to a JDBC relation
### What changes were proposed in this pull request?

When writing rows to a table only the old date time API types are handled in org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils#makeSetter. If the new API is used (spark.sql.datetime.java8API.enabled=true) casting Instant and LocalDate to Timestamp and Date respectively fails. The proposed change is to handle Instant and LocalDate values and transform them to Timestamp and Date.

### Why are the changes needed?

In the current state writing Instant or LocalDate values to a table fails with something like:
Caused by: java.lang.ClassCastException: class java.time.LocalDate cannot be cast to class java.sql.Date (java.time.LocalDate is in module java.base of loader 'bootstrap'; java.sql.Date is in module java.sql of loader 'platform') at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$makeSetter$11(JdbcUtils.scala:573) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$makeSetter$11$adapted(JdbcUtils.scala:572) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.savePartition(JdbcUtils.scala:678) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$saveTable$1(JdbcUtils.scala:858) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$saveTable$1$adapted(JdbcUtils.scala:856) at org.apache.spark.rdd.RDD.$anonfun$foreachPartition$2(RDD.scala:994) at org.apache.spark.rdd.RDD.$anonfun$foreachPartition$2$adapted(RDD.scala:994) at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2139) 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:446) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449) ... 3 more

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

No

### How was this patch tested?

Added tests

Closes #31264 from cristichircu/SPARK-34144.

Lead-authored-by: Chircu <chircu@arezzosky.com>
Co-authored-by: Cristi Chircu <cristian.chircu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-29 17:48:13 +09:00
Bo Zhang 3f350dbd78 [SPARK-33212][FOLLOW-UP][BUILD] Fix test "built-in Hadoop version should support shaded client" for hadoop-2.7
### What changes were proposed in this pull request?
We added test "built-in Hadoop version should support shaded client" in https://github.com/apache/spark/pull/31203, but it fails when profile hadoop-2.7 is activated. This change fixes the test by skipping the assertion when Hadoop version is 2.

### Why are the changes needed?
The test fails in master branch when profile hadoop-2.7 is activated.

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

### How was this patch tested?
Ran the test with hadoop-2.7 profile.

Closes #31391 from bozhang2820/fix-hadoop-2-version-test.

Authored-by: Bo Zhang <bo.zhang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-29 15:47:02 +09:00
Wenchen Fan b891862fb6 [SPARK-34269][SQL] Simplify SQL view resolution
### What changes were proposed in this pull request?

The currently SQL (temp or permanent) view resolution is done in 2 steps:
1. In `SessionCatalog`, we get the view metadata, parse the view SQL string, and wrap it with `View`.
2. At the beginning of the optimizer, we run `EliminateView`, which drops the wrapper `View`, and apply some special logic to match the view schema.

Step 2 is tricky, as we need to retain the output attr expr id, while we need to add an extra `Project` to add cast and alias. This PR simplifies the view solution by building a completed plan (with cast and alias added) in `SessionCatalog`, so that we only have 1 step.

### Why are the changes needed?

Code simplification. It also fixes issues like https://github.com/apache/spark/pull/31352

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

No

### How was this patch tested?

existing tests

Closes #31368 from cloud-fan/try.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-29 06:46:01 +00:00
Cheng Su d871b54a4e [SPARK-34237][SQL] Add more metrics (fallback, spill) to object hash aggregate
### What changes were proposed in this pull request?

This PR is to add two more metrics for `ObjectHashAggregateExec`, i.e. the spill size, and number of fallback to sort-based aggregation.

### Why are the changes needed?

As object hash aggregate fallback mechanism is special - it will fallback to sort-based aggregation based on number of keys seen so far [0]. This fallback logic sometimes is sub-optimal and leads to unnecessary sort, and performance degradation in run-time. The first step to help user/developer debug is to add more related metrics on UI, e.g. spill size, and number of fallback to sort-based aggregation. (spill size metrics was already added for hash aggregate [1])

[0]: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala#L161

[1]: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L68

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

Added two more metrics on Spark UI for operator `ObjectHashAggregateExec`. Screenshot is attached below.

### How was this patch tested?

* Added unit test in `SQLMetricsSuite.scala`.
* Tested on spark shell locally and verified the metrics shown up on UI.

<img width="399" alt="Screen Shot 2021-01-28 at 1 44 40 PM" src="https://user-images.githubusercontent.com/4629931/106204224-7a8a1300-6171-11eb-9814-c3432abadc29.png">

Closes #31340 from c21/object-hash.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-29 04:35:58 +00:00
ulysses-you 72b7f8abfb [SPARK-34261][SQL] Avoid side effect if create exists temporary function
### What changes were proposed in this pull request?

Add function exists check before load resource.

### Why are the changes needed?

We should not add jar into classpath if the create temporary function is already exists.

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

No.

### How was this patch tested?

Add test.

Closes #31358 from ulysses-you/SPARK-34261.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-01-29 10:39:02 +09:00
Yuming Wang a7683afdf4 [SPARK-26346][BUILD][SQL] Upgrade Parquet to 1.11.1
### What changes were proposed in this pull request?

This PR upgrade Parquet to 1.11.1.

Parquet 1.11.1 new features:

- [PARQUET-1201](https://issues.apache.org/jira/browse/PARQUET-1201) - Column indexes
- [PARQUET-1253](https://issues.apache.org/jira/browse/PARQUET-1253) - Support for new logical type representation
- [PARQUET-1388](https://issues.apache.org/jira/browse/PARQUET-1388) - Nanosecond precision time and timestamp - parquet-mr

More details:
https://github.com/apache/parquet-mr/blob/apache-parquet-1.11.1/CHANGES.md

### Why are the changes needed?
Support column indexes to improve query performance.

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

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

Closes #26804 from wangyum/SPARK-26346.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-01-29 08:07:49 +08:00
Cheng Su 3a361cd837 [SPARK-34253][SQL] Object hash aggregate should not fallback if no more input rows
### What changes were proposed in this pull request?

Object hash aggregate will fallback to sort-based aggregation based on number of keys seen so far [0]. The default config threshold is 128 (spark.sql.objectHashAggregate.sortBased.fallbackThreshold in [1]). There's an edge case we can do better, where we do not fallback if there's no more input rows. Suppose the task only has 128 group-by keys in hash ma, we don't need to fallback in this case, and we can save the extra sort. This is an rare edge case in production, but it can happen.

[0]: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala#L161

[1]: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L1615

### Why are the changes needed?

To avoid unnecessary sort in query. Save resource.

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

No.

### How was this patch tested?

Add a unit test to verify task fallback or not is challenging. Given the change is pretty minor, besides relying on existing test in `ObjectHashAggregateSuite.scala`, I manually ran the followed query, and verified in debug mode that the code path for fallback was not executed. And verified the code path for fallback was executed without this change.

```
withSQLConf(
  SQLConf.USE_OBJECT_HASH_AGG.key -> "true",
  SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "1") {
  Seq.fill(1)(Tuple1(Array.empty[Int]))
    .toDF("c0")
    .groupBy(lit(1))
    .agg(typed_count($"c0"), max($"c0")).collect()
}
```

Closes #31353 from c21/object-hash-fallback.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-28 15:18:54 +00:00
MrPowers 9ed0e3cebf [SPARK-34165][SQL] Add count_distinct as an option to Dataset#summary
### What changes were proposed in this pull request?

Add `count_distinct` as an option argument to Dataset#summary (the method already supports count, min, max, etc.)

### Why are the changes needed?

The `summary()` method is used for lightweight exploratory data analysis.  A distinct count of all the columns is one of the most common exploratory data analysis queries.

Distinct counts can be expensive, so this shouldn't be enabled by default.  The proposed implementation is completely backwards compatible.

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

Yes, users can now call `df.summary("count_distinct")`, which wasn't an option before.  Users can still call `df.summary()` without any arguments and the output is the same.  `count_distinct` was not added as one of the `defaultStatistics`.

### How was this patch tested?

Unit tests.

### Additional comments

If this idea is accepted, we should add a PySpark implementation in this PR, as suggested by zero323.

Closes #31254 from MrPowers/SPARK-34165.

Authored-by: MrPowers <matthewkevinpowers@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-28 08:38:01 -06:00
yangjie01 15445a8d9e [SPARK-34275][CORE][SQL][MLLIB] Replaces filter and size with count
### What changes were proposed in this pull request?
Use `count` to simplify `find + size(or length)` operation, it's semantically consistent, but looks simpler.

**Before**
```
seq.filter(p).size
```

**After**
```
seq.count(p)
```

### Why are the changes needed?
Code Simpilefications.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31374 from LuciferYang/SPARK-34275.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 15:27:07 +09:00
Max Gekk d242166b8f [SPARK-34262][SQL] Refresh cached data of v1 table in ALTER TABLE .. SET LOCATION
### What changes were proposed in this pull request?
Invoke `CatalogImpl.refreshTable()` in v1 implementation of the `ALTER TABLE .. SET LOCATION` command to refresh cached table data.

### Why are the changes needed?
The example below portraits the issue:

- Create a source table:
```sql
spark-sql> CREATE TABLE src_tbl (c0 int, part int) USING hive PARTITIONED BY (part);
spark-sql> INSERT INTO src_tbl PARTITION (part=0) SELECT 0;
spark-sql> SHOW TABLE EXTENDED LIKE 'src_tbl' PARTITION (part=0);
default	src_tbl	false	Partition Values: [part=0]
Location: file:/Users/maximgekk/proj/refresh-cache-set-location/spark-warehouse/src_tbl/part=0
...
```
- Set new location for the empty partition (part=0):
```sql
spark-sql> CREATE TABLE dst_tbl (c0 int, part int) USING hive PARTITIONED BY (part);
spark-sql> ALTER TABLE dst_tbl ADD PARTITION (part=0);
spark-sql> INSERT INTO dst_tbl PARTITION (part=1) SELECT 1;
spark-sql> CACHE TABLE dst_tbl;
spark-sql> SELECT * FROM dst_tbl;
1	1
spark-sql> ALTER TABLE dst_tbl PARTITION (part=0) SET LOCATION '/Users/maximgekk/proj/refresh-cache-set-location/spark-warehouse/src_tbl/part=0';
spark-sql> SELECT * FROM dst_tbl;
1	1
```
The last query does not return new loaded data.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above works correctly:
```sql
spark-sql> ALTER TABLE dst_tbl PARTITION (part=0) SET LOCATION '/Users/maximgekk/proj/refresh-cache-set-location/spark-warehouse/src_tbl/part=0';
spark-sql> SELECT * FROM dst_tbl;
0	0
1	1
```

### How was this patch tested?
Added new test to `org.apache.spark.sql.hive.CachedTableSuite`:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *CachedTableSuite"
```

Closes #31361 from MaxGekk/refresh-cache-set-location.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 15:05:22 +09:00
beliefer b12e9a4ea6 [SPARK-33542][SQL][FOLLOWUP] Group exception messages in catalyst/catalog
### What changes were proposed in this pull request?
This PR follows up https://github.com/apache/spark/pull/30870.
Maybe some contributors don't know the job and added some exception by the old way.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31312 from beliefer/SPARK-33542-followup.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-28 05:15:57 +00:00
Angerszhuuuu 850990f40e [SPARK-34238][SQL] Unify output of SHOW PARTITIONS and pass output attributes properly
### What changes were proposed in this pull request?
Passing around the output attributes should have more benefits like keeping the expr ID unchanged to avoid bugs when we apply more operators above the command output dataframe.

This PR keep SHOW PARTITIONS command's output attribute exprId unchanged.
And benefit for https://issues.apache.org/jira/browse/SPARK-34238
### Why are the changes needed?
 Keep SHOW PARTITIONS command's output attribute exprid unchanged.

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

### How was this patch tested?
Added UT

Closes #31341 from AngersZhuuuu/SPARK-34238.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-28 05:13:19 +00:00
Yuming Wang 01d11da84e [SPARK-34268][SQL][DOCS] Correct the documentation of the concat_ws function
### What changes were proposed in this pull request?

This pr correct the documentation of the `concat_ws` function.

### Why are the changes needed?

`concat_ws` doesn't need any str or array(str) arguments:
```
scala> sql("""select concat_ws("s")""").show
+------------+
|concat_ws(s)|
+------------+
|            |
+------------+
```

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

No.

### How was this patch tested?

```
 build/sbt  "sql/testOnly *.ExpressionInfoSuite"
```

Closes #31370 from wangyum/SPARK-34268.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 14:06:36 +09:00
Chao Sun 6ec3cf6219 [SPARK-34271][SQL] Use majorMinorPatchVersion for Hive version parsing
### What changes were proposed in this pull request?

Use `majorMinorPatchVersion` to check major & minor version in `IsolatedClientLoader.hiveVersion`.

### Why are the changes needed?

Currently `IsolatedClientLoader.hiveVersion` needs to enumerate all Hive patch versions. Therefore, whenever we upgrade Hive version we'd need to remember to update the method as well. It would be better if we just check major & minor version.

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

No.

### How was this patch tested?

This is a refactoring and relies on existing tests.

Closes #31371 from sunchao/replace-hive-version.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 14:00:10 +09:00
Linhong Liu cf1400c8dd [SPARK-34260][SQL] Fix UnresolvedException when creating temp view twice
### What changes were proposed in this pull request?
In PR #30140, it will compare new and old plans when replacing view and uncache data
if the view has changed. But the compared new plan is not analyzed which will cause
`UnresolvedException` when calling `sameResult`. So in this PR, we use the analyzed
plan to compare to fix this problem.

### Why are the changes needed?
bug fix

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

### How was this patch tested?
newly added tests

Closes #31360 from linhongliu-db/SPARK-34260.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-27 20:59:23 -08:00
Chircu 829f118f98 [SPARK-33867][SQL] Instant and LocalDate values aren't handled when generating SQL queries
### What changes were proposed in this pull request?

When generating SQL queries only the old date time API types are handled for values in org.apache.spark.sql.jdbc.JdbcDialect#compileValue. If the new API is used (spark.sql.datetime.java8API.enabled=true) Instant and LocalDate values are not quoted and errors are thrown. The change proposed is to handle Instant and LocalDate values the same way that Timestamp and Date are.

### Why are the changes needed?

In the current state if an Instant is used in a filter, an exception will be thrown.
Ex (dataset was read from PostgreSQL): dataset.filter(current_timestamp().gt(col(VALID_FROM)))
Stacktrace (the T11 is from an instant formatted like yyyy-MM-dd'T'HH:mm:ss.SSSSSS'Z'):
Caused by: org.postgresql.util.PSQLException: ERROR: syntax error at or near "T11"Caused by: org.postgresql.util.PSQLException: ERROR: syntax error at or near "T11"  Position: 285 at org.postgresql.core.v3.QueryExecutorImpl.receiveErrorResponse(QueryExecutorImpl.java:2103) at org.postgresql.core.v3.QueryExecutorImpl.processResults(QueryExecutorImpl.java:1836) at org.postgresql.core.v3.QueryExecutorImpl.execute(QueryExecutorImpl.java:257) at org.postgresql.jdbc2.AbstractJdbc2Statement.execute(AbstractJdbc2Statement.java:512) at org.postgresql.jdbc2.AbstractJdbc2Statement.executeWithFlags(AbstractJdbc2Statement.java:388) at org.postgresql.jdbc2.AbstractJdbc2Statement.executeQuery(AbstractJdbc2Statement.java:273) at org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD.compute(JDBCRDD.scala:304) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) at org.apache.spark.scheduler.Task.run(Task.scala:127) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) at java.base/java.lang.Thread.run(Thread.java:834)

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

No

### How was this patch tested?

Test added

Closes #31148 from cristichircu/SPARK-33867.

Lead-authored-by: Chircu <chircu@arezzosky.com>
Co-authored-by: Cristi Chircu <chircu@arezzosky.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-01-28 11:58:20 +09:00
Max Gekk 1318be7ee9 [SPARK-34267][SQL] Remove refreshTable() from SessionState
### What changes were proposed in this pull request?
Remove `SessionState.refreshTable()` and modify the tests where the method is used.

### Why are the changes needed?
There are already 2 methods with the same name in:
- `SessionCatalog`
- `CatalogImpl`

One more method in `SessionState` does not give any benefits. By removing it, we can improve code maintenance.

### Does this PR introduce _any_ user-facing change?
Should not because `SessionState` is an internal class.

### How was this patch tested?
By running the modified test suites:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *MetastoreDataSourcesSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *HiveOrcQuerySuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *HiveParquetMetastoreSuite"
```

Closes #31366 from MaxGekk/remove-refreshTable-from-SessionState.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-27 09:43:59 -08:00
Wenchen Fan 2dbb7d5af8 [SPARK-34212][SQL][FOLLOWUP] Refine the behavior of reading parquet non-decimal fields as decimal
### What changes were proposed in this pull request?

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

When reading parquet int/long as decimal, the behavior should be the same as reading int/long and then cast to the decimal type. This PR changes to the expected behavior.

When reading parquet binary as decimal, we don't really know how to interpret the binary (it may from a string), and should fail. This PR changes to the expected behavior.

### Why are the changes needed?

To make the behavior more sane.

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

Yes, but it's a followup.

### How was this patch tested?

updated test

Closes #31357 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-27 09:34:31 -08:00
Kent Yao 5718d64f31 [SPARK-34083][SQL] Using TPCDS original definitions for char/varchar columns
### What changes were proposed in this pull request?

This PR changes the column types in the table definitions of `TPCDSBase` from string to char and varchar, with respect to the original definitions for char/varchar columns in the official doc - [TPC-DS_v2.9.0](http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf).

### Why are the changes needed?

Comply with both TPCDS standard and ANSI, and using string will get wrong results with those TPCDS queries

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

no

### How was this patch tested?

plan stability check

Closes #31012 from yaooqinn/tpcds.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-27 17:51:49 +08:00
Max Gekk 0d08e22bc7 [SPARK-34251][SQL] Fix table stats calculation by TRUNCATE TABLE
### What changes were proposed in this pull request?
1. Take into account the SQL config `spark.sql.statistics.size.autoUpdate.enabled` in the `TRUNCATE TABLE` command as other commands do.
2. Re-calculate actual table size in fs. Before the changes, `TRUNCATE TABLE` always sets table size to 0 in stats.

### Why are the changes needed?
This fixes the bug that is demonstrated by the example:
1. Create a partitioned table with 2 non-empty partitions:
```sql
spark-sql> CREATE TABLE tbl (c0 int, part int) PARTITIONED BY (part);
spark-sql> INSERT INTO tbl PARTITION (part=0) SELECT 0;
spark-sql> INSERT INTO tbl PARTITION (part=1) SELECT 1;
spark-sql> ANALYZE TABLE tbl COMPUTE STATISTICS;
spark-sql> DESCRIBE TABLE EXTENDED tbl;
...
Statistics	4 bytes, 2 rows
...
```
2. Truncate only one partition:
```sql
spark-sql> TRUNCATE TABLE tbl PARTITION (part=1);
spark-sql> SELECT * FROM tbl;
0	0
```
3. The table is still non-empty but `TRUNCATE TABLE` reseted stats:
```
spark-sql> DESCRIBE TABLE EXTENDED tbl;
...
Statistics	0 bytes, 0 rows
...
```

### Does this PR introduce _any_ user-facing change?
It could impact on performance of following queries.

### How was this patch tested?
Added new test to `StatisticsCollectionSuite`:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *StatisticsCollectionSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *StatisticsSuite"
```

Closes #31350 from MaxGekk/fix-stats-in-trunc-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-27 07:02:04 +00:00
Kent Yao 764582c07a [SPARK-34233][SQL] FIX NPE for char padding in binary comparison
### What changes were proposed in this pull request?

we need to check whether the `lit` is null  before calling `numChars`

### Why are the changes needed?

fix an obvious NPE bug

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

no

### How was this patch tested?

new tests

Closes #31336 from yaooqinn/SPARK-34233.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-27 14:59:53 +08:00
Kent Yao 91ca21d700 [SPARK-34236][SQL] Fix v2 Overwrite w/ null static partition raise Cannot translate expression to source filter: null
### What changes were proposed in this pull request?

For v2 static partitions overwriting, we use `EqualTo ` to generate the `deleteExpr`

This is not right for null partition values, and cause the problem like below because `ConstantFolding` converts it to lit(null)

```scala
SPARK-34223: static partition with null raise NPE *** FAILED *** (19 milliseconds)
[info]   org.apache.spark.sql.AnalysisException: Cannot translate expression to source filter: null
[info]   at org.apache.spark.sql.execution.datasources.v2.V2Writes$$anonfun$apply$1.$anonfun$applyOrElse$1(V2Writes.scala:50)
[info]   at scala.collection.immutable.List.flatMap(List.scala:366)
[info]   at org.apache.spark.sql.execution.datasources.v2.V2Writes$$anonfun$apply$1.applyOrElse(V2Writes.scala:47)
[info]   at org.apache.spark.sql.execution.datasources.v2.V2Writes$$anonfun$apply$1.applyOrElse(V2Writes.scala:39)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:317)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:73)
```

The right way is to use EqualNullSafe instead to delete the null partitions.

### Why are the changes needed?

bugfix

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

no
### How was this patch tested?

an original test to new place

Closes #31339 from yaooqinn/SPARK-34236.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-27 12:05:50 +08:00
Chao Sun abf7e81712 [SPARK-33212][FOLLOW-UP][BUILD] Bring back duplicate dependency check and add more strict Hadoop version check
### What changes were proposed in this pull request?

1. Add back Maven enforcer for duplicate dependencies check
2. More strict check on Hadoop versions which support shaded client in `IsolatedClientLoader`. To do proper version check, this adds a util function `majorMinorPatchVersion` to extract major/minor/patch version from a string.
3. Cleanup unnecessary code

### Why are the changes needed?

The Maven enforcer was removed as part of #30556. This proposes to add it back.

Also, Hadoop shaded client doesn't work in certain cases (see [these comments](https://github.com/apache/spark/pull/30701#discussion_r558522227) for details). This strictly checks that the current Hadoop version (i.e., 3.2.2 at the moment) has good support of shaded client or otherwise fallback to old unshaded ones.

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

No.

### How was this patch tested?

Existing tests.

Closes #31203 from sunchao/SPARK-33212-followup.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-26 15:34:55 -08:00
Dongjoon Hyun dbf051c50a [SPARK-34212][SQL] Fix incorrect decimal reading from Parquet files
### What changes were proposed in this pull request?

This PR aims to the correctness issues during reading decimal values from Parquet files.
- For **MR** code path, `ParquetRowConverter` can read Parquet's decimal values with the original precision and scale written in the corresponding footer.
- For **Vectorized** code path, `VectorizedColumnReader` throws `SchemaColumnConvertNotSupportedException`.

### Why are the changes needed?

Currently, Spark returns incorrect results when the Parquet file's decimal precision and scale are different from the Spark's schema. This happens when there is multiple files with different decimal schema or HiveMetastore has a new schema.

**BEFORE (Simplified example for correctness)**

```scala
scala> sql("SELECT 1.0 a").write.parquet("/tmp/decimal")
scala> spark.read.schema("a DECIMAL(3,2)").parquet("/tmp/decimal").show
+----+
|   a|
+----+
|0.10|
+----+
```

This works correctly in the other data sources, `ORC/JSON/CSV`, like the following.
```scala
scala> sql("SELECT 1.0 a").write.orc("/tmp/decimal_orc")
scala> spark.read.schema("a DECIMAL(3,2)").orc("/tmp/decimal_orc").show
+----+
|   a|
+----+
|1.00|
+----+
```

**AFTER**
1. **Vectorized** path: Instead of incorrect result, we will raise an explicit exception.
```scala
scala> spark.read.schema("a DECIMAL(3,2)").parquet("/tmp/decimal").show
java.lang.UnsupportedOperationException: Schema evolution not supported.
```

2. **MR** path (complex schema or explicit configuration): Spark returns correct results.
```scala
scala> spark.read.schema("a DECIMAL(3,2), b DECIMAL(18, 3), c MAP<INT,INT>").parquet("/tmp/decimal").show
+----+-------+--------+
|   a|      b|       c|
+----+-------+--------+
|1.00|100.000|{1 -> 2}|
+----+-------+--------+

scala> spark.read.schema("a DECIMAL(3,2), b DECIMAL(18, 3), c MAP<INT,INT>").parquet("/tmp/decimal").printSchema
root
 |-- a: decimal(3,2) (nullable = true)
 |-- b: decimal(18,3) (nullable = true)
 |-- c: map (nullable = true)
 |    |-- key: integer
 |    |-- value: integer (valueContainsNull = true)
```

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

Yes. This fixes the correctness issue.

### How was this patch tested?

Pass with the newly added test case.

Closes #31319 from dongjoon-hyun/SPARK-34212.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-26 15:13:39 -08:00
beliefer 99b6af2dd2 [SPARK-34244][SQL] Remove the Scala function version of regexp_extract_all
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/27507 implements `regexp_extract_all` and added the scala function version of it.
According https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L41-L59, it seems good for remove the scala function version. Although I think is regexp_extract_all is very useful, if we just reference the description.

### Why are the changes needed?
`regexp_extract_all` is less common.

### Does this PR introduce _any_ user-facing change?
'No'. `regexp_extract_all` was added in Spark 3.1.0 which isn't released yet.

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

Closes #31346 from beliefer/SPARK-24884-followup.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-26 13:52:51 -08:00
Max Gekk ac8307d75c [SPARK-34215][SQL] Keep tables cached after truncation
### What changes were proposed in this pull request?
Invoke `CatalogImpl.refreshTable()` instead of combination of `SessionCatalog.refreshTable()` + `uncacheQuery()`. This allows to clear cached table data while keeping the table cached.

### Why are the changes needed?
1. To improve user experience with Spark SQL
2. To be consistent to other commands, see https://github.com/apache/spark/pull/31206

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

Before:
```scala
scala> sql("CREATE TABLE tbl (c0 int)")
res1: org.apache.spark.sql.DataFrame = []
scala> sql("INSERT INTO tbl SELECT 0")
res2: org.apache.spark.sql.DataFrame = []
scala> sql("CACHE TABLE tbl")
res3: org.apache.spark.sql.DataFrame = []
scala> sql("SELECT * FROM tbl").show(false)
+---+
|c0 |
+---+
|0  |
+---+
scala> spark.catalog.isCached("tbl")
res5: Boolean = true
scala> sql("TRUNCATE TABLE tbl")
res6: org.apache.spark.sql.DataFrame = []
scala> spark.catalog.isCached("tbl")
res7: Boolean = false
```

After:
```scala
scala> sql("TRUNCATE TABLE tbl")
res6: org.apache.spark.sql.DataFrame = []
scala> spark.catalog.isCached("tbl")
res7: Boolean = true
```

### How was this patch tested?
Added new test to `CachedTableSuite`:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *CachedTableSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31308 from MaxGekk/truncate-table-cached.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 15:36:44 +00:00
Angerszhuuuu dd88eff820 [SPARK-34241][SQL] For DDL command plan, we should define producedAttributes as it's outputSet
### What changes were proposed in this pull request?

When write test about command,  when `checkAnswer`,
Always got error as below
```
[info]   AttributeSet(partition#607) was not empty The analyzed logical plan has missing inputs:
[info]   ShowPartitionsCommand `ns`.`tbl`, [partition#607] (QueryTest.scala:224)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
```

For Command DDL plan, we can define  `producedAttributes` as it's `outputSet` and it's reasonable

### Why are the changes needed?
Add default   `producedAttributes` for Command LogicalPlan

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

### How was this patch tested?
Not need

Closes #31342 from AngersZhuuuu/SPARK-34241.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 15:14:10 +00:00
Anton Okolnychyi 08679646fe [SPARK-34026][SQL] Inject repartition and sort nodes to satisfy required distribution and ordering
### What changes were proposed in this pull request?

This PR adds repartition and sort nodes to satisfy the required distribution and ordering introduced in SPARK-33779.

Note: This PR contains the final part of changes discussed in PR #29066.

### Why are the changes needed?

These changes are the next step as discussed in the [design doc](https://docs.google.com/document/d/1X0NsQSryvNmXBY9kcvfINeYyKC-AahZarUqg3nS1GQs/edit#) for SPARK-23889.

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

No.

### How was this patch tested?

This PR comes with a new test suite.

Closes #31083 from aokolnychyi/spark-34026.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 15:09:30 +00:00
yangjie01 8999e8805d [SPARK-34224][CORE][SQL][SS][DSTREAM][YARN][TEST][EXAMPLES] Ensure all resource opened by Source.fromXXX are closed
### What changes were proposed in this pull request?
Using a function like `.mkString` or `.getLines` directly on a `scala.io.Source` opened by `fromFile`, `fromURL`, `fromURI ` will leak the underlying file handle,  this pr use the `Utils.tryWithResource` method wrap the `BufferedSource` to ensure these `BufferedSource` closed.

### Why are the changes needed?
Avoid file handle leak.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31323 from LuciferYang/source-not-closed.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 19:06:37 +09:00
Wenchen Fan 8dee8a9b7c [SPARK-34227][SQL] WindowFunctionFrame should clear its states during preparation
### What changes were proposed in this pull request?

This PR fixed all `OffsetWindowFunctionFrameBase#prepare` implementations to reset the states, and also add more comments in `WindowFunctionFrame` classdoc to explain why we need to reset states during preparation: `WindowFunctionFrame` instances are reused to process multiple partitions.

### Why are the changes needed?

To fix a correctness bug caused by the new feature "window function with ignore nulls" in the master branch.

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

yes

### How was this patch tested?

new test

Closes #31325 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 08:50:14 +00:00
Yuanjian Li 0a1a029622 [SPARK-34235][SS] Make spark.sql.hive as a private package
### What changes were proposed in this pull request?
Follow the comment https://github.com/apache/spark/pull/31271#discussion_r562598983:

- Remove the API tag `Unstable` for `HiveSessionStateBuilder`
- Add document for spark.sql.hive package to emphasize it's a private package

### Why are the changes needed?
Follow the rule for a private package.

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

### How was this patch tested?
Doc change only.

Closes #31321 from xuanyuanking/SPARK-34185-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 17:13:11 +09:00
Angerszhuuuu 7bd4165c11 [SPARK-32852][SQL][FOLLOW_UP] Add notice about keep hive version consistence when config hive jars location
### What changes were proposed in this pull request?
Add notice about keep hive version consistence when config hive jars location

With PR #29881, if we don't keep hive version consistence. we will got below error.
```
Builtin jars can only be used when hive execution version == hive metastore version. Execution: 2.3.8 != Metastore: 1.2.1. Specify a valid path to the correct hive jars using spark.sql.hive.metastore.jars or change spark.sql.hive.metastore.version to 2.3.8.
```

![image](https://user-images.githubusercontent.com/46485123/105795169-512d8380-5fc7-11eb-97c3-0259a0d2aa58.png)

### Why are the changes needed?
Make config doc detail

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

### How was this patch tested?
Not need

Closes #31317 from AngersZhuuuu/SPARK-32852-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 13:40:20 +09:00
Kent Yao b3915ddd91 [SPARK-34223][SQL] FIX NPE for static partition with null in InsertIntoHadoopFsRelationCommand
### What changes were proposed in this pull request?

with a simple case, the null will be passed to InsertIntoHadoopFsRelationCommand blindly, we should avoid the npe
```scala
 test("NPE") {
    withTable("t") {
      sql(s"CREATE TABLE t(i STRING, c string) USING $format PARTITIONED BY (c)")
      sql("INSERT OVERWRITE t PARTITION (c=null) VALUES ('1')")
      checkAnswer(spark.table("t"), Row("1", null))
    }
  }
```
```logtalk
java.lang.NullPointerException
	at scala.collection.immutable.StringOps$.length(StringOps.scala:51)
	at scala.collection.immutable.StringOps.length(StringOps.scala:51)
	at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:35)
	at scala.collection.IndexedSeqOptimized.foreach
	at scala.collection.immutable.StringOps.foreach(StringOps.scala:33)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.escapePathName(ExternalCatalogUtils.scala:69)
	at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.orig-s0.0000030000-r30676-expand-or-complete(InsertIntoHadoopFsRelationCommand.scala:231)
```

### Why are the changes needed?

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

no
### How was this patch tested?

new tests

Closes #31320 from yaooqinn/SPARK-34223.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 12:05:58 +08:00
Kent Yao d1177b5230 [SPARK-34192][SQL] Move char padding to write side and remove length check on read side too
### What changes were proposed in this pull request?

On the read-side, the char length check and padding bring issues to CBO and predicate pushdown and other issues to the catalyst.

This PR reverts 6da5cdf1db  that added read side length check) so that we only do length check for the write side, and data sources/vendors are responsible to enforce the char/varchar constraints for data import operations like ADD PARTITION. It doesn't make sense for Spark to report errors on the read-side if the data is already dirty.

This PR also moves the char padding to the write-side, so that it 1) avoids read side issues like CBO and filter pushdown. 2) the data source can preserve char type semantic better even if it's read by systems other than Spark.

### Why are the changes needed?

fix perf regression when tables have char/varchar type columns

closes #31278
### Does this PR introduce _any_ user-facing change?

yes, spark will not raise error for oversized char/varchar values in read side
### How was this patch tested?

modified ut

the dropped read side benchmark
```
================================================================================================
Char Varchar Read Side Perf w/o Tailing Spaces
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 20:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 20                         1564           1573           9         63.9          15.6       1.0X
read char with length 20                           1532           1551          18         65.3          15.3       1.0X
read varchar with length 20                        1520           1531          13         65.8          15.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 40:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 40                         1573           1613          41         63.6          15.7       1.0X
read char with length 40                           1575           1577           2         63.5          15.7       1.0X
read varchar with length 40                        1568           1576          11         63.8          15.7       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 60:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 60                         1526           1540          23         65.5          15.3       1.0X
read char with length 60                           1514           1539          23         66.0          15.1       1.0X
read varchar with length 60                        1486           1497          10         67.3          14.9       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 80:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 80                         1531           1542          19         65.3          15.3       1.0X
read char with length 80                           1514           1529          15         66.0          15.1       1.0X
read varchar with length 80                        1524           1565          42         65.6          15.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 100:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 100                        1597           1623          25         62.6          16.0       1.0X
read char with length 100                          1499           1512          16         66.7          15.0       1.1X
read varchar with length 100                       1517           1524           8         65.9          15.2       1.1X

================================================================================================
Char Varchar Read Side Perf w/ Tailing Spaces
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 20:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 20                         1524           1526           1         65.6          15.2       1.0X
read char with length 20                           1532           1537           9         65.3          15.3       1.0X
read varchar with length 20                        1520           1532          15         65.8          15.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 40:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 40                         1556           1580          32         64.3          15.6       1.0X
read char with length 40                           1600           1611          17         62.5          16.0       1.0X
read varchar with length 40                        1648           1716          88         60.7          16.5       0.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 60:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 60                         1504           1524          20         66.5          15.0       1.0X
read char with length 60                           1509           1512           3         66.2          15.1       1.0X
read varchar with length 60                        1519           1535          21         65.8          15.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 80:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 80                         1640           1652          17         61.0          16.4       1.0X
read char with length 80                           1625           1666          35         61.5          16.3       1.0X
read varchar with length 80                        1590           1605          13         62.9          15.9       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 100:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 100                        1622           1628           5         61.6          16.2       1.0X
read char with length 100                          1614           1646          30         62.0          16.1       1.0X
read varchar with length 100                       1594           1606          11         62.7          15.9       1.0X
```

Closes #31281 from yaooqinn/SPARK-34192.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 02:08:35 +08:00
Max Gekk bfc0235013 [SPARK-34203][SQL] Convert null partition values to __HIVE_DEFAULT_PARTITION__ in v1 In-Memory catalog
### What changes were proposed in this pull request?
In the PR, I propose to convert `null` partition values to `"__HIVE_DEFAULT_PARTITION__"` before storing in the `In-Memory` catalog internally. Currently, the `In-Memory` catalog maintains null partitions as `"__HIVE_DEFAULT_PARTITION__"` in file system but as `null` values in memory that could cause some issues like in SPARK-34203.

### Why are the changes needed?
`InMemoryCatalog` stores partitions in the file system in the Hive compatible form, for instance, it converts the `null` partition value to `"__HIVE_DEFAULT_PARTITION__"` but at the same time it keeps null as is internally. That causes an issue demonstrated by the example below:
```
$ ./bin/spark-shell -c spark.sql.catalogImplementation=in-memory
```
```scala
scala> spark.conf.get("spark.sql.catalogImplementation")
res0: String = in-memory

scala> sql("CREATE TABLE tbl (col1 INT, p1 STRING) USING parquet PARTITIONED BY (p1)")
res1: org.apache.spark.sql.DataFrame = []

scala> sql("INSERT OVERWRITE TABLE tbl VALUES (0, null)")
res2: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE tbl DROP PARTITION (p1 = null)")
org.apache.spark.sql.catalyst.analysis.NoSuchPartitionsException: The following partitions not found in table 'tbl' database 'default':
Map(p1 -> null)
  at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.dropPartitions(InMemoryCatalog.scala:440)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, `ALTER TABLE .. DROP PARTITION` can drop the `null` partition in `In-Memory` catalog:
```scala
scala> spark.table("tbl").show(false)
+----+----+
|col1|p1  |
+----+----+
|0   |null|
+----+----+

scala> sql("ALTER TABLE tbl DROP PARTITION (p1 = null)")
res4: org.apache.spark.sql.DataFrame = []

scala> spark.table("tbl").show(false)
+----+---+
|col1|p1 |
+----+---+
+----+---+
```

### How was this patch tested?
Added new test to `AlterTableDropPartitionSuiteBase`:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```

Closes #31322 from MaxGekk/insert-overwrite-null-part.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 15:27:20 +00:00
Dereck Li 096b15fa12 [SPARK-34607][SQL][FOLLOWUP] Change Option[LogicalRelation] to LogicalRelation
### What changes were proposed in this pull request?
optimize: change Option[LogicalRelation] to LogicalRelation

### Why are the changes needed?
simplify code

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

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

Closes #31315 from monkeyboy123/spark-34067-follow-up.

Authored-by: Dereck Li <monkeyboy.ljh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 13:34:07 +00:00
Max Gekk 6fe5a8a2ae [SPARK-34197][SQL] SessionCatalog.refreshTable() should not invalidate the relation cache for temporary views
### What changes were proposed in this pull request?
Check the name passed to `SessionCatalog.refreshTable`, and if it belongs to a temporary view, do not invalidate the relation cache.

### Why are the changes needed?
When `SessionCatalog.refreshTable` refreshes a temporary or global temporary view, it should not invalidate an entry in the relation cache associated to a table with the same name.

### Does this PR introduce _any_ user-facing change?
Should not. The change might improve performance slightly.

### How was this patch tested?
By running new UT:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *SessionCatalogSuite"
```

Closes #31265 from MaxGekk/fix-session-catalog-refresh-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 07:37:24 +00:00
yliou 512cacf7c6 [SPARK-33726][SQL] Fix for Duplicate field names during Aggregation
### What changes were proposed in this pull request?
The `RowBasedKeyValueBatch` has two different implementations depending on whether the aggregation key and value uses only fixed length data types (`FixedLengthRowBasedKeyValueBatch`) or not (`VariableLengthRowBasedKeyValueBatch`).

Before this PR the decision about the used implementation was based on by accessing the schema fields by their name.
But if two fields has the same name and one with variable length and the other with fixed length type (and all the other fields are with fixed length types) a bad decision could be made.

When `FixedLengthRowBasedKeyValueBatch` is chosen but there is a variable length field then an aggregation function could calculate with invalid values. This case is illustrated by the example used in the unit test:

`with T as (select id as a, -id as x from range(3)),
        U as (select id as b, cast(id as string) as x from range(3))
select T.x, U.x, min(a) as ma, min(b) as mb from T join U on a=b group by U.x, T.x`
where the 'x' column in the left side of the join is a Long but on the right side is a String.

### Why are the changes needed?
Fixes the issue where duplicate field name aggregation has null values in the dataframe.

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

### How was this patch tested?
Added UT, tested manually on spark shell.

Closes #30788 from yliou/SPARK-33726.

Authored-by: yliou <yliou@berkeley.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-25 06:53:26 +00:00
Peter Toth 98ec6c27e3 [SPARK-34147][SQL][TEST] Keep table partitioning in TPCDSQueryBenchmak when CBO is enabled
### What changes were proposed in this pull request?
This PR keeps partitioning of input tables in TPCDSQueryBenchmark when `--cbo` option is enabled.

https://github.com/apache/spark/pull/31011 introduced the `--cbo` option but unfortunately in that mode the table partitioning of the input data is lost. This means that the results of CBO mode is very different to non CBO mode, one example is that Dynamic Partition Pruning doesn't kick in in CBO mode.

### Why are the changes needed?
To monitor performance changed with CBO enabled.

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

### How was this patch tested?
Manually checked.

Closes #31218 from peter-toth/SPARK-34147-keep-partitioning-in-tpcdsquerybenchmark.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-25 11:42:47 +09:00
Yuanjian Li 59cbacaddf [SPARK-34185][DOCS] Review and fix issues in API docs
### What changes were proposed in this pull request?
Compare the 3.1.1 API doc with the latest release version 3.0.1. Fix the following issues:
- Add missing `Since` annotation for new APIs
- Remove the leaking class/object in API doc

### Why are the changes needed?
Fix the issues in the Spark 3.1.1 release API docs.

### Does this PR introduce _any_ user-facing change?
Yes, API doc changes.

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

Closes #31271 from xuanyuanking/SPARK-34185.

Lead-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-25 11:38:20 +09:00
Takuya UESHIN 43fdd1271e [SPARK-33489][PYSPARK] Add NullType support for Arrow executions
### What changes were proposed in this pull request?

Adds `NullType` support for Arrow executions.

### Why are the changes needed?

As Arrow supports null type, we can convert `NullType` between PySpark and pandas with Arrow enabled.

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

Yes, if a user has a DataFrame including `NullType`, it will be able to convert with Arrow enabled.

### How was this patch tested?

Added tests.

Closes #31285 from ueshin/issues/SPARK-33489/arrow_nulltype.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-25 11:34:47 +09:00
Yuming Wang 4fce05d93f [SPARK-34155][SQL][TEST] Add partition columns for TPCDS tables
### What changes were proposed in this pull request?

This pr add partition columns for TPCDS tables. The partition column is consistent with the [TPCDSTables](https://github.com/databricks/spark-sql-perf/blob/master/src/main/scala/com/databricks/spark/sql/perf/tpcds/TPCDSTables.scala).

### Why are the changes needed?

Better track plan changes. For example, [this is the change](3fe1a93a40) after SPARK-34119.

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

No.

### How was this patch tested?

N/A

Closes #31243 from wangyum/SPARK-34155.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-24 13:08:26 +09:00
Max Gekk f8bf72ed5d [SPARK-34213][SQL] Refresh cached data of v1 table in LOAD DATA
### What changes were proposed in this pull request?
Invoke `CatalogImpl.refreshTable()` instead of `SessionCatalog.refreshTable` in v1 implementation of the `LOAD DATA` command. `SessionCatalog.refreshTable` just refreshes metadata comparing to `CatalogImpl.refreshTable()` which refreshes cached table data as well.

### Why are the changes needed?
The example below portraits the issue:

- Create a source table:
```sql
spark-sql> CREATE TABLE src_tbl (c0 int, part int) USING hive PARTITIONED BY (part);
spark-sql> INSERT INTO src_tbl PARTITION (part=0) SELECT 0;
spark-sql> SHOW TABLE EXTENDED LIKE 'src_tbl' PARTITION (part=0);
default	src_tbl	false	Partition Values: [part=0]
Location: file:/Users/maximgekk/proj/load-data-refresh-cache/spark-warehouse/src_tbl/part=0
...
```
- Load data from the source table to a cached destination table:
```sql
spark-sql> CREATE TABLE dst_tbl (c0 int, part int) USING hive PARTITIONED BY (part);
spark-sql> INSERT INTO dst_tbl PARTITION (part=1) SELECT 1;
spark-sql> CACHE TABLE dst_tbl;
spark-sql> SELECT * FROM dst_tbl;
1	1
spark-sql> LOAD DATA LOCAL INPATH '/Users/maximgekk/proj/load-data-refresh-cache/spark-warehouse/src_tbl/part=0' INTO TABLE dst_tbl PARTITION (part=0);
spark-sql> SELECT * FROM dst_tbl;
1	1
```
The last query does not return new loaded data.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above works correctly:
```sql
spark-sql> LOAD DATA LOCAL INPATH '/Users/maximgekk/proj/load-data-refresh-cache/spark-warehouse/src_tbl/part=0' INTO TABLE dst_tbl PARTITION (part=0);
spark-sql> SELECT * FROM dst_tbl;
0	0
1	1
```

### How was this patch tested?
Added new test to `org.apache.spark.sql.hive.CachedTableSuite`:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *CachedTableSuite"
```

Closes #31304 from MaxGekk/load-data-refresh-cache.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-23 15:49:10 -08:00
Max Gekk 0592503669 [SPARK-34207][SQL] Rename isTemporaryTable to isTempView in SessionCatalog
### What changes were proposed in this pull request?
Rename `SessionCatalog.isTemporaryTable()` to `SessionCatalog.isTempView()`.

### Why are the changes needed?
To improve code maintenance. Currently, there are two methods that do the same but have different names:
```scala
def isTempView(nameParts: Seq[String]): Boolean
```
and
```scala
def isTemporaryTable(name: TableIdentifier): Boolean
```

### Does this PR introduce _any_ user-facing change?
Should not since `SessionCatalog` is not public API.

### How was this patch tested?
By running the existing tests:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *SessionCatalogSuite"
```

Closes #31295 from MaxGekk/replace-isTemporaryTable-by-isTempView.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-23 08:16:11 -08:00
yangjie01 e48a8ad1a2 [SPARK-34202][SQL][TEST] Add ability to fetch spark release package from internal environment in HiveExternalCatalogVersionsSuite
### What changes were proposed in this pull request?
`HiveExternalCatalogVersionsSuite` can't run in orgs internal environment where access to outside internet is not allowed because `HiveExternalCatalogVersionsSuite` will download spark release package from internet.

Similar to SPARK-32998, this pr add 1 environment variables `SPARK_RELEASE_MIRROR` to let user can specify an accessible download address of spark release package and run `HiveExternalCatalogVersionsSuite`  in orgs internal environment.

### Why are the changes needed?
Let `HiveExternalCatalogVersionsSuite` can run in orgs internal environment without relying on external spark release download address.

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

### How was this patch tested?

- Pass the Jenkins or GitHub Action

- Manual test  with and without env variables set in internal environment can't access internet.

execute
```
mvn clean install -Dhadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -PhPhive -pl  sql/hive -am -DskipTests

mvn clean install -Dhadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -PhPhive -pl  sql/hive -DwildcardSuites=org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite -Dtest=none
```

**Without env**

```
HiveExternalCatalogVersionsSuite:
19:50:35.123 WARN org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite: Failed to download Spark 3.0.1 from https://archive.apache.org/dist/spark/spark-3.0.1/spark-3.0.1-bin-hadoop3.2.tgz: Network is unreachable (connect failed)
19:50:35.126 WARN org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite: Failed to download Spark 3.0.1 from https://dist.apache.org/repos/dist/release/spark/spark-3.0.1/spark-3.0.1-bin-hadoop3.2.tgz: Network is unreachable (connect failed)
org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite *** ABORTED ***
  Exception encountered when invoking run on a nested suite - Unable to download Spark 3.0.1 (HiveExternalCatalogVersionsSuite.scala:125)
Run completed in 2 seconds, 669 milliseconds.
Total number of tests run: 0
Suites: completed 1, aborted 1
Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0
```

**With env**

```
export SPARK_RELEASE_MIRROR=${spark-release.internal.com}/dist/release/
```

```
HiveExternalCatalogVersionsSuite
- backward compatibility
Run completed in 1 minute, 32 seconds.
Total number of tests run: 1
Suites: completed 2, aborted 0
Tests: succeeded 1, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #31294 from LuciferYang/SPARK-34202.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-23 08:02:52 -08:00
Wenchen Fan b8a6906627 [SPARK-34200][SQL] Ambiguous column reference should consider attribute availability
### What changes were proposed in this pull request?

This is a long-standing bug that exists since we have the ambiguous self-join check. A column reference is not ambiguous if it can only come from one join side (e.g. the other side has a project to only pick a few columns). An example is
```
Join(b#1 = 3)
  TableScan(t, [a#0, b#1])
  Project(a#2)
    TableScan(t, [a#2, b#3])
```
It's a self-join, but `b#1` is not ambiguous because it can't come from the right side, which only has column `a`.

### Why are the changes needed?

to not fail valid self-join queries.

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

yea as a bug fix

### How was this patch tested?

a new test

Closes #31287 from cloud-fan/self-join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-22 20:11:53 +09:00
Yu Zhong 2db0a954e3 [SPARK-33933][SQL] Materialize BroadcastQueryStage first to try to avoid broadcast timeout in AQE
### What changes were proposed in this pull request?
This PR is the same as https://github.com/apache/spark/pull/30998, but with a better UT.
In AdaptiveSparkPlanExec.getFinalPhysicalPlan, when newStages are generated, sort the new stages by class type to make sure BroadcastQueryState precede others.
This partial fix only grantee the start of materialization for BroadcastQueryStage is prior to others, but because the submission of collect job for broadcasting is run in another thread, the issue is not completely solved.

### Why are the changes needed?
When enable AQE, in getFinalPhysicalPlan, spark traversal the physical plan bottom up and create query stage for materialized part by createQueryStages and materialize those new created query stages to submit map stages or broadcasting. When ShuffleQueryStage are materializing before BroadcastQueryStage, the map stage(job) and broadcast job are submitted almost at the same time, but map stage will hold all the computing resources. If the map stage runs slow (when lots of data needs to process and the resource is limited), the broadcast job cannot be started(and finished) before spark.sql.broadcastTimeout, thus cause whole job failed (introduced in SPARK-31475).
The workaround to increase spark.sql.broadcastTimeout doesn't make sense and graceful, because the data to broadcast is very small.

The order of calling materialize can guarantee that the order of task to be scheduled in normal circumstances, but, the guarantee is not strict since the submit of broadcast job and shuffle map job are in different thread.
1. for broadcast job, call doPrepare() in main thread, and then start the real materialization in "broadcast-exchange-0" thread pool: calling getByteArrayRdd().collect() to submit collect job
2. for shuffle map job, call ShuffleExchangeExec.mapOutputStatisticsFuture() which call sparkContext.submitMapStage() directly in main thread to submit map stage

1 is trigger before 2, so in normal cases, the broadcast job will be submit first.
However, we can not control how fast the two thread runs, so the "broadcast-exchange-0" thread could run a little bit slower than main thread, result in map stage submit first. So there's still risk for the shuffle map job schedule earlier before broadcast job.

Since completely fix the issue is complex and might introduce major changes, we need more time to follow up. This partial fix is better than do nothing, it resolved most cases in SPARK-33933.

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

### How was this patch tested?
Add UT

Closes #31269 from zhongyu09/aqe-broadcast-partial-fix.

Authored-by: Yu Zhong <zhongyu8@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-22 07:22:53 +00:00
beliefer fec82c9504 [SPARK-33245][SQL] Add built-in UDF - GETBIT
### What changes were proposed in this pull request?
`GETBIT` is a bitwise expression function given an INTEGER value, returns the value of a bit at a specified position.
`GETBIT( <integer_expr>, <bit_position> )`

Examples
select getbit(11, 100), getbit(11, 3), getbit(11, 2), getbit(11, 1), getbit(11, 0);
GETBIT(11, 3) | GETBIT(11, 2) | GETBIT(11, 1) | GETBIT(11, 0)
-- | -- | -- | --
1 | 0 | 1 | 1

The mainstream database support this feature show below:

**Teradata**
https://docs.teradata.com/reader/kmuOwjp1zEYg98JsB8fu_A/PK1oV1b2jqvG~ohRnOro9w

**Impala**
https://docs.cloudera.com/runtime/7.2.0/impala-sql-reference/topics/impala-bit-functions.html#bit_functions__getbit

**Snowflake**
https://docs.snowflake.com/en/sql-reference/functions/getbit.html

**Yellowbrick**
https://www.yellowbrick.com/docs/2.2/ybd_sqlref/getbit.html

### Why are the changes needed?
GETBIT is very useful.

### Does this PR introduce _any_ user-facing change?
Yes. GETBIT is a new bitwise function.

### How was this patch tested?
Jenkins test

Closes #31198 from beliefer/SPARK-33245.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-22 04:57:39 +00:00
beliefer cde697a479 [SPARK-33541][SQL] Group exception messages in catalyst/expressions
### What changes were proposed in this pull request?
This PR group exception messages in `/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31228 from beliefer/SPARK-33541.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-22 04:52:05 +00:00
Kousuke Saruta 45f076336b [SPARK-33813][SQL] Fix the issue that JDBC source can't treat MS SQL Server's spatial types
### What changes were proposed in this pull request?

This PR fixes the issue that reading tables which contain spatial datatypes from MS SQL Server fails.
MS SQL server supports two non-standard spatial JDBC types, `geometry` and  `geography` but Spark SQL can't treat them

```
java.sql.SQLException: Unrecognized SQL type -157
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.getCatalystType(JdbcUtils.scala:251)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$getSchema$1(JdbcUtils.scala:321)
 at scala.Option.getOrElse(Option.scala:189)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.getSchema(JdbcUtils.scala:321)
 at org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD$.resolveTable(JDBCRDD.scala:63)
 at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.getSchema(JDBCRelation.scala:226)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:35)
 at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:364)
 at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:366)
 at org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:355)
 at scala.Option.getOrElse(Option.scala:189)
 at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:355)
 at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:240)
 at org.apache.spark.sql.DataFrameReader.jdbc(DataFrameReader.scala:381)
```

Considering the [data type mapping](https://docs.microsoft.com/ja-jp/sql/connect/jdbc/using-basic-data-types?view=sql-server-ver15) says, I think those spatial types can be mapped to Catalyst's `BinaryType`.

### Why are the changes needed?

To provide better support.

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

Yes. MS SQL Server users can use `geometry` and `geography` types in datasource tables.

### How was this patch tested?

New test case added to `MsSqlServerIntegrationSuite`.

Closes #31283 from sarutak/mssql-spatial-types.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-22 04:28:22 +00:00
Kousuke Saruta 842902154a [SPARK-34180][SQL] Fix the regression brought by SPARK-33888 for PostgresDialect
### What changes were proposed in this pull request?

This PR fixes the regression bug brought by SPARK-33888 (#30902).
After that PR merged, `PostgresDIalect#getCatalystType` throws Exception for array types.
```
[info] - Type mapping for various types *** FAILED *** (551 milliseconds)
[info]   java.util.NoSuchElementException: key not found: scale
[info]   at scala.collection.immutable.Map$EmptyMap$.apply(Map.scala:106)
[info]   at scala.collection.immutable.Map$EmptyMap$.apply(Map.scala:104)
[info]   at org.apache.spark.sql.types.Metadata.get(Metadata.scala:111)
[info]   at org.apache.spark.sql.types.Metadata.getLong(Metadata.scala:51)
[info]   at org.apache.spark.sql.jdbc.PostgresDialect$.getCatalystType(PostgresDialect.scala:43)
[info]   at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.getSchema(JdbcUtils.scala:321)
```

### Why are the changes needed?

To fix the regression bug.

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

No.

### How was this patch tested?

I confirmed the test case `SPARK-22291: Conversion error when transforming array types of uuid, inet and cidr to StingType in PostgreSQL` in `PostgresIntegrationSuite` passed.

I also confirmed whether all the `v2.*IntegrationSuite` pass because this PR changed them and they passed.

Closes #31262 from sarutak/fix-postgres-dialect-regression.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-22 13:03:02 +09:00
Kousuke Saruta 116f4cab6b [SPARK-34094][SQL] Extends StringTranslate to support unicode characters whose code point >= U+10000
### What changes were proposed in this pull request?

This PR extends `StringTranslate` to support unicode characters whose code point >= `U+10000`.

### Why are the changes needed?

To make it work with wide variety of characters.

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

Yes. Users can use `StringTranslate` with unicode characters whose code point >= `U+10000`.

### How was this patch tested?

New assertion added to the existing test.

Closes #31164 from sarutak/extends-translate.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-21 08:15:55 -06:00
Max Gekk e79c1cde1b [SPARK-34138][SQL] Keep dependants cached while refreshing v1 tables
### What changes were proposed in this pull request?
This PR changes cache refreshing of v1 tables in v1 commands. In particular, v1 table dependents are not removed from the cache after this PR. Comparing to current implementation, we just clear cached data of all dependents and keep them in the cache. So, the next actions will fill in the cached data of the original v1 table and its dependents. In more details:
1. Modified the `CatalogImpl.refreshTable()` method to use `recacheByPlan()` instead of `lookupCachedData()`, `uncacheQuery()` and `cacheQuery()`. Users can call this method via public API like `spark.catalog.refreshTable()`.
2. Rewritten the part in `CatalogImpl.refreshTable()` which was responsible for table meta-data refreshing because this code stopped to work properly after removing of the second `sparkSession.table(tableIdent)`.
3. Added new private method `invalidateCachedTable()` to `SessionCatalog`. Comparing to the existing `SessionCatalog.refreshTable`, it invalidates the relation cache only. If we called `SessionCatalog.refreshTable` from `CatalogImpl.refreshTable()`, we would refresh temporary and global temporary views twice (that could lead to refreshing file index twice).

### Why are the changes needed?
1. This should improve user experience with table/view caching. For example, let's imagine that an user has cached v1 table and cached view based on the table. And the user passed the table to external library which drops/renames/adds partitions in the v1 table. Unfortunately, the user gets the view uncached after that even he/she hasn't uncached the view explicitly.
2. To improve code maintenance.
3. To reduce the amount of calls to Hive external catalog.
4. Also this should speed up table recaching.
5. To have the same behavior as for v2 tables supported by https://github.com/apache/spark/pull/31172

### Does this PR introduce _any_ user-facing change?
From the view of the correctness of query results, there are no behavior changes but the changes might influence on consuming memory and query execution time. For example:

Before:
```scala
scala> sql("CREATE TABLE tbl (c int)")
scala> sql("CACHE TABLE tbl")
scala> sql("CREATE VIEW v AS SELECT * FROM tbl")
scala> sql("CACHE TABLE v")

scala> spark.catalog.isCached("v")
res6: Boolean = true
scala> spark.catalog.refreshTable("tbl")

scala> spark.catalog.isCached("v")
res8: Boolean = false
```

After:
```scala
scala> spark.catalog.refreshTable("tbl")

scala> spark.catalog.isCached("v")
res8: Boolean = true
```

### How was this patch tested?
1. Added new unit tests that create a view, a temporary view and a global temporary view on top of v1/v2 tables, and refresh the base table via `ALTER TABLE .. ADD/DROP/RENAME PARTITION`.
2. By running the unified test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
# build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite"
```

Closes #31206 from MaxGekk/refreshTable-recache-by-plan.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-21 13:03:24 +00:00
ulysses-you da4b50f8e2 [SPARK-33901][SQL][FOLLOWUP] Add drop table in charvarchar test
### What changes were proposed in this pull request?

Add `drop table` in charvarchar sql test.

### Why are the changes needed?

1. `drop table` is also a test case, for better coverage.
2. It's more clear to drop table which created in current test.

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

No.

### How was this patch tested?

Add test.

Closes #31277 from ulysses-you/SPARK-33901-FOLLOWUP.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-21 12:41:52 +00:00
Max Gekk 31d828379c [SPARK-34099][SQL][TESTS] Check re-caching of v2 table dependents after table altering
### What changes were proposed in this pull request?
Add tests to check that v2 table dependents are re-cached after table altering via the commands:
- `ALTER TABLE .. ADD PARTITION`
- `ALTER TABLE .. DROP PARTITION`
- `ALTER TABLE .. RENAME PARTITION`

### Why are the changes needed?
To improve test coverage and prevent regressions.

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

### How was this patch tested?
By running the modified test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableDropPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableRenamePartitionSuite"
```

Closes #31250 from MaxGekk/check-v2-dependents-recached.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-21 08:42:17 +00:00
beliefer 140538ea5b [SPARK-34096][SQL] Improve performance for nth_value ignore nulls over offset window
### What changes were proposed in this pull request?
The current implement of `UnboundedOffsetWindowFunctionFrame` and `UnboundedPrecedingOffsetWindowFunctionFrame` only support `nth_value` that respect nulls. So nth_value will execute with `UnboundedWindowFunctionFrame` and `UnboundedPrecedingWindowFunctionFrame`.
`UnboundedWindowFunctionFrame` and `UnboundedPrecedingWindowFunctionFrame` will call `updateExpressions` of `nth_value` multiple times.

### Why are the changes needed?
Improve performance for nth_value ignore nulls over offset window

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

### How was this patch tested?
Jenkins test

Closes #31178 from beliefer/SPARK-34096.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-21 07:31:36 +00:00
Kent Yao d640631e36 [SPARK-34164][SQL] Improve write side varchar check to visit only last few tailing spaces
### What changes were proposed in this pull request?

For varchar(N), we currently trim all spaces first to check whether the remained length exceeds, it not necessary to visit them all but at most to those after N.

### Why are the changes needed?

improve varchar performance for write side
### Does this PR introduce _any_ user-facing change?

no
### How was this patch tested?

benchmark and existing ut

Closes #31253 from yaooqinn/SPARK-34164.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-21 05:30:57 +00:00
Ismaël Mejía e9e81f798f [SPARK-27733][CORE] Upgrade Avro to version 1.10.1
### What changes were proposed in this pull request?

Update Avro dependency to version 1.10.1

### Why are the changes needed?

To catch up multiple improvements of Avro as well as fix security issues on transitive dependencies.

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

No

### How was this patch tested?

Since there were no API changes required we just run the tests

Closes #31232 from iemejia/SPARK-27733-avro-upgrade.

Authored-by: Ismaël Mejía <iemejia@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-20 15:42:27 -08:00
yangjie01 d68612a008 [SPARK-34176][BUILD] Restore the independent mvn test ability of sql/hive module in Scala 2.13
### What changes were proposed in this pull request?
There is one Java UT error when testing sql/hive module independently in Scala 2.13 after SPARK-33212,  the error message as follow:

```
[ERROR] Tests run: 2, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 20.353 s <<< FAILURE! - in org.apache.spark.sql.hive.JavaDataFrameSuite
[ERROR] org.apache.spark.sql.hive.JavaDataFrameSuite.testUDAF  Time elapsed: 18.548 s  <<< ERROR!
java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
	at org.apache.spark.sql.hive.JavaDataFrameSuite.checkAnswer(JavaDataFrameSuite.java:41)
	at org.apache.spark.sql.hive.JavaDataFrameSuite.testUDAF(JavaDataFrameSuite.java:92)
Caused by: java.lang.ClassNotFoundException: scala.collection.parallel.TaskSupport
	at org.apache.spark.sql.hive.JavaDataFrameSuite.checkAnswer(JavaDataFrameSuite.java:41)
	at org.apache.spark.sql.hive.JavaDataFrameSuite.testUDAF(JavaDataFrameSuite.java:92)
```

This pr add a Scala-2.13 profile with dependency of `scala-parallel-collections_` to `sql/hive` module to fix the Java UT in Scala 2.13.

### Why are the changes needed?
Recover the independent mvn test ability of sql/hive module in Scala 2.13.

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

### How was this patch tested?

- Pass the Jenkins or GitHub Action
- Manual test

```
dev/change-scala-version.sh 2.13

mvn clean install -Dhadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13 -pl  sql/hive -am -DskipTests

mvn test -Dhadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13 -pl  sql/hive
```

**Before**

```
[ERROR] Tests run: 2, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 18.725 s <<< FAILURE! - in org.apache.spark.sql.hive.JavaDataFrameSuite
[ERROR] org.apache.spark.sql.hive.JavaDataFrameSuite.testUDAF  Time elapsed: 16.853 s  <<< ERROR!
java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
	at org.apache.spark.sql.hive.JavaDataFrameSuite.checkAnswer(JavaDataFrameSuite.java:41)
	at org.apache.spark.sql.hive.JavaDataFrameSuite.testUDAF(JavaDataFrameSuite.java:92)
Caused by: java.lang.ClassNotFoundException: scala.collection.parallel.TaskSupport
	at org.apache.spark.sql.hive.JavaDataFrameSuite.checkAnswer(JavaDataFrameSuite.java:41)
	at org.apache.spark.sql.hive.JavaDataFrameSuite.testUDAF(JavaDataFrameSuite.java:92)

[INFO] Running org.apache.spark.sql.hive.JavaMetastoreDataSourcesSuite
16:15:36.186 WARN org.apache.spark.sql.hive.test.TestHiveExternalCatalog: Couldn't find corresponding Hive SerDe for data source provider org.apache.spark.sql.json. Persisting data source table `default`.`javasavedtable` into Hive metastore in Spark SQL specific format, which is NOT compatible with Hive.
16:15:36.288 WARN org.apache.hadoop.hive.ql.session.SessionState: METASTORE_FILTER_HOOK will be ignored, since hive.security.authorization.manager is set to instance of HiveAuthorizerFactory.
16:15:36.396 WARN org.apache.hadoop.hive.conf.HiveConf: HiveConf of name hive.internal.ss.authz.settings.applied.marker does not exist
16:15:36.397 WARN org.apache.hadoop.hive.conf.HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
16:15:36.397 WARN org.apache.hadoop.hive.conf.HiveConf: HiveConf of name hive.stats.retries.wait does not exist
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 3.481 s - in org.apache.spark.sql.hive.JavaMetastoreDataSourcesSuite
[INFO]
[INFO] Results:
[INFO]
[ERROR] Errors:
[ERROR]   JavaDataFrameSuite.testUDAF:92->checkAnswer:41 » NoClassDefFound scala/collect...
[INFO]
[ERROR] Tests run: 3, Failures: 0, Errors: 1, Skipped: 0
```

**After**

```
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 19.287 s - in org.apache.spark.sql.hive.JavaDataFrameSuite
[INFO] Running org.apache.spark.sql.hive.JavaMetastoreDataSourcesSuite
16:12:16.697 WARN org.apache.spark.sql.hive.test.TestHiveExternalCatalog: Couldn't find corresponding Hive SerDe for data source provider org.apache.spark.sql.json. Persisting data source table `default`.`javasavedtable` into Hive metastore in Spark SQL specific format, which is NOT compatible with Hive.
16:12:17.540 WARN org.apache.hadoop.hive.ql.session.SessionState: METASTORE_FILTER_HOOK will be ignored, since hive.security.authorization.manager is set to instance of HiveAuthorizerFactory.
16:12:17.653 WARN org.apache.hadoop.hive.conf.HiveConf: HiveConf of name hive.internal.ss.authz.settings.applied.marker does not exist
16:12:17.653 WARN org.apache.hadoop.hive.conf.HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
16:12:17.654 WARN org.apache.hadoop.hive.conf.HiveConf: HiveConf of name hive.stats.retries.wait does not exist
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 3.58 s - in org.apache.spark.sql.hive.JavaMetastoreDataSourcesSuite
[INFO]
[INFO] Results:
[INFO]
[INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0
```

Closes #31259 from LuciferYang/SPARK-34176.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-20 15:33:31 -08:00
yi.wu f498977222 [SPARK-34178][SQL] Copy tags for the new node created by MultiInstanceRelation.newInstance
### What changes were proposed in this pull request?

Call `copyTagsFrom` for the new node created by `MultiInstanceRelation.newInstance()`.

### Why are the changes needed?

```scala
val df = spark.range(2)
df.join(df, df("id") <=> df("id")).show()
```

For this query, it's supposed to be non-ambiguous join by the rule `DetectAmbiguousSelfJoin` because of the same attribute reference in the condition:

537a49fc09/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala (L125)

However, `DetectAmbiguousSelfJoin` can not apply this prediction due to the right side plan doesn't contain the dataset_id TreeNodeTag, which is missing after `MultiInstanceRelation.newInstance`. That's why we should preserve the tags info for the copied node.

Fortunately, the query is still considered as non-ambiguous join because `DetectAmbiguousSelfJoin` only checks the left side plan and the reference is the same as the left side plan. However, this's not the expected behavior but only a coincidence.

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

No.

### How was this patch tested?

Updated a unit test

Closes #31260 from Ngone51/fix-missing-tags.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-20 13:36:14 +00:00
Chao Sun 902a08b9e6 [SPARK-34052][SQL] store SQL text for a temp view created using "CACHE TABLE .. AS SELECT"
### What changes were proposed in this pull request?

This passes original SQL text to `CacheTableAsSelect` command in DSv1 and v2 so that it will be stored instead of the analyzed logical plan, similar to `CREATE VIEW` command.

In addition, this changes the behavior of dropping temporary view to also invalidate dependent caches in a cascade, when the config `SQLConf.STORE_ANALYZED_PLAN_FOR_VIEW` is false (which is the default value).

### Why are the changes needed?

Currently, after creating a temporary view with `CACHE TABLE ... AS SELECT` command, the view can still be queried even after the source table is dropped or replaced (in v2). This can cause correctness issue.

For instance, in the following:
```sql
> CREATE TABLE t ...;
> CACHE TABLE v AS SELECT * FROM t;
> DROP TABLE t;
> SELECT * FROM v;
```
The last select query still returns the old (and stale) result instead of fail. Note that the cache is already invalidated as part of dropping table `t`, but the temporary view `v` still exist.

On the other hand, the following:
```sql
> CREATE TABLE t ...;
> CREATE TEMPORARY VIEW v AS SELECT * FROM t;
> CACHE TABLE v;
> DROP TABLE t;
> SELECT * FROM v;
```
will throw "Table or view not found" error in the last select query.

This is related to #30567 which aligns the behavior of temporary view and global view by storing the original SQL text for temporary view, as opposed to the analyzed logical plan. However, the PR only handles `CreateView` case but not the `CacheTableAsSelect` case.

This also changes uncache logic and use cascade invalidation for temporary views created above. This is to align its behavior to how a permanent view is handled as of today, and also to avoid potential issues where a dependent view becomes invalid while its data is still kept in cache.

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

Yes, now when `SQLConf.STORE_ANALYZED_PLAN_FOR_VIEW` is set to false (the default value), whenever a table/permanent view/temp view that a cached view depends on is dropped, the cached view itself will become invalid during analysis, i.e., user will get "Table or view not found" error. In addition, when the dependent is a temp view in the previous case, the cache itself will also be invalidated.

### How was this patch tested?

Modified/Enhanced some existing tests.

Closes #31107 from sunchao/SPARK-34052.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-20 02:09:39 +00:00
Max Gekk 00b444d5ed [SPARK-34056][SQL][TESTS] Unify v1 and v2 ALTER TABLE .. RECOVER PARTITIONS tests
### What changes were proposed in this pull request?
1. Port DS V2 tests from `AlterTablePartitionV2SQLSuite ` to the test suite `v2.AlterTableRecoverPartitionsSuite`.
2. Port DS v1 tests from `DDLSuite` to `v1.AlterTableRecoverPartitionsSuiteBase`.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsParserSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31105 from MaxGekk/unify-recover-partitions-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-20 01:49:31 +00:00
Angerszhuuuu f6338a3e0b [SPARK-34121][SQL] Intersect operator missing rowCount when CBO enabled
### What changes were proposed in this pull request?

This pr add row count to `Intersect` operator when CBO enabled.

### Why are the changes needed?
Improve query performance, [JoinEstimation.estimateInnerOuterJoin](d6a68e0b67/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala (L55-L156)) need the row count.

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

### How was this patch tested?
Added

Closes #31240 from AngersZhuuuu/SPARK-34121.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-20 10:00:44 +09:00
Max Gekk 32dad1d5a6 [SPARK-34149][SQL] Refresh cache in v2 ALTER TABLE .. ADD PARTITION
### What changes were proposed in this pull request?
Clear table cache after adding partitions to v2 table in `AlterTableAddPartitionExec`.

### Why are the changes needed?
This PR fixes correctness issue. Without the fix, queries on cached tables modified via `ALTER TABLE .. ADD PARTITION` return incorrect results.

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

### How was this patch tested?
Added new UT to `org.apache.spark.sql.execution.command.v2.AlterTableAddPartitionSuite`:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableAddPartitionSuite"
```

Closes #31229 from MaxGekk/v2-add-partition-recache.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-19 09:42:07 +00:00
ulysses-you addbbe8339 [SPARK-33939][SQL] Make Column.named UnresolvedExtractValue use UnresolvedAlias to assign name
### What changes were proposed in this pull request?

Change `Column.named` code to let expression check  if exists `UnresolvedExtractValue` and use `UnresolvedAlias` to assign name.

### Why are the changes needed?

It's more reasonable to treat user specify expression as unresolved expression then we should assign name after analyze.

Let's say we have this code
```
spark.range(1).selectExpr("id as id1", "id as id2")
  .selectExpr("cast(struct(id1, id2).id1 as int)")
```

before this PR, the field name is `CAST(struct(id1, id2)[id1] AS INT)`.

After, the field name is `CAST(struct(id1, id2).id1 AS INT)`.

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

Yes, the default field name may be changed.

### How was this patch tested?

Add test.

Closes #30974 from ulysses-you/SPARK-33939-0.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-19 09:35:56 +00:00
Kent Yao 6fa2fb9eb5 [SPARK-34130][SQL] Impove preformace for char varchar padding and length check with StaticInvoke
### What changes were proposed in this pull request?

This could reduce the `generate.java` size to prevent codegen fallback which causes performance regression.

here is a case from tpcds that could be fixed by this improvement
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133964/testReport/org.apache.spark.sql.execution/LogicalPlanTagInSparkPlanSuite/q41/

The original case generate 20K bytes, we are trying to reduce it to less than 8k
### Why are the changes needed?

performance improvement as in the PR benchmark test, the performance  w/ codegen is 2~3x better than w/o codegen.

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

no

### How was this patch tested?

yes, it's a code reflect so the existing ut should be enough

cross-check with https://github.com/apache/spark/pull/31012 where the tpcds shall all pass

benchmark compared with master

```logtalk
================================================================================================
Char Varchar Read Side Perf
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 20, hasSpaces: false:    Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 20                         1571           1667          83         63.6          15.7       1.0X
read char with length 20                           1710           1764          58         58.5          17.1       0.9X
read varchar with length 20                        1774           1792          16         56.4          17.7       0.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 40, hasSpaces: false:    Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 40                         1824           1927          91         54.8          18.2       1.0X
read char with length 40                           1788           1928         137         55.9          17.9       1.0X
read varchar with length 40                        1676           1700          40         59.7          16.8       1.1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 60, hasSpaces: false:    Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 60                         1727           1762          30         57.9          17.3       1.0X
read char with length 60                           1628           1674          43         61.4          16.3       1.1X
read varchar with length 60                        1651           1665          13         60.6          16.5       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 80, hasSpaces: true:     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 80                         1748           1778          28         57.2          17.5       1.0X
read char with length 80                           1673           1678           9         59.8          16.7       1.0X
read varchar with length 80                        1667           1684          27         60.0          16.7       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Read with length 100, hasSpaces: true:    Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
read string with length 100                        1709           1743          48         58.5          17.1       1.0X
read char with length 100                          1610           1664          67         62.1          16.1       1.1X
read varchar with length 100                       1614           1673          53         61.9          16.1       1.1X

================================================================================================
Char Varchar Write Side Perf
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Write with length 20, hasSpaces: false:   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
write string with length 20                        2277           2327          67          4.4         227.7       1.0X
write char with length 20                          2421           2443          19          4.1         242.1       0.9X
write varchar with length 20                       2393           2419          27          4.2         239.3       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Write with length 40, hasSpaces: false:   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
write string with length 40                        2249           2290          38          4.4         224.9       1.0X
write char with length 40                          2386           2444          57          4.2         238.6       0.9X
write varchar with length 40                       2397           2405          12          4.2         239.7       0.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Write with length 60, hasSpaces: false:   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
write string with length 60                        2326           2367          41          4.3         232.6       1.0X
write char with length 60                          2478           2501          37          4.0         247.8       0.9X
write varchar with length 60                       2475           2503          24          4.0         247.5       0.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Write with length 80, hasSpaces: true:    Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
write string with length 80                        9367           9773         354          1.1         936.7       1.0X
write char with length 80                         10454          10621         238          1.0        1045.4       0.9X
write varchar with length 80                      18943          19503         571          0.5        1894.3       0.5X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Write with length 100, hasSpaces: true:   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
write string with length 100                      11055          11104          59          0.9        1105.5       1.0X
write char with length 100                        12204          12275          63          0.8        1220.4       0.9X
write varchar with length 100                     21737          22275         574          0.5        2173.7       0.5X

```

Closes #31199 from yaooqinn/SPARK-34130.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-19 09:03:06 +00:00
Yuming Wang 030639f456 [SPARK-34119][SQL] Keep necessary stats after partition pruning
### What changes were proposed in this pull request?

This pr keep necessary stats after partition pruning.

### Why are the changes needed?

Improve query performance. It will push down aggregate since SPARK-34081 because it can be planed as BroadcastHashJoin. But it lacks column statistics after [`PruneFileSourcePartitions`](d0c83f372b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala (L102-L103)). Therefore, it will eventually be planned as SortMergeJoin.

Please see the log:
```
join.right.stats: org.apache.spark.sql.catalyst.optimizer.PushDownPredicates: Statistics(sizeInBytes=348.8 KiB, rowCount=1.79E+4)
join.right.stats: org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions: Statistics(sizeInBytes=1414.2 EiB)
```

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

No.

### How was this patch tested?

Unit test and benchmark test

SQL | Before this PR(Seconds) | After this PR(Seconds)
-- | -- | --
q14a | 594 | 384
q14b | 600 | 402

This change will not affect the results of `PlanStabilitySuite`, because it does not have partition column.

Closes #31205 from wangyum/SPARK-34119.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-19 06:09:16 +00:00
Max Gekk a98e77c113 [SPARK-34143][SQL][TESTS] Fix adding partitions to fully partitioned v2 tables
### What changes were proposed in this pull request?
While adding new partition to v2 `InMemoryAtomicPartitionTable`/`InMemoryPartitionTable`, add single row to the table content when the table is fully partitioned.

### Why are the changes needed?
The `ALTER TABLE .. ADD PARTITION` command does not change content of fully partitioned v2 table. For instance, `INSERT INTO` changes table content:
```scala
      sql(s"CREATE TABLE t (p0 INT, p1 STRING) USING _ PARTITIONED BY (p0, p1)")
      sql(s"INSERT INTO t SELECT 1, 'def'")
      sql(s"SELECT * FROM t").show(false)

+---+---+
|p0 |p1 |
+---+---+
|1  |def|
+---+---+
```
but `ALTER TABLE .. ADD PARTITION` doesn't change v2 table content:
```scala
      sql(s"ALTER TABLE t ADD PARTITION (p0 = 0, p1 = 'abc')")
      sql(s"SELECT * FROM t").show(false)

+---+---+
|p0 |p1 |
+---+---+
+---+---+
```

### Does this PR introduce _any_ user-facing change?
No, the changes impact only on tests but for the example above in tests:
```scala
      sql(s"ALTER TABLE t ADD PARTITION (p0 = 0, p1 = 'abc')")
      sql(s"SELECT * FROM t").show(false)

+---+---+
|p0 |p1 |
+---+---+
|0  |abc|
+---+---+
```

### How was this patch tested?
By running the unified tests for `ALTER TABLE .. ADD PARTITION`.

Closes #31216 from MaxGekk/add-partition-by-all-columns.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-19 05:40:15 +00:00
ulysses-you 055124a048 [SPARK-34150][SQL] Strip Null literal.sql in resolve alias
### What changes were proposed in this pull request?

Change null Literal to PrettyAttribute during ResolveAlias.

### Why are the changes needed?

We will convert `Literal(null)` to target data type during analysis. Then the generated alias name will include something like `CAST(NULL AS String)` instead of `NULL`.
```
spark.sql("SELECT RAND(null)").columns

-- before
rand(CAST(NULL AS INT))

-- after
rand(NULL)
```

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

Yes, the default column name maybe changed.

### How was this patch tested?

Add test and pass exists test.

Closes #31233 from ulysses-you/SPARK-34150.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-19 03:35:08 +00:00
Max Gekk bea10a6274 [SPARK-34153][SQL] Remove unused getRawTable() from HiveExternalCatalog.alterPartitions()
### What changes were proposed in this pull request?
Remove unused call of `getRawTable()` from `HiveExternalCatalog.alterPartitions()`.

### Why are the changes needed?
It reduces the number of calls to Hive External catalog.

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

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite"
```

Closes #31234 from MaxGekk/remove-getRawTable-from-alterPartitions.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-19 11:42:33 +09:00
Max Gekk 514172aae7 [SPARK-34074][SQL][TESTS][FOLLOWUP] Fix table size parsing from statistics
### What changes were proposed in this pull request?
Fix table size parsing from the `Statistics` field which is formed at: c3d81fbe79/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala (L573) . Before the fix, `getTableSize()` returns only the last digit. This works for Hive table in the tests because its size < 10 bytes, and accidentally works for V1 In-Memory catalog table in the tests.

### Why are the changes needed?
This makes tests more reliable. For example, the parsing can not work in `AlterTableDropPartitionSuite` when table size before partition dropping:
```
+---------+
|data_type|
+---------+
|878 bytes|
+---------+
```

After:
```
+---------+
|data_type|
+---------+
|439 bytes|
+---------+
```
at:
```scala
        val onePartSize = getTableSize(t)
        assert(0 < onePartSize && onePartSize < twoPartSize)
```

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

### How was this patch tested?
By existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableDropPartitionSuite"
```

Closes #31237 from MaxGekk/optimize-updateTableStats-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-19 09:51:20 +09:00
Liang-Chi Hsieh bba830f523 [SPARK-34148][TEST][SS] Move general StateStore tests to StateStoreSuiteBase
### What changes were proposed in this pull request?

This patch moves move general StateStore-related tests into `StateStoreSuiteBase`.

### Why are the changes needed?

There are some general StateStore tests in `StateStoreSuite` which is `HDFSBackedStateStoreProvider`-specific test suite. We should move general tests into `StateStoreSuiteBase`, so it is easier to incorporate other StateStores.

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #31219 from viirya/SPARK-34148.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-18 13:03:25 -08:00
Max Gekk dee596e3ef [SPARK-34027][SQL] Refresh cache in ALTER TABLE .. RECOVER PARTITIONS
### What changes were proposed in this pull request?
Invoke `refreshTable()` from `CatalogImpl` which refreshes the cache in v1 `ALTER TABLE .. RECOVER PARTITIONS`.

### Why are the changes needed?
This fixes the issues portrayed by the example:
```sql
spark-sql> create table tbl (col int, part int) using parquet partitioned by (part);
spark-sql> insert into tbl partition (part=0) select 0;
spark-sql> cache table tbl;
spark-sql> select * from tbl;
0	0
spark-sql> show table extended like 'tbl' partition(part=0);
default	tbl	false	Partition Values: [part=0]
Location: file:/Users/maximgekk/proj/recover-partitions-refresh-cache/spark-warehouse/tbl/part=0
...
```
Create new partition by copying the existing one:
```
$ cp -r /Users/maximgekk/proj/recover-partitions-refresh-cache/spark-warehouse/tbl/part=0 /Users/maximgekk/proj/recover-partitions-refresh-cache/spark-warehouse/tbl/part=1
```
```sql
spark-sql> alter table tbl recover partitions;
spark-sql> select * from tbl;
0	0
```

The last query must return `0	1` since it has been recovered by `ALTER TABLE .. RECOVER PARTITIONS`.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes for the example above:
```sql
...
spark-sql> alter table tbl recover partitions;
spark-sql> select * from tbl;
0	0
0	1
```

### How was this patch tested?
By running the affected test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CachedTableSuite"
```

Closes #31066 from MaxGekk/recover-partitions-refresh-cache.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-18 13:52:39 +00:00
yangjie01 163afa6fcf [SPARK-34151][SQL] Replaces java.io.File.toURL with java.io.File.toURI.toURL
### What changes were proposed in this pull request?
`java.io.FIle.toURL` method does not automatically escape characters that are illegal in URLs.

Java doc recommended that new code convert an abstract pathname into a URL by first converting it into a URI, via the `toURI` method, and then converting the URI into a URL via the `URI.toURL` method.

So this pr cleaned up the relevant cases in Spark code.

### Why are the changes needed?
Cleaning up `Deprecated` Java API usage.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31230 from LuciferYang/SPARK-34151.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-18 21:39:00 +09:00
Terry Kim 78893b8dc9 [SPARK-34139][SQL] UnresolvedRelation should retain SQL text position for DDL commands
### What changes were proposed in this pull request?

Currently, there are many DDL commands where the position of the unresolved identifiers are incorrect:
```
scala> sql("CACHE TABLE unknown")
org.apache.spark.sql.AnalysisException: Table or view not found: unknown; line 1 pos 0;
```
, whereas the `pos` should be `12`.

This PR proposes to fix this issue for commands using `UnresolvedRelation`:
```
CACHE TABLE unknown
UNCACHE TABLE unknown
DELETE FROM unknown
UPDATE unknown SET name='abc'
MERGE INTO unknown1 AS target USING unknown2 AS source ON target.col = source.col WHEN MATCHED THEN DELETE
INSERT INTO TABLE unknown SELECT 1
INSERT OVERWRITE TABLE unknown VALUES (1, 'a')
```

### Why are the changes needed?

To fix a bug.

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

Yes, now the above example will print the following:
```
org.apache.spark.sql.AnalysisException: Table or view not found: unknown; line 1 pos 12;
```

### How was this patch tested?

Add a new test.

Closes #31209 from imback82/unresolved_relation_message.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-18 05:57:25 +00:00
Yuming Wang c87b0085c9 [SPARK-33696][BUILD][SQL] Upgrade built-in Hive to 2.3.8
### What changes were proposed in this pull request?

Hive 2.3.8 changes:
HIVE-19662: Upgrade Avro to 1.8.2
HIVE-24324: Remove deprecated API usage from Avro
HIVE-23980: Shade Guava from hive-exec in Hive 2.3
HIVE-24436: Fix Avro NULL_DEFAULT_VALUE compatibility issue
HIVE-24512: Exclude calcite in packaging.
HIVE-22708: Fix for HttpTransport to replace String.equals
HIVE-24551: Hive should include transitive dependencies from calcite after shading it
HIVE-24553: Exclude calcite from test-jar dependency of hive-exec

### Why are the changes needed?

Upgrade Avro and Parquet to latest version.

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

No.

### How was this patch tested?

Existing test add test try to upgrade Parquet to 1.11.1 and Avro to 1.10.1: https://github.com/apache/spark/pull/30517

Closes #30657 from wangyum/SPARK-33696.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-17 21:54:35 -08:00
Terry Kim 0540392464 [SPARK-34140][SQL] Move QueryCompilationErrors.scala and QueryExecutionErrors.scala to org/apache/spark/sql/errors
### What changes were proposed in this pull request?

`QueryCompilationErrors.scala` and `QueryExecutionErrors.scala` use the `org.apache.spark.sql.errors` package, but these files are reside in `org/apache/spark/sql` directory. This PR proposes to move these files to `org/apache/spark/sql/errors`.

### Why are the changes needed?

To match the package name with the directory structure.

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

No

### How was this patch tested?

Existing tests

Closes #31211 from imback82/error_package.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-16 13:40:21 -08:00
Max Gekk c3d81fbe79 [SPARK-34060][SQL][FOLLOWUP] Preserve serializability of canonicalized CatalogTable
### What changes were proposed in this pull request?
Replace `toMap` by `map(identity).toMap` while getting canonicalized representation of `CatalogTable`. `CatalogTable` became not serializable after https://github.com/apache/spark/pull/31112 due to usage of `filterKeys`. The workaround was taken from https://github.com/scala/bug/issues/7005.

### Why are the changes needed?
This prevents the errors like:
```
[info]   org.apache.spark.SparkException: Job aborted due to stage failure: Task not serializable: java.io.NotSerializableException: scala.collection.immutable.MapLike$$anon$1
[info]   Cause: java.io.NotSerializableException: scala.collection.immutable.MapLike$$anon$1
```

### Does this PR introduce _any_ user-facing change?
Should not.

### How was this patch tested?
By running the test suite affected by https://github.com/apache/spark/pull/31112:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```

Closes #31197 from MaxGekk/fix-caching-hive-table-2-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-15 17:02:29 -08:00
Chao Sun b6f46ca297 [SPARK-33212][BUILD] Upgrade to Hadoop 3.2.2 and move to shaded clients for Hadoop 3.x profile
### What changes were proposed in this pull request?

This:
1. switches Spark to use shaded Hadoop clients, namely hadoop-client-api and hadoop-client-runtime, for Hadoop 3.x.
2. upgrade built-in version for Hadoop 3.x to Hadoop 3.2.2

Note that for Hadoop 2.7, we'll still use the same modules such as hadoop-client.

In order to still keep default Hadoop profile to be hadoop-3.2, this defines the following Maven properties:

```
hadoop-client-api.artifact
hadoop-client-runtime.artifact
hadoop-client-minicluster.artifact
```

which default to:
```
hadoop-client-api
hadoop-client-runtime
hadoop-client-minicluster
```
but all switch to `hadoop-client` when the Hadoop profile is hadoop-2.7. A side affect from this is we'll import the same dependency multiple times. For this I have to disable Maven enforcer `banDuplicatePomDependencyVersions`.

Besides above, there are the following changes:
- explicitly add a few dependencies which are imported via transitive dependencies from Hadoop jars, but are removed from the shaded client jars.
- removed the use of `ProxyUriUtils.getPath` from `ApplicationMaster` which is a server-side/private API.
- modified `IsolatedClientLoader` to exclude `hadoop-auth` jars when Hadoop version is 3.x. This change should only matter when we're not sharing Hadoop classes with Spark (which is _mostly_ used in tests).

### Why are the changes needed?

Hadoop 3.2.2 is released with new features and bug fixes, so it's good for the Spark community to adopt it. However, latest Hadoop versions starting from Hadoop 3.2.1 have upgraded to use Guava 27+. In order to resolve Guava conflicts, this takes the approach by switching to shaded client jars provided by Hadoop. This also has the benefits of avoid pulling other 3rd party dependencies from Hadoop side so as to avoid more potential future conflicts.

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

When people use Spark with `hadoop-provided` option, they should make sure class path contains `hadoop-client-api` and `hadoop-client-runtime` jars. In addition, they may need to make sure these jars appear before other Hadoop jars in the order. Otherwise, classes may be loaded from the other non-shaded Hadoop jars and cause potential conflicts.

### How was this patch tested?

Relying on existing tests.

Closes #30701 from sunchao/test-hadoop-3.2.2.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-15 14:06:50 -08:00
Kent Yao a235c3b254 [SPARK-34037][SQL] Remove unnecessary upcasting for Avg & Sum which handle by themself internally
### What changes were proposed in this pull request?
The type-coercion for numeric types of average and sum is not necessary at all, as the resultType and sumType can prevent the overflow.

### Why are the changes needed?

rm unnecessary logic which may cause potential performance regressions

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

no
### How was this patch tested?

tpcds tests for plan

Closes #31079 from yaooqinn/SPARK-34037.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-01-15 10:18:58 -08:00
yangjie01 9e33d49b5b [SPARK-33346][CORE][SQL][MLLIB][DSTREAM][K8S] Change the never changed 'var' to 'val'
### What changes were proposed in this pull request?
Some local variables are declared as `var`, but they are never reassigned and should be declared as `val`, so this pr turn these  from `var` to  `val` except for `mockito` related cases.

### Why are the changes needed?
Use `val` instead of `var` when possible.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31142 from LuciferYang/SPARK-33346.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-15 08:47:02 -06:00
Wenchen Fan 6cd0092150 Revert "[SPARK-34064][SQL] Cancel the running broadcast sub-jobs when SQL statement is cancelled"
This reverts commit f1b21ba505.
2021-01-15 21:45:17 +08:00
Peter Toth 00d43b1f82 [SPARK-32864][SQL] Support ORC forced positional evolution
### What changes were proposed in this pull request?
Add support for `orc.force.positional.evolution` config that forces ORC top level column matching by position rather than by name.

This does work in Hive:
```
> set orc.force.positional.evolution;
+--------------------------------------+
|                 set                  |
+--------------------------------------+
| orc.force.positional.evolution=true  |
+--------------------------------------+
> create table t (c1 string, c2 string) stored as orc;
> insert into t values ('foo', 'bar');
> alter table t change c1 c3 string;
```
The orc file in this case contains the original `c1` and `c2` columns that doesn't match the metadata in HMS. But due to the positional evolution setting, Hive is capable to return all the data:
```
> select * from t;
+--------+--------+
| t.c3   | t.c2   |
+--------+--------+
| foo    | bar    |
+--------+--------+
```
Without this PR Spark returns `null`s for the renamed `c3` column.

After this PR Spark returns the data in `c3` column.

### Why are the changes needed?
Hive/ORC does support it.

### Does this PR introduce _any_ user-facing change?
Yes, we will support `orc.force.positional.evolution`.

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

Closes #29737 from peter-toth/SPARK-32864-support-orc-forced-positional-evolution.

Lead-authored-by: Peter Toth <peter.toth@gmail.com>
Co-authored-by: Peter Toth <ptoth@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-14 21:27:25 -08:00
Kent Yao acd6c1271b [SPARK-34114][SQL] should not trim right for read-side char length check and padding
### What changes were proposed in this pull request?

On the read-side, we should respect the original data instead of trimming it first.

It brings extra overhead on the code-gen code side, trimming and padding for the same field, and it's also unnecessary and a bug

### Why are the changes needed?

bugfix and perf regression

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

no
### How was this patch tested?

new tests

Closes #31181 from yaooqinn/SPARK-34114.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-15 04:30:23 +00:00
Kousuke Saruta bec80d7eec [SPARK-34101][SQL] Make spark-sql CLI configurable for the behavior of printing header by SET command
### What changes were proposed in this pull request?

This PR introduces a new property `spark.sql.cli.print.header` to let users change the behavior of printing header for spark-sql CLI by SET command.

### Why are the changes needed?

Like Hive CLI, spark-sql CLI accepts `hive.cli.print.header` property and we can change the behavior of printing header.
But spark-sql CLI doesn't allow users to change Hive specific configurations dynamically by SET command.
So, it's better to support the way to change the behavior by SET command.

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

Yes. Users can dynamically change the behavior by SET command.

### How was this patch tested?

I confirmed with the following commands/queries.
```
spark-sql> select (1) as a, (2) as b, (3) as c, (4) as d;
1	2	3	4
Time taken: 3.218 seconds, Fetched 1 row(s)
spark-sql> set spark.sql.cli.print.header=true;
key	value
spark.sql.cli.print.header	true
Time taken: 1.506 seconds, Fetched 1 row(s)
spark-sql> select (1) as a, (2) as b, (3) as c, (4) as d;
a	b	c	d
1	2	3	4
Time taken: 0.79 seconds, Fetched 1 row(s)
```

Closes #31173 from sarutak/spark-sql-print-header.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-15 13:15:48 +09:00
Max Gekk adba2ec8f2 [SPARK-34099][SQL] Keep dependents of v2 tables cached during cache refreshing in DS v2 commands
### What changes were proposed in this pull request?

This PR changes cache refreshing of v2 tables in v2 commands. In particular, v2 table dependents are not removed from the cache after this PR. Comparing to current implementation, we just clear cached data of all dependents and keep them in the cache. So, the next actions will fill in the cached data of the original v2 table and its dependents. In more details:

1. Add new method `recacheTable()` to `DataSourceV2Strategy` and pass it the exec node where need to recache table. New method uses `recacheByPlan` to refresh data cache of v2 tables, and keeps table dependents still cached **while clearing their caches**.
2. Simplify `invalidateCache` (and rename it `invalidateTableCache`) by retargeting it for only table cache invalidation.
3. Modify a test for `REFRESH TABLE` and check that v2 table dependent is still cached after refreshing the base table.

### Why are the changes needed?
1. This should improve user experience with table/view caching. For example, let's imagine that an user has cached v2 table and cached view based on the table. And the user passed the table to external library which drops/renames/adds partitions in the v2 table. Unfortunately, the user gets the view uncached after that even he/she hasn't uncached the view explicitly.
2. Improve code maintenance.
3. Reduce the number of calls to the Cache Manager when need to recache a table. Before the changes, `invalidateCache()` invokes the Cache Manager 3 times: `lookupCachedData()`, `uncacheQuery()` and `cacheQuery()`.
4. Also this should speed up table recaching.

### Does this PR introduce _any_ user-facing change?
From the view of the correctness of query results, there are no behavior changes but the changes might influence on consuming memory and query execution time.

### How was this patch tested?
By running the existing test suites for v2 the add/drop/rename partition commands.

Closes #31172 from MaxGekk/dsv2-recache-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-15 03:32:49 +00:00
yangjie01 8ed23ed499 [SPARK-34118][CORE][SQL] Replaces filter and check for emptiness with exists or forall
### What changes were proposed in this pull request?
This pr use `exists` or `forall` to simplify `filter + emptiness check`, it's semantically consistent, but looks simpler. The rule as follow:

- `seq.filter(p).size == 0)` -> `!seq.exists(p)`
- `seq.filter(p).length > 0` -> `seq.exists(p)`
- `seq.filterNot(p).isEmpty` -> `seq.forall(p)`
- `seq.filterNot(p).nonEmpty` -> `!seq.forall(p)`

### Why are the changes needed?
Code Simpilefications.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31184 from LuciferYang/SPARK-34118.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-15 12:12:33 +09:00
Liang-Chi Hsieh 0e64a22b28 [SPARK-34116][SS][TEST] Separate state store numKeys metric test and validate metrics after committing
### What changes were proposed in this pull request?

This patch proposes to pull the test of `numKeys` metric into a separate test in `StateStoreSuite`.

### Why are the changes needed?

Right now in `StateStoreSuite`, the tests of get/put/remove/commit are mixed with `numKeys` metric test. I found it is flaky when I was testing with other `StateStore` implementation.

Current test logic is tightly bound to the in-memory map behavior of `HDFSBackedStateStore`. For example, put can immediately show up in the `numKeys` metric.

But for a `StateStore` implementation relying on external storage, e.g. RocksDB, the metric might be updated once the data is actually committed. And `StateStoreSuite` should be a common test suite for all kinds of StateStore implementations.

Specifically, we also are able to check these metrics after state store is updated (committed). So I think we can refactor the test a little bit to make it easier to incorporate other `StateStore` externally.

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #31183 from viirya/SPARK-34116.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-14 15:03:22 -08:00
ulysses-you 92e5cfd58d [SPARK-33989][SQL] Strip auto-generated cast when using Cast.sql
### What changes were proposed in this pull request?

This PR aims to strip auto-generated cast. The main logic is:
1. Add tag if Cast is specified by user.
2. Wrap `PrettyAttribute` in usePrettyExpression.

### Why are the changes needed?

Make sql consistent with dsl. Here is an inconsistent example before this PR:

```
-- output field name: FLOOR(1)
spark.emptyDataFrame.select(floor(lit(1)))

-- output field name: FLOOR(CAST(1 AS DOUBLE))
spark.sql("select floor(1)")
```

Note that, we don't remove the `Cast` so the auto-generated `Cast` can still work. The only changed place is `usePrettyExpression`, we use `PrettyAttribute` replace `Cast` to give a better sql string.

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

Yes, the default field name may change.

### How was this patch tested?

Add test and pass exists test.

Closes #31034 from ulysses-you/SPARK-33989.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-14 15:27:14 +00:00
Dereck Li 1288ad814e [SPARK-34067][SQL] PartitionPruning push down pruningHasBenefit function into insertPredicate function to decrease calculate time
### What changes were proposed in this pull request?
PartitionPruning push down pruningHasBenefit function into insertPredicate function to decrease calculate time

### Why are the changes needed?
 to accelerate PartitionPruning prune calculate

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

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

Closes #31122 from monkeyboy123/optimize-dynamic-pruning.

Authored-by: Dereck Li <monkeyboy.ljh@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-01-14 16:28:06 +08:00
Yuming Wang d3ea308c8f [SPARK-34081][SQL] Only pushdown LeftSemi/LeftAnti over Aggregate if join can be planned as broadcast join
### What changes were proposed in this pull request?

Should not pushdown LeftSemi/LeftAnti over Aggregate for some cases.

```scala
spark.range(50000000L).selectExpr("id % 10000 as a", "id % 10000 as b").write.saveAsTable("t1")
spark.range(40000000L).selectExpr("id % 8000 as c", "id % 8000 as d").write.saveAsTable("t2")
spark.sql("SELECT distinct a, b FROM t1 INTERSECT SELECT distinct c, d FROM t2").explain
```

Before this pr:
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- HashAggregate(keys=[a#16L, b#17L], functions=[])
   +- HashAggregate(keys=[a#16L, b#17L], functions=[])
      +- HashAggregate(keys=[a#16L, b#17L], functions=[])
         +- Exchange hashpartitioning(a#16L, b#17L, 5), ENSURE_REQUIREMENTS, [id=#72]
            +- HashAggregate(keys=[a#16L, b#17L], functions=[])
               +- SortMergeJoin [coalesce(a#16L, 0), isnull(a#16L), coalesce(b#17L, 0), isnull(b#17L)], [coalesce(c#18L, 0), isnull(c#18L), coalesce(d#19L, 0), isnull(d#19L)], LeftSemi
                  :- Sort [coalesce(a#16L, 0) ASC NULLS FIRST, isnull(a#16L) ASC NULLS FIRST, coalesce(b#17L, 0) ASC NULLS FIRST, isnull(b#17L) ASC NULLS FIRST], false, 0
                  :  +- Exchange hashpartitioning(coalesce(a#16L, 0), isnull(a#16L), coalesce(b#17L, 0), isnull(b#17L), 5), ENSURE_REQUIREMENTS, [id=#65]
                  :     +- FileScan parquet default.t1[a#16L,b#17L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/spark/spark-warehouse/org.apache.spark.sql.Data..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:bigint,b:bigint>
                  +- Sort [coalesce(c#18L, 0) ASC NULLS FIRST, isnull(c#18L) ASC NULLS FIRST, coalesce(d#19L, 0) ASC NULLS FIRST, isnull(d#19L) ASC NULLS FIRST], false, 0
                     +- Exchange hashpartitioning(coalesce(c#18L, 0), isnull(c#18L), coalesce(d#19L, 0), isnull(d#19L), 5), ENSURE_REQUIREMENTS, [id=#66]
                        +- HashAggregate(keys=[c#18L, d#19L], functions=[])
                           +- Exchange hashpartitioning(c#18L, d#19L, 5), ENSURE_REQUIREMENTS, [id=#61]
                              +- HashAggregate(keys=[c#18L, d#19L], functions=[])
                                 +- FileScan parquet default.t2[c#18L,d#19L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/spark/spark-warehouse/org.apache.spark.sql.Data..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c:bigint,d:bigint>
```

After this pr:
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- HashAggregate(keys=[a#16L, b#17L], functions=[])
   +- Exchange hashpartitioning(a#16L, b#17L, 5), ENSURE_REQUIREMENTS, [id=#74]
      +- HashAggregate(keys=[a#16L, b#17L], functions=[])
         +- SortMergeJoin [coalesce(a#16L, 0), isnull(a#16L), coalesce(b#17L, 0), isnull(b#17L)], [coalesce(c#18L, 0), isnull(c#18L), coalesce(d#19L, 0), isnull(d#19L)], LeftSemi
            :- Sort [coalesce(a#16L, 0) ASC NULLS FIRST, isnull(a#16L) ASC NULLS FIRST, coalesce(b#17L, 0) ASC NULLS FIRST, isnull(b#17L) ASC NULLS FIRST], false, 0
            :  +- Exchange hashpartitioning(coalesce(a#16L, 0), isnull(a#16L), coalesce(b#17L, 0), isnull(b#17L), 5), ENSURE_REQUIREMENTS, [id=#67]
            :     +- HashAggregate(keys=[a#16L, b#17L], functions=[])
            :        +- Exchange hashpartitioning(a#16L, b#17L, 5), ENSURE_REQUIREMENTS, [id=#61]
            :           +- HashAggregate(keys=[a#16L, b#17L], functions=[])
            :              +- FileScan parquet default.t1[a#16L,b#17L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/spark/spark-warehouse/org.apache.spark.sql.Data..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:bigint,b:bigint>
            +- Sort [coalesce(c#18L, 0) ASC NULLS FIRST, isnull(c#18L) ASC NULLS FIRST, coalesce(d#19L, 0) ASC NULLS FIRST, isnull(d#19L) ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(coalesce(c#18L, 0), isnull(c#18L), coalesce(d#19L, 0), isnull(d#19L), 5), ENSURE_REQUIREMENTS, [id=#68]
                  +- HashAggregate(keys=[c#18L, d#19L], functions=[])
                     +- Exchange hashpartitioning(c#18L, d#19L, 5), ENSURE_REQUIREMENTS, [id=#63]
                        +- HashAggregate(keys=[c#18L, d#19L], functions=[])
                           +- FileScan parquet default.t2[c#18L,d#19L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/spark/spark-warehouse/org.apache.spark.sql.Data..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c:bigint,d:bigint>
```

### Why are the changes needed?

1. Pushdown LeftSemi/LeftAnti over Aggregate will affect performance.
2. It will remove user added DISTINCT operator, e.g.: [q38](https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q38.sql), [q87](https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q87.sql).

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

No.

### How was this patch tested?

Unit test and benchmark test.

SQL | Before this PR(Seconds) | After this PR(Seconds)
-- | -- | --
q14a | 660 | 594
q14b | 660 | 600
q38 | 55 | 29
q87 | 66 | 35

Before this pr:
![image](https://user-images.githubusercontent.com/5399861/104452849-8789fc80-55de-11eb-88da-44059899f9a9.png)

After this pr:
![image](https://user-images.githubusercontent.com/5399861/104452899-9a043600-55de-11eb-9286-d8f3a23ca3b8.png)

Closes #31145 from wangyum/SPARK-34081.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-14 04:37:55 +00:00
Gengliang Wang 467d758973 [SPARK-34075][SQL][CORE] Hidden directories are being listed for partition inference
### What changes were proposed in this pull request?

Fix a regression from https://github.com/apache/spark/pull/29959.

In Spark, the following file paths are considered as hidden paths and they are ignored on file reads:
1. starts with "_" and doesn't contain "="
2. starts with "."

However, after the refactoring PR https://github.com/apache/spark/pull/29959, the hidden paths are not filtered out on partition inference: https://github.com/apache/spark/pull/29959/files#r556432426

This PR is to fix the bug. To archive the goal, the method `InMemoryFileIndex.shouldFilterOut` is refactored as `HadoopFSUtils.shouldFilterOutPathName`

### Why are the changes needed?

Bugfix

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

Yes, it fixes a bug for reading file paths with partitions.

### How was this patch tested?

Unit test

Closes #31169 from gengliangwang/fileListingBug.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-14 09:39:38 +09:00
Kousuke Saruta b7da108cae [SPARK-33690][SQL][FOLLOWUP] Escape further meta-characters in showString
### What changes were proposed in this pull request?

This is a followup PR for SPARK-33690 (#30647) .
In addition to the original PR, this PR intends to escape the following meta-characters in `Dataset#showString`.

  * `\r` (carrige ret)
  * `\f` (form feed)
  * `\b` (backspace)
  * `\u000B` (vertical tab)
  * `\u0007` (bell)

### Why are the changes needed?

To avoid breaking the layout of `Dataset#showString`.
`\u0007` does not break the layout of `Dataset#showString` but it's noisy (beeps for each row) so it should be also escaped.

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

No.

### How was this patch tested?

Modified the existing tests.
I also build the documents and check the generated html for `sql-migration-guide.md`.

Closes #31144 from sarutak/escape-metacharacters-in-getRows.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-13 18:13:01 -06:00
Kousuke Saruta 62d8466c74 [SPARK-34051][SQL] Support 32-bit unicode escape in string literals
### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
This PR adds a feature which supports 32-bit unicode escape in string literals like PostgreSQL or some modern programming languages do (e.g, Python3, C++11 and Rust).
In addition to the feature which supports 16-bit unicode escape like `"\u0041"`, users can express unicode characters like `"\U00020BB7"` with this change.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
Users can express unicode characters straightly without surrogate pair.

### Does this PR introduce _any_ user-facing change?
<!--
Note that it means *any* user-facing change including all aspects such as the documentation fix.
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
If no, write 'No'.
-->
Yes. Users an express all the unicode characters straightly.

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
Added new assertions to the existing test case.

Closes #31096 from sarutak/32-bit-unicode-escape.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-13 18:10:03 -06:00
yangjie01 8b1ba233f1 [SPARK-34068][CORE][SQL][MLLIB][GRAPHX] Remove redundant collection conversion
### What changes were proposed in this pull request?
There are some redundant collection conversion can be removed, for version compatibility, clean up these with Scala-2.13 profile.

### Why are the changes needed?
Remove redundant collection conversion

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

### How was this patch tested?
- Pass the Jenkins or GitHub  Action
- Manual test `core`, `graphx`, `mllib`, `mllib-local`, `sql`, `yarn`,`kafka-0-10` in Scala 2.13 passed

Closes #31125 from LuciferYang/SPARK-34068.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-13 18:07:02 -06:00
yangjie01 8c5fecda73 [SPARK-34070][CORE][SQL] Replaces find and emptiness check with exists
### What changes were proposed in this pull request?
This pr use `exists` to simplify `find + emptiness check`, it's semantically consistent, but looks simpler.

**Before**

```
seq.find(p).isDefined

or

seq.find(p).isEmpty
```

**After**

```
seq.exists(p)

or

!seq.exists(p)
```
### Why are the changes needed?
Code Simpilefications.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31130 from LuciferYang/SPARK-34070.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-13 10:42:24 -06:00
Chao Sun 62d82b5b27 [SPARK-34076][SQL] SQLContext.dropTempTable fails if cache is non-empty
### What changes were proposed in this pull request?

This changes `CatalogImpl.dropTempView` and `CatalogImpl.dropGlobalTempView` use analyzed logical plan instead of `viewDef` which is unresolved.

### Why are the changes needed?

Currently, `CatalogImpl.dropTempView` is implemented as following:

```scala
override def dropTempView(viewName: String): Boolean = {
  sparkSession.sessionState.catalog.getTempView(viewName).exists { viewDef =>
    sparkSession.sharedState.cacheManager.uncacheQuery(
      sparkSession, viewDef, cascade = false)
    sessionCatalog.dropTempView(viewName)
  }
}
```

Here, the logical plan `viewDef` is not resolved, and when passing to `uncacheQuery`, it could fail at `sameResult` call, where canonicalized plan is compared. The error message looks like:
```
Invalid call to qualifier on unresolved object, tree: 'key
```

This can be reproduced via:
```scala
sql(s"CREATE TEMPORARY VIEW $v AS SELECT key FROM src LIMIT 10")
sql(s"CREATE TABLE $t AS SELECT * FROM src")
sql(s"CACHE TABLE $t")
dropTempTable(v)
```

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

The only user-facing change is that, previously `SQLContext.dropTempTable` may fail in the above scenario but will work with this fix.

### How was this patch tested?

Added new unit tests.

Closes #31136 from sunchao/SPARK-34076.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-13 13:22:21 +00:00
LantaoJin f1b21ba505 [SPARK-34064][SQL] Cancel the running broadcast sub-jobs when SQL statement is cancelled
### What changes were proposed in this pull request?
#24595 introduced `private val runId: UUID = UUID.randomUUID` in `BroadcastExchangeExec` to cancel the broadcast execution in the Future when timeout happens. Since the runId is a random UUID instead of inheriting the job group id, when a SQL statement is cancelled, these broadcast sub-jobs are still executing. This PR uses the job group id of the outside thread as its `runId` to abort these broadcast sub-jobs when the SQL statement is cancelled.

### Why are the changes needed?
When broadcasting a table takes too long and the SQL statement is cancelled. However, the background Spark job is still running and it wastes resources.

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

### How was this patch tested?
Manually test.
Since broadcasting a table is too fast to cancel in UT, but it is very easy to verify manually:
1. Start a Spark thrift-server with less resource in YARN.
2. When the driver is running but no executors are launched, submit a SQL which will broadcast tables from beeline.
3. Cancel the SQL in beeline

Without the patch, broadcast sub-jobs won't be cancelled.
![Screen Shot 2021-01-11 at 12 03 13 PM](https://user-images.githubusercontent.com/1853780/104150975-ab024b00-5416-11eb-8bf9-b5167bdad80a.png)

With this patch, broadcast sub-jobs will be cancelled.
![Screen Shot 2021-01-11 at 11 43 40 AM](https://user-images.githubusercontent.com/1853780/104150994-be151b00-5416-11eb-80ff-313d423c8a2e.png)

Closes #31119 from LantaoJin/SPARK-34064.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-13 12:58:27 +00:00
Kent Yao 04f031acb3 [SPARK-34086][SQL] RaiseError generates too much code and may fails codegen in length check for char varchar
### What changes were proposed in this pull request?

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133928/testReport/org.apache.spark.sql.execution/LogicalPlanTagInSparkPlanSuite/q41/

We can reduce more than 8000 bytes by removing the unnecessary CONCAT expression.

W/ this fix, for q41 in TPCDS with [Using TPCDS original definitions for char/varchar columns](https://github.com/apache/spark/pull/31012) applied, we can reduce the stage code-gen size from 22523 to 14369
```
14369  - 22523 = - 8154
```

### Why are the changes needed?

fix the perf regression(we need other improvements for q41 works), there will be a huge performance regression if codegen fails

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

no

### How was this patch tested?

modified uts

Closes #31150 from yaooqinn/SPARK-34086.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-13 09:52:36 +00:00
Max Gekk 861f8bb5fb [SPARK-34071][SQL][TESTS] Check stats of cached v1 tables after altering
### What changes were proposed in this pull request?
Port the test added by https://github.com/apache/spark/pull/31112 to:
1. v1 In-Memory catalog for `ALTER TABLE .. DROP PARTITION`
2. v1 In-Memory and Hive external catalogs for `ALTER TABLE .. ADD PARTITION`
3. v1 In-Memory and Hive external catalogs for `ALTER TABLE .. RENAME PARTITION`

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running the modified test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableDropPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableRenamePartitionSuite"
```

Closes #31131 from MaxGekk/cache-stats-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-13 04:58:01 +00:00
Takuya UESHIN ad8e40e2ab [SPARK-32338][SQL][PYSPARK][FOLLOW-UP][TEST] Add more tests for slice function
### What changes were proposed in this pull request?

This PR is a follow-up of #29138 and #29195 to add more tests for `slice` function.

### Why are the changes needed?

The original PRs are missing tests with column-based arguments instead of literals.

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

No.

### How was this patch tested?

Added tests and existing tests.

Closes #31159 from ueshin/issues/SPARK-32338/slice_tests.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-13 09:56:38 +09:00
yi.wu 0099715aae [SPARK-34091][SQL] Shuffle batch fetch should be able to disable after it's been enabled
### What changes were proposed in this pull request?

Fix the setting issue of shuffle batch fetch in `ShuffledRowRDD`.

### Why are the changes needed?

Currently, we can not disable the shuffle batch fetch mode once the batch fetch mode has been enabled. This PR fixes the issue to make `ShuffledRowRDD` respects the `spark.sql.adaptive.fetchShuffleBlocksInBatch` at runtime.

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

Yes. Before this PR, users can not disable batch fetch if they enabled first. After this PR, they can.

### How was this patch tested?

Added unit test.

Closes #31155 from Ngone51/fix-batchfetch-set-issue.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-12 15:45:15 +00:00
Max Gekk 6c047958f9 [SPARK-34084][SQL] Fix auto updating of table stats in ALTER TABLE .. ADD PARTITION
### What changes were proposed in this pull request?
Fix an issue in `ALTER TABLE .. ADD PARTITION` which happens when:
- A table doesn't have stats
- `spark.sql.statistics.size.autoUpdate.enabled` is `true`

In that case, `ALTER TABLE .. ADD PARTITION` does not update table stats automatically.

### Why are the changes needed?
The changes fix the issue demonstrated by the example:
```sql
spark-sql> create table tbl (col0 int, part int) partitioned by (part);
spark-sql> insert into tbl partition (part = 0) select 0;
spark-sql> set spark.sql.statistics.size.autoUpdate.enabled=true;
spark-sql> alter table tbl add partition (part = 1);
```
the `add partition` command should update table stats but it does not. There is no stats in the output of:
```
spark-sql> describe table extended tbl;
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, `ALTER TABLE .. ADD PARTITION` updates stats even when a table does have them before the command:
```sql
spark-sql> alter table tbl add partition (part = 1);
spark-sql> describe table extended tbl;
col0	int	NULL
part	int	NULL
# Partition Information
# col_name	data_type	comment
part	int	NULL

# Detailed Table Information
...
Statistics	2 bytes
```

### How was this patch tested?
By running new UT and existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableAddPartitionSuite"
```

Closes #31149 from MaxGekk/fix-stats-in-add-partition.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-12 14:34:17 +00:00
Kent Yao 99f84892a5 [SPARK-34003][SQL][FOLLOWUP] Avoid pushing modified Char/Varchar sort attributes into aggregate for existing ones
### What changes were proposed in this pull request?

In 0f8e5dd445, we partially fix the rule conflicts between `PaddingAndLengthCheckForCharVarchar` and `ResolveAggregateFunctions`, as error still exists in

sql like ```SELECT substr(v, 1, 2), sum(i) FROM t GROUP BY v ORDER BY substr(v, 1, 2)```

```sql
[info]   Failed to analyze query: org.apache.spark.sql.AnalysisException: expression 'spark_catalog.default.t.`v`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;
[info]   Project [substr(v, 1, 2)#100, sum(i)#101L]
[info]   +- Sort [aggOrder#102 ASC NULLS FIRST], true
[info]      +- !Aggregate [v#106], [substr(v#106, 1, 2) AS substr(v, 1, 2)#100, sum(cast(i#98 as bigint)) AS sum(i)#101L, substr(v#103, 1, 2) AS aggOrder#102
[info]         +- SubqueryAlias spark_catalog.default.t
[info]            +- Project [if ((length(v#97) <= 3)) v#97 else if ((length(rtrim(v#97, None)) > 3)) cast(raise_error(concat(input string of length , cast(length(v#97) as string),  exceeds varchar type length limitation: 3)) as string) else rpad(rtrim(v#97, None), 3,  ) AS v#106, i#98]
[info]               +- Relation[v#97,i#98] parquet
[info]
[info]   Project [substr(v, 1, 2)#100, sum(i)#101L]
[info]   +- Sort [aggOrder#102 ASC NULLS FIRST], true
[info]      +- !Aggregate [v#106], [substr(v#106, 1, 2) AS substr(v, 1, 2)#100, sum(cast(i#98 as bigint)) AS sum(i)#101L, substr(v#103, 1, 2) AS aggOrder#102
[info]         +- SubqueryAlias spark_catalog.default.t
[info]            +- Project [if ((length(v#97) <= 3)) v#97 else if ((length(rtrim(v#97, None)) > 3)) cast(raise_error(concat(input string of length , cast(length(v#97) as string),  exceeds varchar type length limitation: 3)) as string) else rpad(rtrim(v#97, None), 3,  ) AS v#106, i#98]
[info]               +- Relation[v#97,i#98] parquet

```
We need to look recursively into children to find char/varchars.

In this PR,  we try to resolve the full attributes including the original `Aggregate` expressions and the candidates in `SortOrder` together, then use the new re-resolved `Aggregate` expressions to determine which candidate in the `SortOrder` shall be pushed. This can avoid mismatch for the same attributes w/o this change, as the expressions returned by `executeSameContext` will change when `PaddingAndLengthCheckForCharVarchar` takes effects. W/ this change, the expressions can be matched correctly.

For those unmatched, w need to look recursively into children to find char/varchars instead of the expression itself only.

### Why are the changes needed?

bugfix

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

no
### How was this patch tested?

add new tests

Closes #31129 from yaooqinn/SPARK-34003-F.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-12 08:20:39 +00:00
Gengliang Wang 02a17e92f1 [SPARK-28646][SQL][FOLLOWUP] Add legacy config for allowing parameterless count
### What changes were proposed in this pull request?

Add a legacy configuration `spark.sql.legacy.allowParameterlessCount` in case users need the parameterless count.
This is a follow-up for https://github.com/apache/spark/pull/30541.

### Why are the changes needed?

There can be some users depends on the legacy behavior. We need a legacy flag for it.

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

Yes, adding a legacy flag `spark.sql.legacy.allowParameterlessCount`.

### How was this patch tested?

Unit tests

Closes #31143 from gengliangwang/countLegacy.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-12 16:31:22 +09:00
Max Gekk f7cbeec487 [SPARK-34074][SQL] Update stats only when table size changes
### What changes were proposed in this pull request?
Do not alter table stats if they are the same as in the catalog (at least since the recent retrieve).

### Why are the changes needed?
The changes reduce the number of calls to Hive external catalog.

### Does this PR introduce _any_ user-facing change?
Should not.

### How was this patch tested?
By running the modified test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```

Closes #31135 from MaxGekk/optimize-updateTableStats.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-12 03:28:28 +00:00
Dongjoon Hyun 3556929c43 [SPARK-33970][SQL][TEST][FOLLOWUP] Use String comparision
### What changes were proposed in this pull request?

This is a follow-up to replace `version.toDouble > 2` with `version >= "2.0"`

### Why are the changes needed?

`toDouble` has some assumption and can cause `java.lang.NumberFormatException`.

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

No.

### How was this patch tested?

Pass the CIs.

Closes #31134 from dongjoon-hyun/SPARK-33970-FOLLOWUP.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-11 13:40:03 -08:00
Liang-Chi Hsieh 0bcbafb4b8 [SPARK-34002][SQL] Fix the usage of encoder in ScalaUDF
### What changes were proposed in this pull request?

This patch fixes few issues when using encoders to serialize input/output in `ScalaUDF`.

### Why are the changes needed?

This fixes a bug when using encoders in Scala UDF. First, the output data type should be corrected to the corresponding data type of the object serializer. Second, `catalystConverter` should not serialize `Option[_]` as the ordinary row because in `ScalaUDF` case it is serialized to a column, not the top-level row. Otherwise, there will be a redundant `value` struct wrapping the serialized `Option[_]` object.

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

Yes, fixing a bug of `ScalaUDF`.

### How was this patch tested?

Unit test.

Closes #31103 from viirya/SPARK-34002.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-11 11:31:35 -08:00
yi.wu 4afca0f706 [SPARK-31952][SQL] Fix incorrect memory spill metric when doing Aggregate
### What changes were proposed in this pull request?

This PR takes over https://github.com/apache/spark/pull/28780.

1. Counted the spilled memory size when creating the `UnsafeExternalSorter` with the existing `InMemorySorter`

2. Accumulate the `totalSpillBytes` when merging two `UnsafeExternalSorter`

### Why are the changes needed?

As mentioned in https://github.com/apache/spark/pull/28780:

> It happends when hash aggregate downgrades to sort based aggregate.
`UnsafeExternalSorter.createWithExistingInMemorySorter` calls spill on an `InMemorySorter` immediately, but the memory pointed by `InMemorySorter` is acquired by outside `BytesToBytesMap`, instead the allocatedPages in `UnsafeExternalSorter`. So the memory spill bytes metric is always 0, but disk bytes spill metric is right.

Besides, this PR also fixes the `UnsafeExternalSorter.merge` by accumulating the `totalSpillBytes` of two sorters. Thus, we can report the correct spilled size in `HashAggregateExec.finishAggregate`.

Issues can be reproduced by the following step by checking the SQL metrics in UI:

```
bin/spark-shell --driver-memory 512m --executor-memory 512m --executor-cores 1 --conf "spark.default.parallelism=1"
scala> sql("select id, count(1) from range(10000000) group by id").write.csv("/tmp/result.json")
```

Before:

<img width="200" alt="WeChatfe5146180d91015e03b9a27852e9a443" src="https://user-images.githubusercontent.com/16397174/103625414-e6fc6280-4f75-11eb-8b93-c55095bdb5b8.png">

After:

<img width="200" alt="WeChat42ab0e73c5fbc3b14c12ab85d232071d" src="https://user-images.githubusercontent.com/16397174/103625420-e8c62600-4f75-11eb-8e1f-6f5e8ab561b9.png">

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

Yes, users can see the correct spill metrics after this PR.

### How was this patch tested?

Tested manually and added UTs.

Closes #31035 from Ngone51/SPARK-31952.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-11 07:15:28 +00:00
Max Gekk d97e99157e [SPARK-34060][SQL] Fix Hive table caching while updating stats by ALTER TABLE .. DROP PARTITION
### What changes were proposed in this pull request?
Fix canonicalisation of `HiveTableRelation` by normalisation of `CatalogTable`, and exclude table stats and temporary fields from the canonicalized plan.

### Why are the changes needed?
This fixes the issue demonstrated by the example below:
```scala
scala> spark.conf.set("spark.sql.statistics.size.autoUpdate.enabled", true)
scala> sql(s"CREATE TABLE tbl (id int, part int) USING hive PARTITIONED BY (part)")
scala> sql("INSERT INTO tbl PARTITION (part=0) SELECT 0")
scala> sql("INSERT INTO tbl PARTITION (part=1) SELECT 1")
scala> sql("CACHE TABLE tbl")
scala> sql("SELECT * FROM tbl").show(false)
+---+----+
|id |part|
+---+----+
|0  |0   |
|1  |1   |
+---+----+

scala> spark.catalog.isCached("tbl")
scala> sql("ALTER TABLE tbl DROP PARTITION (part=0)")
scala> spark.catalog.isCached("tbl")
res19: Boolean = false
```
`ALTER TABLE .. DROP PARTITION` must keep the table in the cache.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the drop partition command keeps the table in the cache while updating table stats:
```scala
scala> sql("ALTER TABLE tbl DROP PARTITION (part=0)")
scala> spark.catalog.isCached("tbl")
res19: Boolean = true
```

### How was this patch tested?
By running new UT in `AlterTableDropPartitionSuite`.

Closes #31112 from MaxGekk/fix-caching-hive-table-2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-11 07:03:44 +00:00
Max Gekk 664ef184c1 [SPARK-34055][SQL][TESTS][FOLLOWUP] Check partition adding to cached Hive table
### What changes were proposed in this pull request?
Replace `USING parquet` by `$defaultUsing` which is `USING parquet` for v1 In-Memory catalog and `USING hive` for v1 Hive external catalog.

### Why are the changes needed?
The PR https://github.com/apache/spark/pull/31101 added UT test but it checks only v1 In-Memory catalog. This PR runs this test for Hive external catalog as well to improve test coverage.

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

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableAddPartitionSuite"
```

Closes #31117 from MaxGekk/add-partition-refresh-cache-2-followup-2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-11 07:02:49 +00:00
Yuming Wang f77eeb0451 [SPARK-33970][SQL][TEST] Add test default partition in metastoredirectsql
### What changes were proposed in this pull request?

This pr add test default partition in metastoredirectsql.

### Why are the changes needed?

Improve test.

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

No.

### How was this patch tested?

N/A

Closes #31109 from wangyum/SPARK-33970.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-11 14:19:53 +09:00
Terry Kim 8391a4a687 [SPARK-34057][SQL] UnresolvedTableOrView should retain SQL text position for DDL commands
### What changes were proposed in this pull request?

Currently, there are many DDL commands where the position of the unresolved identifiers are incorrect:
```
scala> sql("DROP TABLE unknown")
org.apache.spark.sql.AnalysisException: Table or view not found: unknown; line 1 pos 0;
```
, whereas the `pos` should be `11`.

This PR proposes to fix this issue for commands using `UnresolvedTableOrView`:
```
DROP TABLE unknown
DESCRIBE TABLE unknown
ANALYZE TABLE unknown COMPUTE STATISTICS
ANALYZE TABLE unknown COMPUTE STATISTICS FOR COLUMNS col
ANALYZE TABLE unknown COMPUTE STATISTICS FOR ALL COLUMNS
SHOW CREATE TABLE unknown
REFRESH TABLE unknown
SHOW COLUMNS FROM unknown
SHOW COLUMNS FROM unknown IN db
ALTER TABLE unknown RENAME TO t
ALTER VIEW unknown RENAME TO v
```

### Why are the changes needed?

To fix a bug.

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

Yes, now the above example will print the following:
```
org.apache.spark.sql.AnalysisException: Table or view not found: unknown; line 1 pos 11;
```

### How was this patch tested?

Add a new test.

Closes #31106 from imback82/unresolved_table_or_view_message.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-11 04:28:39 +00:00
HyukjinKwon 830249284d [SPARK-34059][SQL][CORE] Use for/foreach rather than map to make sure execute it eagerly
### What changes were proposed in this pull request?

This PR is basically a followup of https://github.com/apache/spark/pull/14332.
Calling `map` alone might leave it not executed due to lazy evaluation, e.g.)

```
scala> val foo = Seq(1,2,3)
foo: Seq[Int] = List(1, 2, 3)

scala> foo.map(println)
1
2
3
res0: Seq[Unit] = List((), (), ())

scala> foo.view.map(println)
res1: scala.collection.SeqView[Unit,Seq[_]] = SeqViewM(...)

scala> foo.view.foreach(println)
1
2
3
```

We should better use `foreach` to make sure it's executed where the output is unused or `Unit`.

### Why are the changes needed?

To prevent the potential issues by not executing `map`.

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

No, the current codes look not causing any problem for now.

### How was this patch tested?

I found these item by running IntelliJ inspection, double checked one by one, and fixed them. These should be all instances across the codebase ideally.

Closes #31110 from HyukjinKwon/SPARK-34059.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-01-10 15:22:24 -08:00
Max Gekk 9a8d275226 [SPARK-34055][SQL][TESTS][FOLLOWUP] Increase the expected number of calls to Hive external catalog in partition adding
### What changes were proposed in this pull request?
Increase the number of calls to Hive external catalog in the test for `ALTER TABLE .. ADD PARTITION`.

### Why are the changes needed?
There is a logical conflict between https://github.com/apache/spark/pull/31101 and https://github.com/apache/spark/pull/31092. The first one fixes a caching issue and increases the number of calls to Hive external catalog.

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

### How was this patch tested?
By running the modified test:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableAddPartitionSuite"
```

Closes #31111 from MaxGekk/add-partition-refresh-cache-2-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-10 18:29:02 +09:00
ulysses-you 48b9611ba3 [SPARK-32668][SQL] HiveGenericUDTF initialize UDTF should use StructObjectInspector method
### What changes were proposed in this pull request?

Use `initialize(StructObjectInspector argOIs)` instead `initialize(ObjectInspector[] args)` in `HiveGenericUDTF`.

### Why are the changes needed?

In our case, we implement a Hive `GenericUDTF` and override `initialize(StructObjectInspector argOIs)`. Then it's ok to execute with Hive, but failed with Spark SQL. Here is the Spark SQL error msg:
```
No handler for UDF/UDAF/UDTF 'com.xxxx.xxxUDTF': java.lang.IllegalStateException:
Should not be called directly Please make sure your function overrides
`public StructObjectInspector initialize(ObjectInspector[] args)`.
```

The reason is Spark `HiveGenericUDTF` call `initialize(ObjectInspector[] argOIs)` to init a UDTF, but it's a Deprecated method.
```
    public StructObjectInspector initialize(StructObjectInspector argOIs) throws UDFArgumentException {
        List<? extends StructField> inputFields = argOIs.getAllStructFieldRefs();
        ObjectInspector[] udtfInputOIs = new ObjectInspector[inputFields.size()];

        for(int i = 0; i < inputFields.size(); ++i) {
            udtfInputOIs[i] = ((StructField)inputFields.get(i)).getFieldObjectInspector();
        }

        return this.initialize(udtfInputOIs);
    }

    Deprecated
    public StructObjectInspector initialize(ObjectInspector[] argOIs) throws UDFArgumentException {
        throw new IllegalStateException("Should not be called directly");
    }
```

We should use `initialize(StructObjectInspector argOIs)` to do this so that we can be compatible both of the two method. Same as Hive.

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

Yes, fix UDTF initialize method.

### How was this patch tested?

manual test and passed `HiveUDFDynamicLoadSuite`

Closes #29490 from ulysses-you/SPARK-32668.

Lead-authored-by: ulysses-you <ulyssesyou18@gmail.com>
Co-authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-01-10 13:19:04 +08:00
Max Gekk e0e06c18fd [SPARK-34055][SQL] Refresh cache in ALTER TABLE .. ADD PARTITION
### What changes were proposed in this pull request?
Invoke `refreshTable()` from `CatalogImpl` which refreshes the cache in v1 `ALTER TABLE .. ADD PARTITION`.

### Why are the changes needed?
This fixes the issues portrayed by the example:
```sql
spark-sql> create table tbl (col int, part int) using parquet partitioned by (part);
spark-sql> insert into tbl partition (part=0) select 0;
spark-sql> cache table tbl;
spark-sql> select * from tbl;
0	0
spark-sql> show table extended like 'tbl' partition(part=0);
default	tbl	false	Partition Values: [part=0]
Location: file:/Users/maximgekk/proj/add-partition-refresh-cache-2/spark-warehouse/tbl/part=0
...
```
Create new partition by copying the existing one:
```
$ cp -r /Users/maximgekk/proj/add-partition-refresh-cache-2/spark-warehouse/tbl/part=0 /Users/maximgekk/proj/add-partition-refresh-cache-2/spark-warehouse/tbl/part=1
```
```sql
spark-sql> alter table tbl add partition (part=1) location '/Users/maximgekk/proj/add-partition-refresh-cache-2/spark-warehouse/tbl/part=1';
spark-sql> select * from tbl;
0	0
```

The last query must return `0	1` since it has been added by `ALTER TABLE .. ADD PARTITION`.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes for the example above:
```sql
...
spark-sql> alter table tbl add partition (part=1) location '/Users/maximgekk/proj/add-partition-refresh-cache-2/spark-warehouse/tbl/part=1';
spark-sql> select * from tbl;
0	0
0	1
```

### How was this patch tested?
By running the affected test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
```

Closes #31101 from MaxGekk/add-partition-refresh-cache-2.

Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-10 14:06:17 +09:00
HyukjinKwon 105ba6e5f0 Revert "[SPARK-33933][SQL] Materialize BroadcastQueryStage first to avoid broadcast timeout in AQE"
This reverts commit d36cdd5541.
2021-01-10 13:52:48 +09:00
ulysses-you 48cd11c483 [SPARK-34030][SQL] Fold RepartitionByExpression num partition should at Optimizer
### What changes were proposed in this pull request?

Move `RepartitionByExpression` fold partition number code to a new rule at `Optimizer`.

### Why are the changes needed?

We meet some ploblem when backport SPARK-33806. It is because the UnresolvedFunction.foldable will throw a exception. It's ok with master branch, but it's better to do it at Optimizer. Some reason:

1. It's not always safe to call Expression.foldable before analysis.
2. fold num partition to 1 more like a optimize behavior.

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

No.

### How was this patch tested?

Add test.

Closes #31077 from ulysses-you/SPARK-34030.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-10 13:00:40 +09:00
Max Gekk 0af387480c [SPARK-34048][SQL][TESTS] Check the amount of calls to Hive external catalog
### What changes were proposed in this pull request?
Add new tests to unified test suites to check the total amount of calls via the Hive client.

### Why are the changes needed?
1. To improve test coverage
2. To make foundation for future optimizations

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

### How was this patch tested?
By running the affected test suites like:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```

Closes #31092 from MaxGekk/access-to-catalog-refreshTable.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-09 15:33:08 -08:00
Anton Okolnychyi 6b34745cb9 [SPARK-34049][SS] DataSource V2: Use Write abstraction in StreamExecution
### What changes were proposed in this pull request?

This PR makes `StreamExecution` use the `Write` abstraction introduced in SPARK-33779.

Note: we will need separate plans for streaming writes in order to support the required distribution and ordering in SS. This change only migrates to the `Write` abstraction.

### Why are the changes needed?

These changes prevent exceptions from data sources that implement only the `build` method in `WriteBuilder`.

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

No.

### How was this patch tested?

Existing tests.

Closes #31093 from aokolnychyi/spark-34049.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-08 20:37:35 -08:00
Kousuke Saruta 0781ed4f5b [MINOR][SQL][TESTS] Fix the incorrect unicode escape test in ParserUtilsSuite
### What changes were proposed in this pull request?

This PR fixes an incorrect unicode literal test in `ParserUtilsSuite`.
In that suite, string literals in queries have unicode escape characters like `\u7328` but the backslash should be escaped because
the queriy strings are given as Java strings.

### Why are the changes needed?

Correct the test.

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

No.

### How was this patch tested?

Run `ParserUtilsSuite` and it passed.

Closes #31088 from sarutak/fix-incorrect-unicode-test.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-08 09:44:33 -06:00
Max Gekk 157b72ac9f [SPARK-33591][SQL] Recognize null in partition spec values
### What changes were proposed in this pull request?
1. Recognize `null` while parsing partition specs, and put `null` instead of `"null"` as partition values.
2. For V1 catalog: replace `null` by `__HIVE_DEFAULT_PARTITION__`.
3. For V2 catalogs: pass `null` AS IS, and let catalog implementations to decide how to handle `null`s as partition values in spec.

### Why are the changes needed?
Currently, `null` in partition specs is recognized as the `"null"` string which could lead to incorrect results, for example:
```sql
spark-sql> CREATE TABLE tbl5 (col1 INT, p1 STRING) USING PARQUET PARTITIONED BY (p1);
spark-sql> INSERT INTO TABLE tbl5 PARTITION (p1 = null) SELECT 0;
spark-sql> SELECT isnull(p1) FROM tbl5;
false
```
Even we inserted a row to the partition with the `null` value, **the resulted table doesn't contain `null`**.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above works as expected:
```sql
spark-sql> SELECT isnull(p1) FROM tbl5;
true
```

### How was this patch tested?
1. By running the affected test suites `SQLQuerySuite`, `AlterTablePartitionV2SQLSuite` and `v1/ShowPartitionsSuite`.
2. Compiling by Scala 2.13:
```
$  ./dev/change-scala-version.sh 2.13
$ ./build/sbt -Pscala-2.13 compile
```

Closes #30538 from MaxGekk/partition-spec-value-null.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-08 14:14:27 +00:00
Kent Yao 0f8e5dd445 [SPARK-34003][SQL] Fix Rule conflicts between PaddingAndLengthCheckForCharVarchar and ResolveAggregateFunctions
### What changes were proposed in this pull request?

ResolveAggregateFunctions is a hacky rule and it calls `executeSameContext` to generate a `resolved agg` to determine which unresolved sort attribute should be pushed into the agg. However, after we add the PaddingAndLengthCheckForCharVarchar rule which will rewrite the query output, thus, the `resolved agg` cannot match original attributes anymore.

It causes some dissociative sort attribute to be pushed in and fails the query

``` logtalk
[info]   Failed to analyze query: org.apache.spark.sql.AnalysisException: expression 'testcat.t1.`v`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;
[info]   Project [v#14, sum(i)#11L]
[info]   +- Sort [aggOrder#12 ASC NULLS FIRST], true
[info]      +- !Aggregate [v#14], [v#14, sum(cast(i#7 as bigint)) AS sum(i)#11L, v#13 AS aggOrder#12]
[info]         +- SubqueryAlias testcat.t1
[info]            +- Project [if ((length(v#6) <= 3)) v#6 else if ((length(rtrim(v#6, None)) > 3)) cast(raise_error(concat(input string of length , cast(length(v#6) as string),  exceeds varchar type length limitation: 3)) as string) else rpad(rtrim(v#6, None), 3,  ) AS v#14, i#7]
[info]               +- RelationV2[v#6, i#7, index#15, _partition#16] testcat.t1
[info]
[info]   Project [v#14, sum(i)#11L]
[info]   +- Sort [aggOrder#12 ASC NULLS FIRST], true
[info]      +- !Aggregate [v#14], [v#14, sum(cast(i#7 as bigint)) AS sum(i)#11L, v#13 AS aggOrder#12]
[info]         +- SubqueryAlias testcat.t1
[info]            +- Project [if ((length(v#6) <= 3)) v#6 else if ((length(rtrim(v#6, None)) > 3)) cast(raise_error(concat(input string of length , cast(length(v#6) as string),  exceeds varchar type length limitation: 3)) as string) else rpad(rtrim(v#6, None), 3,  ) AS v#14, i#7]
[info]               +- RelationV2[v#6, i#7, index#15, _partition#16] testcat.t1
```

### Why are the changes needed?

bugfix
### Does this PR introduce _any_ user-facing change?

no
### How was this patch tested?

new tests

Closes #31027 from yaooqinn/SPARK-34003.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-08 09:05:22 +00:00
Gengliang Wang b95a847ce1 [SPARK-34046][SQL][TESTS] Use join hint for constructing joins in JoinSuite and WholeStageCodegenSuite
### What changes were proposed in this pull request?

There are some existing test cases that constructing various joins by tuning the SQL configuration AUTO_BROADCASTJOIN_THRESHOLD, PREFER_SORTMERGEJOIN,SHUFFLE_PARTITIONS, etc.

This can be tricky and not straight-forward. In the future development we might have to tweak the configurations again .
This PR is to construct specific joins by using join hint in test cases.
### Why are the changes needed?

Make test cases for join simpler and more robust.

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

No

### How was this patch tested?

Unit test

Closes #31087 from gengliangwang/joinhintInTest.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-08 07:52:39 +00:00
Chao Sun 0de7f2ff1e [SPARK-34039][SQL] ReplaceTable should invalidate cache
### What changes were proposed in this pull request?

This changes `ReplaceTableExec`/`AtomicReplaceTableExec`, and uncaches the target table before it is dropped. In addition, this includes some refactoring by moving the `uncacheTable` method to `DataSourceV2Strategy` so that we don't need to pass a Spark session to the v2 exec.

### Why are the changes needed?

Similar to SPARK-33492 (#30429). When a table is refreshed, the associated cache should be invalidated to avoid potential incorrect results.

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

Yes. Now When a data source v2 is cached (either directly or indirectly), all the relevant caches will be refreshed or invalidated if the table is replaced.

### How was this patch tested?

Added a new unit test.

Closes #31081 from sunchao/SPARK-34039.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-07 21:13:22 -08:00
fwang12 7b06acc28b [SPARK-33100][SQL][FOLLOWUP] Find correct bound of bracketed comment in spark-sql
### What changes were proposed in this pull request?

This PR help find correct bound of bracketed comment in spark-sql.

Here is the log for UT of SPARK-33100 in CliSuite before:
```
2021-01-05 13:22:34.768 - stdout> spark-sql> /* SELECT 'test';*/ SELECT 'test';
2021-01-05 13:22:41.523 - stderr> Time taken: 6.716 seconds, Fetched 1 row(s)
2021-01-05 13:22:41.599 - stdout> test
2021-01-05 13:22:41.6 - stdout> spark-sql> ;;/* SELECT 'test';*/ SELECT 'test';
2021-01-05 13:22:41.709 - stdout> test
2021-01-05 13:22:41.709 - stdout> spark-sql> /* SELECT 'test';*/;; SELECT 'test';
2021-01-05 13:22:41.902 - stdout> spark-sql> SELECT 'test'; -- SELECT 'test';
2021-01-05 13:22:41.902 - stderr> Time taken: 0.129 seconds, Fetched 1 row(s)
2021-01-05 13:22:41.902 - stderr> Error in query:
2021-01-05 13:22:41.902 - stderr> mismatched input '<EOF>' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 19)
2021-01-05 13:22:42.006 - stderr>
2021-01-05 13:22:42.006 - stderr> == SQL ==
2021-01-05 13:22:42.006 - stderr> /* SELECT 'test';*/
2021-01-05 13:22:42.006 - stderr> -------------------^^^
2021-01-05 13:22:42.006 - stderr>
2021-01-05 13:22:42.006 - stderr> Time taken: 0.226 seconds, Fetched 1 row(s)
2021-01-05 13:22:42.006 - stdout> test
```
The root cause is that the insideBracketedComment is not accurate.

For `/* comment */`, the last character `/` is not insideBracketedComment and it would be treat as beginning of statements.

In this PR, this issue is fixed.

### Why are the changes needed?
To fix the issue described above.

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

### How was this patch tested?
Existing UT

Closes #31054 from turboFei/SPARK-33100-followup.

Authored-by: fwang12 <fwang12@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-01-07 20:49:37 +09:00
Yu Zhong d36cdd5541 [SPARK-33933][SQL] Materialize BroadcastQueryStage first to avoid broadcast timeout in AQE
### What changes were proposed in this pull request?
In AdaptiveSparkPlanExec.getFinalPhysicalPlan, when newStages are generated, sort the new stages by class type to make sure BroadcastQueryState precede others.
It can make sure the broadcast job are submitted before map jobs to avoid waiting for job schedule and cause broadcast timeout.

### Why are the changes needed?
When enable AQE, in getFinalPhysicalPlan, spark traversal the physical plan bottom up and create query stage for materialized part by createQueryStages and materialize those new created query stages to submit map stages or broadcasting. When ShuffleQueryStage are materializing before BroadcastQueryStage, the map job and broadcast job are submitted almost at the same time, but map job will hold all the computing resources. If the map job runs slow (when lots of data needs to process and the resource is limited), the broadcast job cannot be started(and finished) before spark.sql.broadcastTimeout, thus cause whole job failed (introduced in SPARK-31475).
The workaround to increase spark.sql.broadcastTimeout doesn't make sense and graceful, because the data to broadcast is very small.

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

### How was this patch tested?
1. Add UT
2. Test the code using dev environment in https://issues.apache.org/jira/browse/SPARK-33933

Closes #30998 from zhongyu09/aqe-broadcast.

Authored-by: Yu Zhong <yzhong@freewheel.tv>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-07 08:59:26 +00:00
Dongjoon Hyun 194edc86a2 Revert "[SPARK-34029][SQL][TESTS] Add OrcEncryptionSuite and FakeKeyProvider"
This reverts commit 8bb70bf0d6.
2021-01-06 23:41:27 -08:00
Yuming Wang aa509c1eee [SPARK-34031][SQL] Union operator missing rowCount when CBO enabled
### What changes were proposed in this pull request?

This pr add row count to `Union` operator when CBO enabled.
```scala
spark.sql("CREATE TABLE t1 USING parquet AS SELECT id FROM RANGE(10)")
spark.sql("CREATE TABLE t2 USING parquet AS SELECT id FROM RANGE(10)")
spark.sql("ANALYZE TABLE t1 COMPUTE STATISTICS FOR ALL COLUMNS")
spark.sql("ANALYZE TABLE t2 COMPUTE STATISTICS FOR ALL COLUMNS")
spark.sql("set spark.sql.cbo.enabled=true")
spark.sql("SELECT * FROM t1 UNION ALL SELECT * FROM t2").explain("cost")
```

Before this pr:
```
== Optimized Logical Plan ==
Union false, false, Statistics(sizeInBytes=320.0 B)
:- Relation[id#5880L] parquet, Statistics(sizeInBytes=160.0 B, rowCount=10)
+- Relation[id#5881L] parquet, Statistics(sizeInBytes=160.0 B, rowCount=10)
```

After this pr:
```
== Optimized Logical Plan ==
Union false, false, Statistics(sizeInBytes=320.0 B, rowCount=20)
:- Relation[id#2138L] parquet, Statistics(sizeInBytes=160.0 B, rowCount=10)
+- Relation[id#2139L] parquet, Statistics(sizeInBytes=160.0 B, rowCount=10)
```

### Why are the changes needed?

Improve query performance,  [`JoinEstimation.estimateInnerOuterJoin`](d6a68e0b67/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala (L55-L156)) need the row count.

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

No.

### How was this patch tested?

Unit test.

Closes #31068 from wangyum/SPARK-34031.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-07 14:41:10 +09:00
Yuming Wang 3aa4e113c5 [SPARK-33861][SQL][FOLLOWUP] Simplify conditional in predicate should consider deterministic
### What changes were proposed in this pull request?

This pr address https://github.com/apache/spark/pull/30865#pullrequestreview-562344089 to fix simplify conditional in predicate should consider deterministic.

### Why are the changes needed?

Fix bug.

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

No.

### How was this patch tested?

Unit test.

Closes #31067 from wangyum/SPARK-33861-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-07 14:28:30 +09:00
yangjie01 26b603992c [SPARK-34028][SQL] Cleanup "unreachable code" compilation warning
### What changes were proposed in this pull request?
There is one compilation warning as follow:

```
[WARNING] [Warn] /spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1555: [other-match-analysis  org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupFunction.catalogFunction] unreachable code
```

This compilation warning is due to `NoSuchPermanentFunctionException` is sub-class of `AnalysisException` and if there is `NoSuchPermanentFunctionException` be thrown out,  it will be catch by `case _: AnalysisException => failFunctionLookup(name)`,  so `case _: NoSuchPermanentFunctionException => failFunctionLookup(name)` is `unreachable code`.

This pr remove `case _: NoSuchPermanentFunctionException => failFunctionLookup(name)` directly because both these 2 branches handle exceptions in the same way: `failFunctionLookup(name)`

### Why are the changes needed?
Cleanup "unreachable code" compilation warnings.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31064 from LuciferYang/SPARK-34028.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-07 14:26:04 +09:00
ulysses-you f9daf035f4 [SPARK-33806][SQL][FOLLOWUP] Fold RepartitionExpression num partition should check if partition expression is empty
### What changes were proposed in this pull request?

Add check partition expressions is empty.

### Why are the changes needed?

We should keep `spark.range(1).hint("REPARTITION_BY_RANGE")` has default shuffle number instead of 1.

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

Yes.

### How was this patch tested?

Add test.

Closes #31074 from ulysses-you/SPARK-33806-FOLLOWUP.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-06 17:22:14 -08:00
Dongjoon Hyun 8bb70bf0d6 [SPARK-34029][SQL][TESTS] Add OrcEncryptionSuite and FakeKeyProvider
### What changes were proposed in this pull request?

This PR aims to add a basis for columnar encryption test framework by add `OrcEncryptionSuite` and `FakeKeyProvider`.

Please note that we will improve more in both Apache Spark and Apache ORC in Apache Spark 3.2.0 timeframe.

### Why are the changes needed?

Apache ORC 1.6 supports columnar encryption.

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

No. This is for a test case.

### How was this patch tested?

Pass the newly added test suite.

Closes #31065 from dongjoon-hyun/SPARK-34029.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-06 12:59:47 -08:00
Kazuaki Ishizaki a0269bb419 [SPARK-34022][DOCS][FOLLOW-UP] Fix typo in SQL built-in function docs
### What changes were proposed in this pull request?

This PR is a follow-up of #31061. It fixes a typo in a document: `Finctions` -> `Functions`

### Why are the changes needed?

Make the change better documented.

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

No

### How was this patch tested?

N/A

Closes #31069 from kiszk/SPARK-34022-followup.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-06 09:28:22 -08:00
HyukjinKwon 0d86a02ffb [SPARK-34022][DOCS] Support latest mkdocs in SQL built-in function docs
### What changes were proposed in this pull request?

This PR adds the support of the latest mkdocs, and makes the sidebar properly show. It works in lower versions too.

Before:

![Screen Shot 2021-01-06 at 5 11 56 PM](https://user-images.githubusercontent.com/6477701/103745131-4e7fe400-5042-11eb-9c09-84f9f95e9fb9.png)

After:

![Screen Shot 2021-01-06 at 5 10 53 PM](https://user-images.githubusercontent.com/6477701/103745139-5049a780-5042-11eb-8ded-30b6f7ef48aa.png)

### Why are the changes needed?

This is a regression in the documentation.

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

Technically no. It's not related yet. It fixes the list on the sidebar appears properly.

### How was this patch tested?

Manually built the docs via `./sql/create-docs.sh` and `open ./sql/site/index.html`

Closes #31061 from HyukjinKwon/SPARK-34022.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-06 20:31:27 +09:00