Commit graph

8968 commits

Author SHA1 Message Date
yi.wu da076153aa [SPARK-30433][SQL] Make conflict attributes resolution more scalable in ResolveReferences
### What changes were proposed in this pull request?

This PR tries to make conflict attributes resolution in `ResolveReferences` more scalable by doing resolution in batch way.

### Why are the changes needed?

Currently, `ResolveReferences` rule only resolves conflict attributes of one single conflict plan pair in one iteration, which can be inefficient when there're many conflicts.

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

No.

### How was this patch tested?

Covered by existed tests.

Closes #27105 from Ngone51/resolve-conflict-columns-in-batch.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 12:05:27 +08:00
Yuming Wang 17881a467a [SPARK-19784][SPARK-25403][SQL] Refresh the table even table stats is empty
## What changes were proposed in this pull request?

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

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

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

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

## How was this patch tested?

unit tests

Closes #22721 from wangyum/SPARK-25403.

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

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

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

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

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

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

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

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

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

### How was this patch tested?
Manuel Tested

Closes #26437 from AngersZhuuuu/SPARK-29800.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-06 22:54:37 +08:00
root1 ebd2fd7e02 [SPARK-30415][SQL] Improve Readability of SQLConf Doc
### What changes were proposed in this pull request?
SQLCOnf Doc updated.

### Why are the changes needed?
Some doc comments were not written properly. Space was missing at many places. This patch updates the doc.

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

### How was this patch tested?
Documentation update.

Closes #27091 from iRakson/SQLConfDoc.

Authored-by: root1 <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-04 15:49:11 -06:00
Wenchen Fan be4faafee4 Revert "[SPARK-23264][SQL] Make INTERVAL keyword optional when ANSI enabled"
### What changes were proposed in this pull request?

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

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

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

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

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

### How was this patch tested?

existing tests

Closes #27080 from cloud-fan/interval.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-03 12:51:10 -08:00
Yuming Wang 568ad4e77a [SPARK-29947][SQL] Improve ResolveRelations performance
### What changes were proposed in this pull request?

It is very common for a SQL query to query a table more than once. For example:
```
== Physical Plan ==
*(12) HashAggregate(keys=[cmn_mtrc_summ_dt#21, rev_rollup#1279, CASE WHEN (rev_rollup#1319 = rev_rollup#1279) THEN 0 ELSE 1 END#1366, CASE WHEN cast(sap_category_id#24 as decimal(10,0)) IN (5,7,23,41) THEN 0 ELSE 1 END#1367], functions=[sum(coalesce(bid_count#34, 0)), sum(coalesce(ck_trans_count#35, 0)), sum(coalesce(ended_bid_count#36, 0)), sum(coalesce(ended_lstg_count#37, 0)), sum(coalesce(ended_success_lstg_count#38, 0)), sum(coalesce(item_sold_count#39, 0)), sum(coalesce(new_lstg_count#40, 0)), sum(coalesce(gmv_us_amt#41, 0.00)), sum(coalesce(gmv_slr_lc_amt#42, 0.00)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_insrtn_fee_us_amt#46, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_insrtn_crd_us_amt#50, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_fetr_fee_us_amt#54, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_fetr_crd_us_amt#58, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_fv_fee_us_amt#62, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_fv_crd_us_amt#67, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_othr_l_fee_us_amt#72, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_othr_l_crd_us_amt#76, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_othr_nl_fee_us_amt#80, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_othr_nl_crd_us_amt#84, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_slr_tools_fee_us_amt#88, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_slr_tools_crd_us_amt#92, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(coalesce(rvnu_unasgnd_us_amt#96, 0.000000)), sum((coalesce(rvnu_transaction_us_amt#112, 0.0) + coalesce(rvnu_transaction_crd_us_amt#115, 0.0))), sum((coalesce(rvnu_total_us_amt#118, 0.0) + coalesce(rvnu_total_crd_us_amt#121, 0.0)))])
+- Exchange hashpartitioning(cmn_mtrc_summ_dt#21, rev_rollup#1279, CASE WHEN (rev_rollup#1319 = rev_rollup#1279) THEN 0 ELSE 1 END#1366, CASE WHEN cast(sap_category_id#24 as decimal(10,0)) IN (5,7,23,41) THEN 0 ELSE 1 END#1367, 200), true, [id=#403]
   +- *(11) HashAggregate(keys=[cmn_mtrc_summ_dt#21, rev_rollup#1279, CASE WHEN (rev_rollup#1319 = rev_rollup#1279) THEN 0 ELSE 1 END AS CASE WHEN (rev_rollup#1319 = rev_rollup#1279) THEN 0 ELSE 1 END#1366, CASE WHEN cast(sap_category_id#24 as decimal(10,0)) IN (5,7,23,41) THEN 0 ELSE 1 END AS CASE WHEN cast(sap_category_id#24 as decimal(10,0)) IN (5,7,23,41) THEN 0 ELSE 1 END#1367], functions=[partial_sum(coalesce(bid_count#34, 0)), partial_sum(coalesce(ck_trans_count#35, 0)), partial_sum(coalesce(ended_bid_count#36, 0)), partial_sum(coalesce(ended_lstg_count#37, 0)), partial_sum(coalesce(ended_success_lstg_count#38, 0)), partial_sum(coalesce(item_sold_count#39, 0)), partial_sum(coalesce(new_lstg_count#40, 0)), partial_sum(coalesce(gmv_us_amt#41, 0.00)), partial_sum(coalesce(gmv_slr_lc_amt#42, 0.00)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_insrtn_fee_us_amt#46, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_insrtn_crd_us_amt#50, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_fetr_fee_us_amt#54, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_fetr_crd_us_amt#58, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_fv_fee_us_amt#62, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_fv_crd_us_amt#67, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_othr_l_fee_us_amt#72, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_othr_l_crd_us_amt#76, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_othr_nl_fee_us_amt#80, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_othr_nl_crd_us_amt#84, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_slr_tools_fee_us_amt#88, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_slr_tools_crd_us_amt#92, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(coalesce(rvnu_unasgnd_us_amt#96, 0.000000)), partial_sum((coalesce(rvnu_transaction_us_amt#112, 0.0) + coalesce(rvnu_transaction_crd_us_amt#115, 0.0))), partial_sum((coalesce(rvnu_total_us_amt#118, 0.0) + coalesce(rvnu_total_crd_us_amt#121, 0.0)))])
      +- *(11) Project [cmn_mtrc_summ_dt#21, sap_category_id#24, bid_count#34, ck_trans_count#35, ended_bid_count#36, ended_lstg_count#37, ended_success_lstg_count#38, item_sold_count#39, new_lstg_count#40, gmv_us_amt#41, gmv_slr_lc_amt#42, rvnu_insrtn_fee_us_amt#46, rvnu_insrtn_crd_us_amt#50, rvnu_fetr_fee_us_amt#54, rvnu_fetr_crd_us_amt#58, rvnu_fv_fee_us_amt#62, rvnu_fv_crd_us_amt#67, rvnu_othr_l_fee_us_amt#72, rvnu_othr_l_crd_us_amt#76, rvnu_othr_nl_fee_us_amt#80, rvnu_othr_nl_crd_us_amt#84, rvnu_slr_tools_fee_us_amt#88, rvnu_slr_tools_crd_us_amt#92, rvnu_unasgnd_us_amt#96, ... 6 more fields]
         +- *(11) BroadcastHashJoin [byr_cntry_id#23], [cntry_id#1309], LeftOuter, BuildRight
            :- *(11) Project [cmn_mtrc_summ_dt#21, byr_cntry_id#23, sap_category_id#24, bid_count#34, ck_trans_count#35, ended_bid_count#36, ended_lstg_count#37, ended_success_lstg_count#38, item_sold_count#39, new_lstg_count#40, gmv_us_amt#41, gmv_slr_lc_amt#42, rvnu_insrtn_fee_us_amt#46, rvnu_insrtn_crd_us_amt#50, rvnu_fetr_fee_us_amt#54, rvnu_fetr_crd_us_amt#58, rvnu_fv_fee_us_amt#62, rvnu_fv_crd_us_amt#67, rvnu_othr_l_fee_us_amt#72, rvnu_othr_l_crd_us_amt#76, rvnu_othr_nl_fee_us_amt#80, rvnu_othr_nl_crd_us_amt#84, rvnu_slr_tools_fee_us_amt#88, rvnu_slr_tools_crd_us_amt#92, ... 6 more fields]
            :  +- *(11) BroadcastHashJoin [slr_cntry_id#28], [cntry_id#1269], LeftOuter, BuildRight
            :     :- *(11) Project [gen_attr_1#360 AS cmn_mtrc_summ_dt#21, gen_attr_5#267 AS byr_cntry_id#23, gen_attr_7#268 AS sap_category_id#24, gen_attr_15#272 AS slr_cntry_id#28, gen_attr_27#278 AS bid_count#34, gen_attr_29#279 AS ck_trans_count#35, gen_attr_31#280 AS ended_bid_count#36, gen_attr_33#282 AS ended_lstg_count#37, gen_attr_35#283 AS ended_success_lstg_count#38, gen_attr_37#284 AS item_sold_count#39, gen_attr_39#281 AS new_lstg_count#40, gen_attr_41#285 AS gmv_us_amt#41, gen_attr_43#287 AS gmv_slr_lc_amt#42, gen_attr_51#290 AS rvnu_insrtn_fee_us_amt#46, gen_attr_59#294 AS rvnu_insrtn_crd_us_amt#50, gen_attr_67#298 AS rvnu_fetr_fee_us_amt#54, gen_attr_75#302 AS rvnu_fetr_crd_us_amt#58, gen_attr_83#306 AS rvnu_fv_fee_us_amt#62, gen_attr_93#311 AS rvnu_fv_crd_us_amt#67, gen_attr_103#316 AS rvnu_othr_l_fee_us_amt#72, gen_attr_111#320 AS rvnu_othr_l_crd_us_amt#76, gen_attr_119#324 AS rvnu_othr_nl_fee_us_amt#80, gen_attr_127#328 AS rvnu_othr_nl_crd_us_amt#84, gen_attr_135#332 AS rvnu_slr_tools_fee_us_amt#88, ... 6 more fields]
            :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_308#777 as decimal(20,0))], [cast(gen_attr_309#803 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 6 more fields]
            :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_310#674 as int)], [cast(gen_attr_311#774 as int)], LeftOuter, BuildRight
            :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 6 more fields]
            :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_5#267 as decimal(20,0))], [cast(gen_attr_312#665 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 5 more fields]
            :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_313#565 as decimal(20,0))], [cast(gen_attr_314#591 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 6 more fields]
            :     :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_315#462 as int)], [cast(gen_attr_316#562 as int)], LeftOuter, BuildRight
            :     :     :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 6 more fields]
            :     :     :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_15#272 as decimal(20,0))], [cast(gen_attr_317#453 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 5 more fields]
            :     :     :     :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_25#277 as decimal(20,0))], [cast(gen_attr_318#379 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_25#277, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, ... 6 more fields]
            :     :     :     :     :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_23#276 as decimal(20,0))], [cast(gen_attr_319#367 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :     :     :     :     :     :- *(11) Project [byr_cntry_id#1169 AS gen_attr_5#267, sap_category_id#1170 AS gen_attr_7#268, slr_cntry_id#1174 AS gen_attr_15#272, lstg_curncy_id#1178 AS gen_attr_23#276, blng_curncy_id#1179 AS gen_attr_25#277, bid_count#1180 AS gen_attr_27#278, ck_trans_count#1181 AS gen_attr_29#279, ended_bid_count#1182 AS gen_attr_31#280, new_lstg_count#1183 AS gen_attr_39#281, ended_lstg_count#1184 AS gen_attr_33#282, ended_success_lstg_count#1185 AS gen_attr_35#283, item_sold_count#1186 AS gen_attr_37#284, gmv_us_amt#1187 AS gen_attr_41#285, gmv_slr_lc_amt#1189 AS gen_attr_43#287, rvnu_insrtn_fee_us_amt#1192 AS gen_attr_51#290, rvnu_insrtn_crd_us_amt#1196 AS gen_attr_59#294, rvnu_fetr_fee_us_amt#1200 AS gen_attr_67#298, rvnu_fetr_crd_us_amt#1204 AS gen_attr_75#302, rvnu_fv_fee_us_amt#1208 AS gen_attr_83#306, rvnu_fv_crd_us_amt#1213 AS gen_attr_93#311, rvnu_othr_l_fee_us_amt#1218 AS gen_attr_103#316, rvnu_othr_l_crd_us_amt#1222 AS gen_attr_111#320, rvnu_othr_nl_fee_us_amt#1226 AS gen_attr_119#324, rvnu_othr_nl_crd_us_amt#1230 AS gen_attr_127#328, ... 7 more fields]
            :     :     :     :     :     :     :     :     :     :  +- *(11) ColumnarToRow
            :     :     :     :     :     :     :     :     :     :     +- FileScan parquet default.big_table1[byr_cntry_id#1169,sap_category_id#1170,slr_cntry_id#1174,lstg_curncy_id#1178,blng_curncy_id#1179,bid_count#1180,ck_trans_count#1181,ended_bid_count#1182,new_lstg_count#1183,ended_lstg_count#1184,ended_success_lstg_count#1185,item_sold_count#1186,gmv_us_amt#1187,gmv_slr_lc_amt#1189,rvnu_insrtn_fee_us_amt#1192,rvnu_insrtn_crd_us_amt#1196,rvnu_fetr_fee_us_amt#1200,rvnu_fetr_crd_us_amt#1204,rvnu_fv_fee_us_amt#1208,rvnu_fv_crd_us_amt#1213,rvnu_othr_l_fee_us_amt#1218,rvnu_othr_l_crd_us_amt#1222,rvnu_othr_nl_fee_us_amt#1226,rvnu_othr_nl_crd_us_amt#1230,... 7 more fields] Batched: true, DataFilters: [], Format: Parquet, Location: PrunedInMemoryFileIndex[], PartitionFilters: [isnotnull(cmn_mtrc_summ_dt#1262), (cmn_mtrc_summ_dt#1262 >= 18078), (cmn_mtrc_summ_dt#1262 <= 18..., PushedFilters: [], ReadSchema: struct<byr_cntry_id:decimal(4,0),sap_category_id:decimal(9,0),slr_cntry_id:decimal(4,0),lstg_curn...
            :     :     :     :     :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(9,0), true] as decimal(20,0)))), [id=#288]
            :     :     :     :     :     :     :     :     :        +- *(1) Project [CURNCY_ID#1263 AS gen_attr_319#367]
            :     :     :     :     :     :     :     :     :           +- *(1) Filter isnotnull(CURNCY_ID#1263)
            :     :     :     :     :     :     :     :     :              +- *(1) ColumnarToRow
            :     :     :     :     :     :     :     :     :                 +- FileScan parquet default.small_table1[CURNCY_ID#1263] Batched: true, DataFilters: [isnotnull(CURNCY_ID#1263)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table1], PartitionFilters: [], PushedFilters: [IsNotNull(CURNCY_ID)], ReadSchema: struct<CURNCY_ID:decimal(9,0)>, SelectedBucketsCount: 1 out of 1
            :     :     :     :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(9,0), true] as decimal(20,0)))), [id=#297]
            :     :     :     :     :     :     :     :        +- *(2) Project [CURNCY_ID#1263 AS gen_attr_318#379]
            :     :     :     :     :     :     :     :           +- *(2) Filter isnotnull(CURNCY_ID#1263)
            :     :     :     :     :     :     :     :              +- *(2) ColumnarToRow
            :     :     :     :     :     :     :     :                 +- FileScan parquet default.small_table1[CURNCY_ID#1263] Batched: true, DataFilters: [isnotnull(CURNCY_ID#1263)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table1], PartitionFilters: [], PushedFilters: [IsNotNull(CURNCY_ID)], ReadSchema: struct<CURNCY_ID:decimal(9,0)>, SelectedBucketsCount: 1 out of 1
            :     :     :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(4,0), true] as decimal(20,0)))), [id=#306]
            :     :     :     :     :     :     :        +- *(3) Project [cntry_id#1269 AS gen_attr_317#453, rev_rollup_id#1278 AS gen_attr_315#462]
            :     :     :     :     :     :     :           +- *(3) Filter isnotnull(cntry_id#1269)
            :     :     :     :     :     :     :              +- *(3) ColumnarToRow
            :     :     :     :     :     :     :                 +- FileScan parquet default.small_table2[cntry_id#1269,rev_rollup_id#1278] Batched: true, DataFilters: [isnotnull(cntry_id#1269)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table2], PartitionFilters: [], PushedFilters: [IsNotNull(cntry_id)], ReadSchema: struct<cntry_id:decimal(4,0),rev_rollup_id:smallint>
            :     :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(cast(input[0, smallint, true] as int) as bigint))), [id=#315]
            :     :     :     :     :     :        +- *(4) Project [rev_rollup_id#1286 AS gen_attr_316#562, curncy_id#1289 AS gen_attr_313#565]
            :     :     :     :     :     :           +- *(4) Filter isnotnull(rev_rollup_id#1286)
            :     :     :     :     :     :              +- *(4) ColumnarToRow
            :     :     :     :     :     :                 +- FileScan parquet default.small_table3[rev_rollup_id#1286,curncy_id#1289] Batched: true, DataFilters: [isnotnull(rev_rollup_id#1286)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table3], PartitionFilters: [], PushedFilters: [IsNotNull(rev_rollup_id)], ReadSchema: struct<rev_rollup_id:smallint,curncy_id:decimal(4,0)>
            :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(9,0), true] as decimal(20,0)))), [id=#324]
            :     :     :     :     :        +- *(5) Project [CURNCY_ID#1263 AS gen_attr_314#591]
            :     :     :     :     :           +- *(5) Filter isnotnull(CURNCY_ID#1263)
            :     :     :     :     :              +- *(5) ColumnarToRow
            :     :     :     :     :                 +- FileScan parquet default.small_table1[CURNCY_ID#1263] Batched: true, DataFilters: [isnotnull(CURNCY_ID#1263)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table1], PartitionFilters: [], PushedFilters: [IsNotNull(CURNCY_ID)], ReadSchema: struct<CURNCY_ID:decimal(9,0)>, SelectedBucketsCount: 1 out of 1
            :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(4,0), true] as decimal(20,0)))), [id=#333]
            :     :     :     :        +- *(6) Project [cntry_id#1269 AS gen_attr_312#665, rev_rollup_id#1278 AS gen_attr_310#674]
            :     :     :     :           +- *(6) Filter isnotnull(cntry_id#1269)
            :     :     :     :              +- *(6) ColumnarToRow
            :     :     :     :                 +- FileScan parquet default.small_table2[cntry_id#1269,rev_rollup_id#1278] Batched: true, DataFilters: [isnotnull(cntry_id#1269)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table2], PartitionFilters: [], PushedFilters: [IsNotNull(cntry_id)], ReadSchema: struct<cntry_id:decimal(4,0),rev_rollup_id:smallint>
            :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(cast(input[0, smallint, true] as int) as bigint))), [id=#342]
            :     :     :        +- *(7) Project [rev_rollup_id#1286 AS gen_attr_311#774, curncy_id#1289 AS gen_attr_308#777]
            :     :     :           +- *(7) Filter isnotnull(rev_rollup_id#1286)
            :     :     :              +- *(7) ColumnarToRow
            :     :     :                 +- FileScan parquet default.small_table3[rev_rollup_id#1286,curncy_id#1289] Batched: true, DataFilters: [isnotnull(rev_rollup_id#1286)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table3], PartitionFilters: [], PushedFilters: [IsNotNull(rev_rollup_id)], ReadSchema: struct<rev_rollup_id:smallint,curncy_id:decimal(4,0)>
            :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(9,0), true] as decimal(20,0)))), [id=#351]
            :     :        +- *(8) Project [CURNCY_ID#1263 AS gen_attr_309#803]
            :     :           +- *(8) Filter isnotnull(CURNCY_ID#1263)
            :     :              +- *(8) ColumnarToRow
            :     :                 +- FileScan parquet default.small_table1[CURNCY_ID#1263] Batched: true, DataFilters: [isnotnull(CURNCY_ID#1263)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table1], PartitionFilters: [], PushedFilters: [IsNotNull(CURNCY_ID)], ReadSchema: struct<CURNCY_ID:decimal(9,0)>, SelectedBucketsCount: 1 out of 1
            :     +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, decimal(4,0), true])), [id=#360]
            :        +- *(9) Project [cntry_id#1269, rev_rollup#1279]
            :           +- *(9) Filter isnotnull(cntry_id#1269)
            :              +- *(9) ColumnarToRow
            :                 +- FileScan parquet default.small_table2[cntry_id#1269,rev_rollup#1279] Batched: true, DataFilters: [isnotnull(cntry_id#1269)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table2], PartitionFilters: [], PushedFilters: [IsNotNull(cntry_id)], ReadSchema: struct<cntry_id:decimal(4,0),rev_rollup:string>
            +- ReusedExchange [cntry_id#1309, rev_rollup#1319], BroadcastExchange HashedRelationBroadcastMode(List(input[0, decimal(4,0), true])), [id=#360]
```
This PR try to improve `ResolveTables` and `ResolveRelations` performance by reducing the connection times to Hive Metastore Server in such case.

### Why are the changes needed?
1. Reduce the connection times to Hive Metastore Server.
2. Improve `ResolveTables` and `ResolveRelations` performance.

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

### How was this patch tested?

manual test.
After [SPARK-29606](https://issues.apache.org/jira/browse/SPARK-29606) and before this PR:
```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 9323
Total time: 2.687441263 seconds

Rule                                                                                               Effective Time / Total Time                     Effective Runs / Total Runs

org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations                                   929173767 / 930133504                           2 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTables                                      0 / 383363402                                   0 / 18
org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin                                         0 / 99433540                                    0 / 4
org.apache.spark.sql.catalyst.analysis.DecimalPrecision                                            41809394 / 83727901                             2 / 18
org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions                               71372977 / 71372977                             1 / 1
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts                              0 / 59071933                                    0 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  37858325 / 58471776                             5 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$PromoteStrings                                 20889892 / 53229016                             1 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion                     23428968 / 50890815                             1 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$InConversion                                   23230666 / 49182607                             1 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator                                   0 / 43638350                                    0 / 18
org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              17194844 / 42530885                             1 / 6
```
After [SPARK-29606](https://issues.apache.org/jira/browse/SPARK-29606) and after this PR:
```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 9323
Total time: 2.163765869 seconds

Rule                                                                                               Effective Time / Total Time                     Effective Runs / Total Runs

org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations                                   658905353 / 659829383                           2 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTables                                      0 / 220708715                                   0 / 18
org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin                                         0 / 99606816                                    0 / 4
org.apache.spark.sql.catalyst.analysis.DecimalPrecision                                            39616060 / 78215752                             2 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  36706549 / 54917789                             5 / 18
org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions                               53561921 / 53561921                             1 / 1
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts                              0 / 52329678                                    0 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$PromoteStrings                                 20945755 / 49695998                             1 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion                     20872241 / 46740145                             1 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$InConversion                                   19780298 / 44327227                             1 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator                                   0 / 42312023                                    0 / 18
org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              17197393 / 39501424                             1 / 6
```

Closes #26589 from wangyum/SPARK-29947.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 22:36:31 +08:00
yi.wu e38964c442 [SPARK-29768][SQL][FOLLOW-UP] Improve handling non-deterministic filter of ScanOperation
### What changes were proposed in this pull request?

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Test exists.

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

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

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

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

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

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

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

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

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

They are the best practices from PostgreSQL and presto.

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

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

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

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

### How was this patch tested?

add uts.

Closes #26847 from yaooqinn/SPARK-30214.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

1. bug fix
2. `ANSI` support

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

### How was this patch tested?

add unit tests

Closes #26995 from yaooqinn/SPARK-30341.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 02:04:20 +08:00
Wenchen Fan 68260f5297 revert [SPARK-29680][SQL] Remove ALTER TABLE CHANGE COLUMN syntax
### What changes were proposed in this pull request?

Revert https://github.com/apache/spark/pull/26338 , as the syntax is actually the [hive style ALTER COLUMN](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-ChangeColumnName/Type/Position/Comment).

This PR brings it back, and make it support multi-catalog:
1. renaming is not allowed as `AlterTableAlterColumnStatement` can't do renaming.
2. column name should be multi-part

### Why are the changes needed?

to not break hive compatibility.

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

no, as the removal was merged in 3.0.

### How was this patch tested?

new parser tests

Closes #27076 from cloud-fan/alter.

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

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

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

### Why are the changes needed?

To resolve relations inside view correctly.

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

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

### How was this patch tested?

a new test

Closes #26923 from cloud-fan/view.

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

### Why are the changes needed?
Fix exception

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

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

Closes #27068 from JkSelf/fixSubqueryIssue.

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

### Why are the changes needed?
Fix exception

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

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

Closes #27074 from JkSelf/resetMetricsIssue.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-02 21:55:36 +08:00
Jungtaek Lim (HeartSaVioR) 5d870ef0bc [SPARK-26560][SQL] Spark should be able to run Hive UDF using jar regardless of current thread context classloader
### What changes were proposed in this pull request?

This patch is based on #23921 but revised to be simpler, as well as adds UT to test the behavior.
(This patch contains the commit from #23921 to retain credit.)

Spark loads new JARs for `ADD JAR` and `CREATE FUNCTION ... USING JAR` into jar classloader in shared state, and changes current thread's context classloader to jar classloader as many parts of remaining codes rely on current thread's context classloader.

This would work if the further queries will run in same thread and there's no change on context classloader for the thread, but once the context classloader of current thread is switched back by various reason, Spark fails to create instance of class for the function.

This bug mostly affects spark-shell, as spark-shell will roll back current thread's context classloader at every prompt. But it may also affects the case of job-server, where the queries may be running in multiple threads.

This patch fixes the issue via switching the context classloader to the classloader which loads the class. Hopefully FunctionBuilder created by `makeFunctionBuilder` has the information of Class as a part of closure, hence the Class itself can be provided regardless of current thread's context classloader.

### Why are the changes needed?

Without this patch, end users cannot execute Hive UDF using JAR twice in spark-shell.

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

No.

### How was this patch tested?

New UT.

Closes #27025 from HeartSaVioR/SPARK-26560-revised.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: nivo091 <nivedeeta.singh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-02 15:44:45 +08:00
yi.wu 83d289eef4 [SPARK-27638][SQL][FOLLOW-UP] Format config name to follow the other boolean conf naming convention
### What changes were proposed in this pull request?

Change config name from `spark.sql.legacy.typeCoercion.datetimeToString` to `spark.sql.legacy.typeCoercion.datetimeToString.enabled`.

### Why are the changes needed?

To follow the other boolean conf naming convention.

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

No, it's newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins

Closes #27065 from Ngone51/SPARK-27638-FOLLOWUP.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-02 15:35:33 +09:00
yi.wu 90794b617c [SPARK-27871][SQL][FOLLOW-UP] Format config name to follow the other boolean conf naming convention
### What changes were proposed in this pull request?

Change config name from `spark.sql.optimizer.reassignLambdaVariableID` to `spark.sql.optimizer.reassignLambdaVariableID.enabled`.

### Why are the changes needed?

To follow the other boolean conf naming convention.

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

No, it's newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27063 from Ngone51/SPARK-27871-FOLLOWUP.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-02 09:59:15 +09:00
Maxim Gekk b316d37365 [SPARK-30401][SQL] Call requireNonStaticConf() only once in set()
### What changes were proposed in this pull request?
Calls of `requireNonStaticConf()` are removed from the `set()` methods in RuntimeConfig because those methods invoke `def set(key: String, value: String): Unit` where `requireNonStaticConf()` is called as well.

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

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

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

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

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-02 09:56:50 +09:00
Zhenhua Wang a8bf5d823b [SPARK-30339][SQL] Avoid to fail twice in function lookup
### What changes were proposed in this pull request?

Currently if function lookup fails, spark will give it a second change by casting decimal type to double type. But for cases where decimal type doesn't exist, it's meaningless to lookup again and causes extra cost like unnecessary metastore access. We should throw exceptions directly in these cases.

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

No.

### How was this patch tested?

Covered by existing tests.

Closes #26994 from wzhfy/avoid_udf_fail_twice.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-31 01:09:51 +09:00
Jungtaek Lim (HeartSaVioR) e054a0af6f [SPARK-29348][SQL][FOLLOWUP] Fix slight bug on streaming example for Dataset.observe
### What changes were proposed in this pull request?

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

### Why are the changes needed?

Described above.

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

No.

### How was this patch tested?

Ran below query via `spark-shell`:

**Streaming**

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

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

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

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

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

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

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

This PR proposes a `tail` API.

Namely, as below:

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

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

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

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

### Why are the changes needed?

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

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

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

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

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

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

No, (new API)

### How was this patch tested?

Unit tests were added and manually tested.

Closes #26809 from HyukjinKwon/wip-tail.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-31 01:07:09 +09:00
Terry Kim a90ad5bf2a [SPARK-30370][SQL] Update SqlBase.g4 to combine namespace and database tokens
### What changes were proposed in this pull request?

In `SqlBase.g4`, `database` is defined as
```
database : DATABASE | SCHEMA;
```
and it is being used as `(database | NAMESPACE)` in many places.

This PR proposes to define the following and use it as discussed in https://github.com/apache/spark/pull/26847/files#r359754778:
```
namespace : NAMESPACE | DATABASE | SCHEMA;
```

### Why are the changes needed?

To simplify the grammar.

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

No.

### How was this patch tested?

There is no change in the actual grammar, so the existing tests should be sufficient.

Closes #27027 from imback82/sqlbase_namespace.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-30 14:58:02 +08:00
Xiao Li 919d551ddb Revert "[SPARK-29390][SQL] Add the justify_days(), justify_hours() and justif_interval() functions"
This reverts commit f926809a1f.

Closes #27032 from gatorsmile/revertSPARK-29390.

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

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

## How was this patch tested?

Added UT and also tested manually

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

**After Fix**

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

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

Closes #25399 from sandeep-katta/funcIssue.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-28 14:35:33 +09:00
Kent Yao f0bf2eb006 [SPARK-30356][SQL] Codegen support for the function str_to_map
### What changes were proposed in this pull request?
`str_to_map ` has not implemented with codegen support, which prevents a query that contains this expression from being whole stage codegen-ed.
This PR removes `CodegenFallBack` from `StringToMap`, add the codegen support for it.

### Why are the changes needed?

improve codegen coverage and gain better perfomance

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

no

### How was this patch tested?

1. pass ComplexTypeSuite

2. manually review generated code

```java

-- !query 12
explain codegen select v, str_to_map(v) from values ('abc🅰️a,:'), (null), (''), ('1:2') t(v)
-- !query 12 schema
struct<plan:string>
-- !query 12 output
Found 1 WholeStageCodegen subtrees.
== Subtree 1 / 1 (maxMethodCodeSize:511; maxConstantPoolSize:188(0.29% used); numInnerClasses:0) ==
*Project [v#x, str_to_map(v#x, ,, :) AS str_to_map(v, ,, :)#x]
+- *LocalTableScan [v#x]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=1
/* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator localtablescan_input_0;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] project_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter[] project_mutableStateArray_1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter[2];
/* 012 */
/* 013 */   public GeneratedIteratorForCodegenStage1(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     localtablescan_input_0 = inputs[0];
/* 021 */     project_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 64);
/* 022 */     project_mutableStateArray_1[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(project_mutableStateArray_0[0], 8);
/* 023 */     project_mutableStateArray_1[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(project_mutableStateArray_0[0], 8);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   private void project_doConsume_0(InternalRow localtablescan_row_0, UTF8String project_expr_0_0, boolean project_exprIsNull_0_0) throws java.io.IOException {
/* 028 */     boolean project_isNull_1 = true;
/* 029 */     MapData project_value_1 = null;
/* 030 */
/* 031 */     if (!project_exprIsNull_0_0) {
/* 032 */       project_isNull_1 = false; // resultCode could change nullability.
/* 033 */
/* 034 */       int project_i_0 = 0;
/* 035 */       UTF8String[] project_kvs_0 = project_expr_0_0.split(((UTF8String) references[2] /* literal */), -1);
/* 036 */       while (project_i_0 < project_kvs_0.length) {
/* 037 */         UTF8String[] kv = project_kvs_0[project_i_0].split(((UTF8String) references[3] /* literal */), 2);
/* 038 */         UTF8String key = kv[0];
/* 039 */         UTF8String value = null;
/* 040 */         if (kv.length == 2) {
/* 041 */           value = kv[1];
/* 042 */         }
/* 043 */         ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[1] /* mapBuilder */).put(key, value);
/* 044 */         project_i_0++;
/* 045 */       }
/* 046 */       project_value_1 = ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[1] /* mapBuilder */).build();
/* 047 */
/* 048 */     }
/* 049 */     project_mutableStateArray_0[0].reset();
/* 050 */
/* 051 */     project_mutableStateArray_0[0].zeroOutNullBytes();
/* 052 */
/* 053 */     if (project_exprIsNull_0_0) {
/* 054 */       project_mutableStateArray_0[0].setNullAt(0);
/* 055 */     } else {
/* 056 */       project_mutableStateArray_0[0].write(0, project_expr_0_0);
/* 057 */     }
/* 058 */
/* 059 */     if (project_isNull_1) {
/* 060 */       project_mutableStateArray_0[0].setNullAt(1);
/* 061 */     } else {
/* 062 */       final MapData project_tmpInput_0 = project_value_1;
/* 063 */       if (project_tmpInput_0 instanceof UnsafeMapData) {
/* 064 */         project_mutableStateArray_0[0].write(1, (UnsafeMapData) project_tmpInput_0);
/* 065 */       } else {
/* 066 */         // Remember the current cursor so that we can calculate how many bytes are
/* 067 */         // written later.
/* 068 */         final int project_previousCursor_0 = project_mutableStateArray_0[0].cursor();
/* 069 */
/* 070 */         // preserve 8 bytes to write the key array numBytes later.
/* 071 */         project_mutableStateArray_0[0].grow(8);
/* 072 */         project_mutableStateArray_0[0].increaseCursor(8);
/* 073 */
/* 074 */         // Remember the current cursor so that we can write numBytes of key array later.
/* 075 */         final int project_tmpCursor_0 = project_mutableStateArray_0[0].cursor();
/* 076 */
/* 077 */         final ArrayData project_tmpInput_1 = project_tmpInput_0.keyArray();
/* 078 */         if (project_tmpInput_1 instanceof UnsafeArrayData) {
/* 079 */           project_mutableStateArray_0[0].write((UnsafeArrayData) project_tmpInput_1);
/* 080 */         } else {
/* 081 */           final int project_numElements_0 = project_tmpInput_1.numElements();
/* 082 */           project_mutableStateArray_1[0].initialize(project_numElements_0);
/* 083 */
/* 084 */           for (int project_index_0 = 0; project_index_0 < project_numElements_0; project_index_0++) {
/* 085 */             project_mutableStateArray_1[0].write(project_index_0, project_tmpInput_1.getUTF8String(project_index_0));
/* 086 */           }
/* 087 */         }
/* 088 */
/* 089 */         // Write the numBytes of key array into the first 8 bytes.
/* 090 */         Platform.putLong(
/* 091 */           project_mutableStateArray_0[0].getBuffer(),
/* 092 */           project_tmpCursor_0 - 8,
/* 093 */           project_mutableStateArray_0[0].cursor() - project_tmpCursor_0);
/* 094 */
/* 095 */         final ArrayData project_tmpInput_2 = project_tmpInput_0.valueArray();
/* 096 */         if (project_tmpInput_2 instanceof UnsafeArrayData) {
/* 097 */           project_mutableStateArray_0[0].write((UnsafeArrayData) project_tmpInput_2);
/* 098 */         } else {
/* 099 */           final int project_numElements_1 = project_tmpInput_2.numElements();
/* 100 */           project_mutableStateArray_1[1].initialize(project_numElements_1);
/* 101 */
/* 102 */           for (int project_index_1 = 0; project_index_1 < project_numElements_1; project_index_1++) {
/* 103 */             if (project_tmpInput_2.isNullAt(project_index_1)) {
/* 104 */               project_mutableStateArray_1[1].setNull8Bytes(project_index_1);
/* 105 */             } else {
/* 106 */               project_mutableStateArray_1[1].write(project_index_1, project_tmpInput_2.getUTF8String(project_index_1));
/* 107 */             }
/* 108 */
/* 109 */           }
/* 110 */         }
/* 111 */
/* 112 */         project_mutableStateArray_0[0].setOffsetAndSizeFromPreviousCursor(1, project_previousCursor_0);
/* 113 */       }
/* 114 */     }
/* 115 */     append((project_mutableStateArray_0[0].getRow()));
/* 116 */
/* 117 */   }
/* 118 */
/* 119 */   protected void processNext() throws java.io.IOException {
/* 120 */     while ( localtablescan_input_0.hasNext()) {
/* 121 */       InternalRow localtablescan_row_0 = (InternalRow) localtablescan_input_0.next();
/* 122 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 123 */       boolean localtablescan_isNull_0 = localtablescan_row_0.isNullAt(0);
/* 124 */       UTF8String localtablescan_value_0 = localtablescan_isNull_0 ?
/* 125 */       null : (localtablescan_row_0.getUTF8String(0));
/* 126 */
/* 127 */       project_doConsume_0(localtablescan_row_0, localtablescan_value_0, localtablescan_isNull_0);
/* 128 */       if (shouldStop()) return;
/* 129 */     }
/* 130 */   }
/* 131 */
/* 132 */ }

-- !query 13
select v, str_to_map(v) from values ('abc🅰️a,:'), (null), (''), ('1:2') t(v)
-- !query 13 schema
struct<v:string,str_to_map(v, ,, :):map<string,string>>
-- !query 13 output
	{"":null}
1:2	{"1":"2"}
NULL	NULL
abc🅰️a,:	{"":"","abc":"a:a"}
```

Closes #27013 from yaooqinn/SPARK-30356.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-27 21:44:03 +08:00
Jungtaek Lim (HeartSaVioR) 7adf886792 [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
### What changes were proposed in this pull request?

This patch fixes the intermittent test failure on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite, getting ConnectException when querying to thrift server.
(https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115646/testReport/)

The relevant unit test log messages are following:

```
19/12/23 13:33:01.875 pool-1-thread-1 INFO AbstractService: Service:ThriftBinaryCLIService is started.
19/12/23 13:33:01.875 pool-1-thread-1 INFO AbstractService: Service:HiveServer2 is started.
...
19/12/23 13:33:01.888 pool-1-thread-1 INFO ThriftServerWithSparkContextSuite: HiveThriftServer2 started successfully
...
19/12/23 13:33:01.909 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO ThriftServerWithSparkContextSuite:

===== TEST OUTPUT FOR o.a.s.sql.hive.thriftserver.ThriftServerWithSparkContextSuite: 'SPARK-29911: Uncache cached tables when session closed' =====

...
19/12/23 13:33:02.017 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO Utils: Supplied authorities: localhost:15441
19/12/23 13:33:02.018 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO Utils: Resolved authority: localhost:15441
19/12/23 13:33:02.078 HiveServer2-Background-Pool: Thread-213 INFO BaseSessionStateBuilder$$anon$2: Optimization rule 'org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation' is excluded from the optimizer.
19/12/23 13:33:02.078 HiveServer2-Background-Pool: Thread-213 INFO BaseSessionStateBuilder$$anon$2: Optimization rule 'org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation' is excluded from the optimizer.
19/12/23 13:33:02.121 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite WARN HiveConnection: Failed to connect to localhost:15441
19/12/23 13:33:02.124 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO ThriftServerWithSparkContextSuite:

===== FINISHED o.a.s.sql.hive.thriftserver.ThriftServerWithSparkContextSuite: 'SPARK-29911: Uncache cached tables when session closed' =====

19/12/23 13:33:02.143 Thread-35 INFO ThriftCLIService: Starting ThriftBinaryCLIService on port 15441 with 5...500 worker threads
19/12/23 13:33:02.327 pool-1-thread-1 INFO HiveServer2: Shutting down HiveServer2
19/12/23 13:33:02.328 pool-1-thread-1 INFO ThriftCLIService: Thrift server has stopped
```
(Here the error is logged as `WARN HiveConnection: Failed to connect to localhost:15441` - the actual stack trace can be seen on Jenkins test summary.)

The reason of test failure: Thrift(Binary|Http)CLIService prepare and launch the service asynchronously (in new thread), which suites are not waiting for completion and just start running tests, ends up with race condition.

That can be easily reproduced, via adding artificial sleep in `ThriftBinaryCLIService.run()` here:
ba3f6330dd/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (L49)

(Note that `sleep` should be added before initializing server socket. E.g. Line 57)

This patch changes the test initialization logic to try executing simple query to wait until the service is available. The patch also refactors the code to apply the change both ThriftServerQueryTestSuite and ThriftServerWithSparkContextSuite easily.

### Why are the changes needed?

This patch fixes the intermittent failure observed here:
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115646/testReport/

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

No

### How was this patch tested?

Artificially made the test fail consistently (by the approach described above), and confirmed the patch fixed the test.

Closes #27001 from HeartSaVioR/SPARK-30345.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-27 15:30:54 +08:00
lijunqing a2de20c0e6 [SPARK-30036][SQL] Fix: REPARTITION hint does not work with order by
### Why are the changes needed?
`EnsureRequirements` adds `ShuffleExchangeExec` (RangePartitioning) after Sort if `RoundRobinPartitioning` behinds it. This will cause 2 shuffles, and the number of partitions in the final stage is not the number specified by `RoundRobinPartitioning.

**Example SQL**
```
SELECT /*+ REPARTITION(5) */ * FROM test ORDER BY a
```

**BEFORE**
```
== Physical Plan ==
*(1) Sort [a#0 ASC NULLS FIRST], true, 0
+- Exchange rangepartitioning(a#0 ASC NULLS FIRST, 200), true, [id=#11]
   +- Exchange RoundRobinPartitioning(5), false, [id=#9]
      +- Scan hive default.test [a#0, b#1], HiveTableRelation `default`.`test`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#0, b#1]
```

**AFTER**
```
== Physical Plan ==
*(1) Sort [a#0 ASC NULLS FIRST], true, 0
+- Exchange rangepartitioning(a#0 ASC NULLS FIRST, 5), true, [id=#11]
   +- Scan hive default.test [a#0, b#1], HiveTableRelation `default`.`test`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#0, b#1]
```

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

### How was this patch tested?
Run suite Tests and add new test for this.

Closes #26946 from stczwd/RoundRobinPartitioning.

Lead-authored-by: lijunqing <lijunqing@baidu.com>
Co-authored-by: stczwd <qcsd2011@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-27 11:52:39 +08:00
Kengo Seki 59c014e120 [SPARK-30350][SQL] Fix ScalaReflection to use an empty array for getting its class object
### What changes were proposed in this pull request?

This PR fixes `ScalaReflection.arrayClassFor()` to use an empty array instead of a one-element array for getting its class object by reflection.

### Why are the changes needed?

Because it may reduce unnecessary memory allocation.

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

No

### How was this patch tested?

Ran the existing unit tests for sql/catalyst and confirmed that all of them succeeded.

Closes #27005 from sekikn/SPARK-30350.

Authored-by: Kengo Seki <sekikn@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-26 22:54:29 +09:00
gengjiaan d59e7195f6 [SPARK-27986][SQL] Support ANSI SQL filter clause for aggregate expression
### What changes were proposed in this pull request?
The filter predicate for aggregate expression is an `ANSI SQL`.
```
<aggregate function> ::=
COUNT <left paren> <asterisk> <right paren> [ <filter clause> ]
| <general set function> [ <filter clause> ]
| <binary set function> [ <filter clause> ]
| <ordered set function> [ <filter clause> ]
| <array aggregate function> [ <filter clause> ]
| <row pattern count function> [ <filter clause> ]
```
There are some mainstream database support this syntax.
**PostgreSQL:**
https://www.postgresql.org/docs/current/sql-expressions.html#SYNTAX-AGGREGATES
For example:
```
SELECT
  year,
  count(*) FILTER (WHERE gdp_per_capita >= 40000)
FROM
  countries
GROUP BY
  year
```
```
SELECT
  year,
  code,
  gdp_per_capita,
  count(*)
    FILTER (WHERE gdp_per_capita >= 40000)
    OVER   (PARTITION BY year)
FROM
  countries
```
**jOOQ:**
https://blog.jooq.org/2014/12/30/the-awesome-postgresql-9-4-sql2003-filter-clause-for-aggregate-functions/

**Notice:**
1.This PR only supports FILTER predicate without codegen. maropu will create another PR is related to SPARK-30027 to support codegen.
2.This PR only supports FILTER predicate without DISTINCT. I will create another PR is related to SPARK-30276 to support this.
3.This PR only supports FILTER predicate that can't reference the outer query. I created ticket SPARK-30219 to support it.
4.This PR only supports FILTER predicate that can't use IN/EXISTS predicate sub-queries. I created ticket SPARK-30220 to support it.
5.Spark SQL cannot supports a SQL with nested aggregate. I created ticket SPARK-30182 to support it.

There are some show of the PR on my production environment.
```
spark-sql> desc gja_test_partition;
key     string  NULL
value   string  NULL
other   string  NULL
col2    int     NULL
# Partition Information
# col_name      data_type       comment
col2    int     NULL
Time taken: 0.79 s
```
```
spark-sql> select * from gja_test_partition;
a       A       ao      1
b       B       bo      1
c       C       co      1
d       D       do      1
e       E       eo      2
g       G       go      2
h       H       ho      2
j       J       jo      2
f       F       fo      3
k       K       ko      3
l       L       lo      4
i       I       io      4
Time taken: 1.75 s
```
```
spark-sql> select count(key), sum(col2) from gja_test_partition;
12      26
Time taken: 1.848 s
```
```
spark-sql> select count(key) filter (where col2 > 1) from gja_test_partition;
8
Time taken: 2.926 s
```
```
spark-sql> select sum(col2) filter (where col2 > 2) from gja_test_partition;
14
Time taken: 2.087 s
```
```
spark-sql> select count(key) filter (where col2 > 1), sum(col2) filter (where col2 > 2) from gja_test_partition;
8       14
Time taken: 2.847 s
```
```
spark-sql> select count(key), count(key) filter (where col2 > 1), sum(col2), sum(col2) filter (where col2 > 2) from gja_test_partition;
12      8       26      14
Time taken: 1.787 s
```
```
spark-sql> desc student;
id      int     NULL
name    string  NULL
sex     string  NULL
class_id        int     NULL
Time taken: 0.206 s
```
```
spark-sql> select * from student;
1       张三    man     1
2       李四    man     1
3       王五    man     2
4       赵六    man     2
5       钱小花  woman   1
6       赵九红  woman   2
7       郭丽丽  woman   2
Time taken: 0.786 s
```
```
spark-sql> select class_id, count(id), sum(id) from student group by class_id;
1       3       8
2       4       20
Time taken: 18.783 s
```
```
spark-sql> select class_id, count(id) filter (where sex = 'man'), sum(id) filter (where sex = 'woman') from student group by class_id;
1       2       5
2       2       13
Time taken: 3.887 s
```

### Why are the changes needed?
Add new SQL feature.

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

### How was this patch tested?
Exists UT and new UT.

Closes #26656 from beliefer/support-aggregate-clause.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-26 17:41:50 +08:00
wenfang 4d58cd77f9 [SPARK-30330][SQL] Support single quotes json parsing for get_json_object and json_tuple
### What changes were proposed in this pull request?

I execute some query as` select get_json_object(ytag, '$.y1') AS y1 from t4`; SparkSQL return null but  Hive return correct results.
In my production environment, ytag is a json wrapped by single quotes,as follows
```
{'y1': 'shuma', 'y2': 'shuma:shouji'}
{'y1': 'jiaoyu', 'y2': 'jiaoyu:gaokao'}
{'y1': 'yule', 'y2': 'yule:mingxing'}
```
Then l realized some functions including get_json_object and json_tuple does not support  single quotes json parsing.
So l provide this PR to resolve the question.

### Why are the changes needed?

Enabled for Hive compatibility

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

NO

### How was this patch tested?

NEW TESTS

Closes #26965 from wenfang6/enableSingleQuotesJsonForSparkSQL.

Authored-by: wenfang <wenfang@360.cn>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-26 11:45:31 +09:00
yi.wu 6d64fc2407 [SPARK-26389][SS][FOLLOW-UP] Format config name to follow the other boolean conf naming convention
### What changes were proposed in this pull request?

Rename `spark.sql.streaming.forceDeleteTempCheckpointLocation` to `spark.sql.streaming.forceDeleteTempCheckpointLocation.enabled`.

### Why are the changes needed?

To follow the other boolean conf naming convention.

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

No, as this config is newly added in 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #26981 from Ngone51/SPARK-26389-FOLLOWUP.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 21:45:01 +08:00
Kent Yao da65a955ed [SPARK-30266][SQL] Avoid match error and int overflow in ApproximatePercentile and Percentile
### What changes were proposed in this pull request?
accuracyExpression can accept Long which may cause overflow error.
accuracyExpression can accept fractions which are implicitly floored.
accuracyExpression can accept null which is implicitly changed to 0.
percentageExpression can accept null but cause MatchError.
percentageExpression can accept ArrayType(_, nullable=true) in which the nulls are implicitly changed to zeros.

##### cases
```sql
select percentile_approx(10.0, 0.5, 2147483648); -- overflow and fail
select percentile_approx(10.0, 0.5, 4294967297); -- overflow but success
select percentile_approx(10.0, 0.5, null); -- null cast to 0
select percentile_approx(10.0, 0.5, 1.2); -- 1.2 cast to 1
select percentile_approx(10.0, null, 1); -- scala.MatchError
select percentile_approx(10.0, array(0.2, 0.4, null), 1); -- null cast to zero.
```

##### behavior before

```sql
+select percentile_approx(10.0, 0.5, 2147483648)
+org.apache.spark.sql.AnalysisException
+cannot resolve 'percentile_approx(10.0BD, CAST(0.5BD AS DOUBLE), CAST(2147483648L AS INT))' due to data type mismatch: The accuracy provided must be a positive integer literal (current value = -2147483648); line 1 pos 7
+
+select percentile_approx(10.0, 0.5, 4294967297)
+10.0
+

+select percentile_approx(10.0, 0.5, null)
+org.apache.spark.sql.AnalysisException
+cannot resolve 'percentile_approx(10.0BD, CAST(0.5BD AS DOUBLE), CAST(NULL AS INT))' due to data type mismatch: The accuracy provided must be a positive integer literal (current value = 0); line 1 pos 7
+
+select percentile_approx(10.0, 0.5, 1.2)
+10.0
+
+select percentile_approx(10.0, null, 1)
+scala.MatchError
+null
+
+
+select percentile_approx(10.0, array(0.2, 0.4, null), 1)
+[10.0,10.0,10.0]
```

##### behavior after

```sql

+select percentile_approx(10.0, 0.5, 2147483648)
+10.0
+
+select percentile_approx(10.0, 0.5, 4294967297)
+10.0
+
+select percentile_approx(10.0, 0.5, null)
+org.apache.spark.sql.AnalysisException
+cannot resolve 'percentile_approx(10.0BD, 0.5BD, NULL)' due to data type mismatch: argument 3 requires integral type, however, 'NULL' is of null type.; line 1 pos 7
+
+select percentile_approx(10.0, 0.5, 1.2)
+org.apache.spark.sql.AnalysisException
+cannot resolve 'percentile_approx(10.0BD, 0.5BD, 1.2BD)' due to data type mismatch: argument 3 requires integral type, however, '1.2BD' is of decimal(2,1) type.; line 1 pos 7
+

+select percentile_approx(10.0, null, 1)
+java.lang.IllegalArgumentException
+The value of percentage must be be between 0.0 and 1.0, but got null
+
+select percentile_approx(10.0, array(0.2, 0.4, null), 1)
+java.lang.IllegalArgumentException
+Each value of the percentage array must be be between 0.0 and 1.0, but got [0.2,0.4,null]
```

### Why are the changes needed?

bug fix

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

yes, fix some improper usages of percentile_approx as cases list above

### How was this patch tested?

add ut

Closes #26905 from yaooqinn/SPARK-30266.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 20:03:26 +08:00
manuzhang ef6f9e9668 [SPARK-30331][SQL] Set isFinalPlan to true before posting the final AdaptiveSparkPlan event
### What changes were proposed in this pull request?

Set `isFinalPlan=true` before posting the final AdaptiveSparkPlan event (`SparkListenerSQLAdaptiveExecutionUpdate`)

### Why are the changes needed?

Otherwise, any attempt to listen on the final event by pattern matching `isFinalPlan=true` would fail

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

No.

### How was this patch tested?

All tests in `AdaptiveQueryExecSuite` are exteneded with a verification that a `SparkListenerSQLAdaptiveExecutionUpdate` event with `isFinalPlan=True` exists

Closes #26983 from manuzhang/spark-30331.

Authored-by: manuzhang <owenzhang1990@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 19:08:24 +08:00
Pavithra Ramachandran 57ca95246c [SPARK-29505][SQL] Make DESC EXTENDED <table name> <column name> case insensitive
### What changes were proposed in this pull request?
While querying using **desc** , if column name is not entered exactly as per the column name given during the table creation, the colstats are wrong. fetching of col stats has been made case insensitive.

### Why are the changes needed?
functions like **analyze**, etc support case insensitive retrieval of column data.

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

### How was this patch tested?
<!--
Unit test has been rewritten and tested.

Closes #26927 from PavithraRamachandran/desc_caseinsensitive.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-25 08:57:34 +09:00
wangguangxin.cn 640dcc435b [SPARK-28332][SQL] Reserve init value -1 only when do min max statistics in SQLMetrics
### What changes were proposed in this pull request?
This is an alternative solution to https://github.com/apache/spark/pull/25095.
SQLMetrics use -1 as init value as a work around for [SPARK-11013](https://issues.apache.org/jira/browse/SPARK-11013.) However, it may bring out some badcases as https://github.com/apache/spark/pull/26726 reporting. In fact, we only need to reserve -1 when doing min max statistics in `SQLMetrics.stringValue` so that we can filter out those not initialized accumulators.

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

### How was this patch tested?
Existing UTs

Closes #26899 from WangGuangxin/sqlmetrics.

Authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-23 13:13:35 +08:00
HyukjinKwon e5abbab0ed [SPARK-30128][DOCS][PYTHON][SQL] Document/promote 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC
### What changes were proposed in this pull request?

This PR adds and exposes the options, 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC, into documentation.

- `recursiveFileLookup` at file sources: https://github.com/apache/spark/pull/24830 ([SPARK-27627](https://issues.apache.org/jira/browse/SPARK-27627))
- `pathGlobFilter` at file sources: https://github.com/apache/spark/pull/24518 ([SPARK-27990](https://issues.apache.org/jira/browse/SPARK-27990))
- `mergeSchema` at ORC: https://github.com/apache/spark/pull/24043 ([SPARK-11412](https://issues.apache.org/jira/browse/SPARK-11412))

**Note that** `timeZone` option was not moved from `DataFrameReader.options` as I assume it will likely affect other datasources as well once DSv2 is complete.

### Why are the changes needed?

To document available options in sources properly.

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

In PySpark, `pathGlobFilter` can be set via `DataFrameReader.(text|orc|parquet|json|csv)` and `DataStreamReader.(text|orc|parquet|json|csv)`.

### How was this patch tested?

Manually built the doc and checked the output. Option setting in PySpark is rather a logical change. I manually tested one only:

```bash
$ ls -al tmp
...
-rw-r--r--   1 hyukjin.kwon  staff     3 Dec 20 12:19 aa
-rw-r--r--   1 hyukjin.kwon  staff     3 Dec 20 12:19 ab
-rw-r--r--   1 hyukjin.kwon  staff     3 Dec 20 12:19 ac
-rw-r--r--   1 hyukjin.kwon  staff     3 Dec 20 12:19 cc
```

```python
>>> spark.read.text("tmp", pathGlobFilter="*c").show()
```

```
+-----+
|value|
+-----+
|   ac|
|   cc|
+-----+
```

Closes #26958 from HyukjinKwon/doc-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-23 09:57:42 +09:00
gengjiaan a38bf7e051 [SPARK-28083][SQL][TEST][FOLLOW-UP] Enable LIKE ... ESCAPE test cases
### What changes were proposed in this pull request?
This PR is a follow-up to https://github.com/apache/spark/pull/25001

### Why are the changes needed?
No

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

### How was this patch tested?
Pass the Jenkins with the newly update test files.

Closes #26949 from beliefer/uncomment-like-escape-tests.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-21 14:40:07 -08:00
Kazuaki Ishizaki f31d9a629b [MINOR][DOC][SQL][CORE] Fix typo in document and comments
### What changes were proposed in this pull request?

Fixed typo in `docs` directory and in other directories

1. Find typo in `docs` and apply fixes to files in all directories
2. Fix `the the` -> `the`

### Why are the changes needed?

Better readability of documents

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

No

### How was this patch tested?

No test needed

Closes #26976 from kiszk/typo_20191221.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-21 14:08:58 -08:00
Wenchen Fan cd84400271 [SPARK-29906][SQL][FOLLOWUP] Update the final plan in UI for AQE
### What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/26576, which mistakenly removes the UI update of the final plan.

### Why are the changes needed?

fix mistake.

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

no

### How was this patch tested?

existing tests

Closes #26968 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-21 09:56:15 -08:00
Wing Yew Poon c72f88b0ba [SPARK-17398][SQL] Fix ClassCastException when querying partitioned JSON table
### What changes were proposed in this pull request?

When querying a partitioned table with format `org.apache.hive.hcatalog.data.JsonSerDe` and more than one task runs in each executor concurrently, the following exception is encountered:

`java.lang.ClassCastException: java.util.ArrayList cannot be cast to org.apache.hive.hcatalog.data.HCatRecord`

The exception occurs in `HadoopTableReader.fillObject`.

`org.apache.hive.hcatalog.data.JsonSerDe#initialize` populates a `cachedObjectInspector` field by calling `HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector`, which is not thread-safe; this `cachedObjectInspector` is returned by `JsonSerDe#getObjectInspector`.

We protect against this Hive bug by synchronizing on an object when we need to call `initialize` on `org.apache.hadoop.hive.serde2.Deserializer` instances (which may be `JsonSerDe` instances). By doing so, the `ObjectInspector` for the `Deserializer` of the partitions of the JSON table and that of the table `SerDe` are the same cached `ObjectInspector` and `HadoopTableReader.fillObject` then works correctly. (If the `ObjectInspector`s are different, then a bug in `HCatRecordObjectInspector` causes an `ArrayList` to be created instead of an `HCatRecord`, resulting in the `ClassCastException` that is seen.)

### Why are the changes needed?

To avoid HIVE-15773 / HIVE-21752.

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

No.

### How was this patch tested?

Tested manually on a cluster with a partitioned JSON table and running a query using more than one core per executor. Before this change, the ClassCastException happens consistently. With this change it does not happen.

Closes #26895 from wypoon/SPARK-17398.

Authored-by: Wing Yew Poon <wypoon@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-12-20 10:39:26 -08:00
Sean Owen 7dff3b125d [SPARK-30272][SQL][CORE] Remove usage of Guava that breaks in 27; replace with workalikes
### What changes were proposed in this pull request?

Remove usages of Guava that no longer work in Guava 27, and replace with workalikes. I'll comment on key types of changes below.

### Why are the changes needed?

Hadoop 3.2.1 uses Guava 27, so this helps us avoid problems running on Hadoop 3.2.1+ and generally lowers our exposure to Guava.

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

Should not be, but see notes below on hash codes and toString.

### How was this patch tested?

Existing tests will verify whether these changes break anything for Guava 14.
I manually built with an updated version and it compiles with Guava 27; tests running manually locally now.

Closes #26911 from srowen/SPARK-30272.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-20 08:55:04 -06:00
Prakhar Jain 07b04c4c72 [SPARK-29938][SQL] Add batching support in Alter table add partition flow
### What changes were proposed in this pull request?
Add batching support in Alter table add partition flow. Also calculate new partition sizes faster by doing listing in parallel.

### Why are the changes needed?
This PR split the the single createPartitions() call AlterTableAddPartition flow into smaller batches, which could prevent
 - SocketTimeoutException: Adding thousand of partitions in Hive metastore itself takes lot of time. Because of this hive client fails with SocketTimeoutException.

- Hive metastore from OOM (caused by millions of partitions).

It will also try to gather stats (total size of all files in all new partitions) faster by parallely listing the new partition paths.

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

### How was this patch tested?
Added UT.
Also tested on a cluster in HDI with 15000 partitions with remote metastore server. Without batching - operation fails with SocketTimeoutException, With batching it finishes in 25 mins.

Closes #26569 from prakharjain09/add_partition_batching_r1.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-20 08:54:14 -06:00
Niranjan Artal 0d2ef3ae2b [SPARK-30300][SQL][WEB-UI] Fix updating the UI max value string when driver updates the same metric id as the tasks
### What changes were proposed in this pull request?

In this PR, For a given metrics id we are checking if the driver side accumulator's value is greater than max of all stages value. If it's true, then we are removing that entry from the Hashmap. By doing this, for this metrics, "driver" would be displayed on the UI(As the driver would have the maximum value)

### Why are the changes needed?

This PR fixes https://issues.apache.org/jira/browse/SPARK-30300. Currently driver's metric value is not compared while caluculating the max.

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

For the metrics where driver's value is greater than max of all stages, this is the change.
Previous : (min, median, max (stageId 0( attemptId 1): taskId 2))
Now:   (min, median, max (driver))

### How was this patch tested?

Ran unit tests.

Closes #26941 from nartal1/SPARK-30300.

Authored-by: Niranjan Artal <nartal@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-12-20 07:29:28 -06:00
Kent Yao 12249fcdc7 [SPARK-30301][SQL] Fix wrong results when datetimes as fields of complex types
### What changes were proposed in this pull request?

When date and timestamp values are fields of arrays, maps, etc, we convert them to hive string using `toString`. This makes the result wrong before the default transition ’1582-10-15‘.

https://bugs.openjdk.java.net/browse/JDK-8061577?focusedCommentId=13566712&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-13566712

cases to reproduce:

```sql
+-- !query 47
+select array(cast('1582-10-13' as date), date '1582-10-14', date '1582-10-15', null)
+-- !query 47 schema
+struct<array(CAST(1582-10-13 AS DATE), DATE '1582-10-14', DATE '1582-10-15', CAST(NULL AS DATE)):array<date>>
+-- !query 47 output
+[1582-10-03,1582-10-04,1582-10-15,null]
+
+
+-- !query 48
+select cast('1582-10-13' as date), date '1582-10-14', date '1582-10-15'
+-- !query 48 schema
+struct<CAST(1582-10-13 AS DATE):date,DATE '1582-10-14':date,DATE '1582-10-15':date>
+-- !query 48 output
+1582-10-13     1582-10-14      1582-10-15
```

other refencences
https://github.com/h2database/h2database/issues/831
### Why are the changes needed?

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

yes, complex types containing datetimes in `spark-sql `script and thrift server can result same as self-contained spark app or `spark-shell` script
### How was this patch tested?

add uts

Closes #26942 from yaooqinn/SPARK-30301.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-20 19:21:43 +08:00
jiake a296d15235 [SPARK-30291] catch the exception when doing materialize in AQE
### What changes were proposed in this pull request?
AQE need catch the exception when doing materialize. And then user can get more information about the exception when enable AQE.

### Why are the changes needed?
provide more cause about the exception when doing materialize

### Does this PR introduce any user-facing change?
Before this PR,  the error in the added unit test is
java.lang.RuntimeException: Invalid bucket file file:///${SPARK_HOME}/assembly/spark-warehouse/org.apache.spark.sql.execution.adaptive.AdaptiveQueryExecSuite/bucketed_table/part-00000-3551343c-d003-4ada-82c8-45c712a72efe-c000.snappy.parquet

After this PR, the error in the added unit test is:
org.apache.spark.SparkException: Adaptive execution failed due to stage materialization failures.

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

Closes #26931 from JkSelf/catchMoreException.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-12-20 00:23:26 -08:00
Wenchen Fan 18e8d1d5b2 [SPARK-30307][SQL] remove ReusedQueryStageExec
### What changes were proposed in this pull request?

When we reuse exchanges in AQE, what we produce is `ReuseQueryStage(QueryStage(Exchange))`. This PR changes it to `QueryStage(ReusedExchange(Exchange))`.

This PR also fixes an issue in `LocalShuffleReaderExec.outputPartitioning`. We can only preserve the partitioning if we read one mapper per task.

### Why are the changes needed?

`QueryStage` is light-weighted and we don't need to reuse its instance. What we really care is to reuse the exchange instance, which has heavy states (e.g. broadcasted valued, submitted map stage).

To simplify the framework, we should use the existing `ReusedExchange` node to do the reuse work, instead of creating a new node.

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

no

### How was this patch tested?

existing tests

Closes #26952 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-12-19 20:56:06 -08:00
Aman Omer 726f6d3e3c [SPARK-30184][SQL] Implement a helper method for aliasing functions
### What changes were proposed in this pull request?
This PR is to use `expressionWithAlias` for remaining functions for which alias name can be used. Remaining functions are:
`Average, First, Last, ApproximatePercentile, StddevSamp, VarianceSamp`

PR https://github.com/apache/spark/pull/26712 introduced `expressionWithAlias`
### Why are the changes needed?
Error message is wrong when alias name is used for above mentioned functions.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Manually

Closes #26808 from amanomer/fncAlias.

Lead-authored-by: Aman Omer <amanomer1996@gmail.com>
Co-authored-by: Aman Omer <40591404+amanomer@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-20 12:49:16 +08:00
Maxim Gekk dea18231d4 [SPARK-30309][SQL] Mark Filter as a sealed class
### What changes were proposed in this pull request?
Added the `sealed` keyword to the `Filter` class

### Why are the changes needed?
To do not miss handling of new filters in a datasource in the future. For example, `AlwaysTrue` and `AlwaysFalse` were added recently by https://github.com/apache/spark/pull/23606

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

### How was this patch tested?
By existing tests.

Closes #26950 from MaxGekk/sealed-filter.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-19 12:30:34 -08:00
Jungtaek Lim (HeartSaVioR) ab87bfd087 [SPARK-29450][SS] Measure the number of output rows for streaming aggregation with append mode
### What changes were proposed in this pull request?

This patch addresses missing metric, the number of output rows for streaming aggregation with append mode. Other modes are correctly measuring it.

### Why are the changes needed?

Without the patch, the value for such metric is always 0.

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

No.

### How was this patch tested?

Unit test added. Also manually tested with below query:

> query

```
import spark.implicits._

spark.conf.set("spark.sql.shuffle.partitions", "5")

val df = spark.readStream
  .format("rate")
  .option("rowsPerSecond", 1000)
  .load()
  .withWatermark("timestamp", "5 seconds")
  .selectExpr("timestamp", "mod(value, 100) as mod", "value")
  .groupBy(window($"timestamp", "10 seconds"), $"mod")
  .agg(max("value").as("max_value"), min("value").as("min_value"), avg("value").as("avg_value"))

val query = df
  .writeStream
  .format("memory")
  .option("queryName", "test")
  .outputMode("append")
  .start()

query.awaitTermination()
```

> before the patch

![screenshot-before-SPARK-29450](https://user-images.githubusercontent.com/1317309/69023217-58d7bc80-0a01-11ea-8cac-40f1cced6d16.png)

> after the patch

![screenshot-after-SPARK-29450](https://user-images.githubusercontent.com/1317309/69023221-5c6b4380-0a01-11ea-8a66-7bf1b7d09fc7.png)

Closes #26104 from HeartSaVioR/SPARK-29450.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-19 18:20:41 +09:00
Xingbo Jiang 2af5237fe8 [SPARK-29918][SQL][FOLLOWUP][TEST] Fix arrayOffset in RecordBinaryComparatorSuite
### What changes were proposed in this pull request?

As mentioned in https://github.com/apache/spark/pull/26548#pullrequestreview-334345333, some test cases in `RecordBinaryComparatorSuite` use a fixed arrayOffset when writing to long arrays, this  could lead to weird stuff including crashing with a SIGSEGV.

This PR fix the problem by computing the arrayOffset based on `Platform.LONG_ARRAY_OFFSET`.

### How was this patch tested?
Tested locally. Previously, when we try to add `System.gc()` between write into long array and compare by RecordBinaryComparator, there is a chance to hit JVM crash with SIGSEGV like:
```
#
# A fatal error has been detected by the Java Runtime Environment:
#
#  SIGSEGV (0xb) at pc=0x00007efc66970bcb, pid=11831, tid=0x00007efc0f9f9700
#
# JRE version: OpenJDK Runtime Environment (8.0_222-b10) (build 1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10)
# Java VM: OpenJDK 64-Bit Server VM (25.222-b10 mixed mode linux-amd64 compressed oops)
# Problematic frame:
# V  [libjvm.so+0x5fbbcb]
#
# Core dump written. Default location: /home/jenkins/workspace/sql/core/core or core.11831
#
# An error report file with more information is saved as:
# /home/jenkins/workspace/sql/core/hs_err_pid11831.log
#
# If you would like to submit a bug report, please visit:
#   http://bugreport.java.com/bugreport/crash.jsp
#
```
After the fix those test cases didn't crash the JVM anymore.

Closes #26939 from jiangxb1987/rbc.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-19 17:01:40 +08:00
Gengliang Wang ab8eb86a77 Revert "[SPARK-29629][SQL] Support typed integer literal expression"
This reverts commit 8e667db5d8.

Closes #26940 from gengliangwang/revert_Spark_29629.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-19 16:34:27 +09:00
ulysses 1e48b43a0e [SPARK-30254][SQL] Fix LikeSimplification optimizer to use a given escapeChar
Since [25001](https://github.com/apache/spark/pull/25001), spark support like escape syntax.

We should also sync the escape used by `LikeSimplification`.

Avoid optimize failed.

No.

Add UT.

Closes #26880 from ulysses-you/SPARK-30254.

Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-18 15:56:13 -08:00
chenliang abfc267f0c [SPARK-30262][SQL] Avoid NumberFormatException when totalSize is empty
### What changes were proposed in this pull request?

We could get the Partitions Statistics Info.But in some specail case, The Info  like  totalSize,rawDataSize,rowCount maybe empty. When we do some ddls like
`desc formatted partition` ,the NumberFormatException is showed as below:
```
spark-sql> desc formatted table1 partition(year='2019', month='10', day='17', hour='23');
19/10/19 00:02:40 ERROR SparkSQLDriver: Failed in [desc formatted table1 partition(year='2019', month='10', day='17', hour='23')]
java.lang.NumberFormatException: Zero length BigInteger
at java.math.BigInteger.(BigInteger.java:411)
at java.math.BigInteger.(BigInteger.java:597)
at scala.math.BigInt$.apply(BigInt.scala:77)
at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$31.apply(HiveClientImpl.scala:1056)
```
Although we can use 'Analyze table partition ' to update the totalSize,rawDataSize or rowCount, it's unresonable for normal SQL to throw NumberFormatException for Empty totalSize.We should fix the empty case when readHiveStats.

### Why are the changes needed?

This is a related to the robustness of the code and may lead to unexpected exception in some unpredictable situation.Here is the case:
<img width="981" alt="image" src="https://user-images.githubusercontent.com/20614350/70845771-7b88b400-1e8d-11ea-95b0-df5c58097d7d.png">

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

No

### How was this patch tested?

manual

Closes #26892 from southernriver/SPARK-30262.

Authored-by: chenliang <southernriver@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-18 15:12:32 -08:00
Jalpan Randeri f15eee18cc [SPARK-29493][SQL] Arrow MapType support
### What changes were proposed in this pull request?
This pull request add support for Arrow MapType into Spark SQL.

### Why are the changes needed?
Without this change User's of spark are not able to query data in spark if one of columns is stored as map and Apache Arrow execution mode is preferred by user.
More info: https://issues.apache.org/jira/projects/SPARK/issues/SPARK-29493

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

### How was this patch tested?
Introduced few unit tests around map type in existing arrow test suit

Closes #26512 from jalpan-randeri/feature-arrow-java-map-type.

Authored-by: Jalpan Randeri <randerij@amazon.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-18 23:59:27 +09:00
Kent Yao d38f816748 [MINOR][SQL][DOC] Fix some format issues in Dataset API Doc
### What changes were proposed in this pull request?

fix listing up format issues in Dataset API Doc (scala & java)

### Why are the changes needed?

improve doc

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

yes, API doc changing

### How was this patch tested?

no

Closes #26922 from yaooqinn/datasetdoc.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-18 15:25:40 +09:00
Aman Omer 297f406425 [SPARK-29600][SQL] ArrayContains function may return incorrect result for DecimalType
### What changes were proposed in this pull request?
Use `TypeCoercion.findWiderTypeForTwo()` instead of `TypeCoercion.findTightestCommonType()` while preprocessing `inputTypes` in `ArrayContains`.

### Why are the changes needed?
`TypeCoercion.findWiderTypeForTwo()` also handles cases for DecimalType.

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

### How was this patch tested?
Test cases to be added.

Closes #26811 from amanomer/29600.

Authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-18 01:30:28 +08:00
Zhenhua Wang 18431c7baa [SPARK-30269][SQL] Should use old partition stats to decide whether to update stats when analyzing partition
### What changes were proposed in this pull request?
It's an obvious bug: currently when analyzing partition stats, we use old table stats to compare with newly computed stats to decide whether it should update stats or not.

### Why are the changes needed?
bug fix

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

### How was this patch tested?
add new tests

Closes #26908 from wzhfy/failto_update_part_stats.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-17 22:21:26 +09:00
Kent Yao bf7215c510 [SPARK-30066][SQL][FOLLOWUP] Remove size field for interval column cache
### What changes were proposed in this pull request?

A followup for #26699, clear the size field for interval column cache, which is needless and can reduce the memory cost.

### Why are the changes needed?
followup

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

no

### How was this patch tested?

existing ut.

Closes #26906 from yaooqinn/SPARK-30066-f.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-17 15:36:21 +09:00
ulysses 1da7e8295c [SPARK-30201][SQL] HiveOutputWriter standardOI should use ObjectInspectorCopyOption.DEFAULT
### What changes were proposed in this pull request?

Now spark use `ObjectInspectorCopyOption.JAVA` as oi option which will convert any string to UTF-8 string. When write non UTF-8 code data, then `EFBFBD` will appear.
We should use `ObjectInspectorCopyOption.DEFAULT` to support pass the bytes.

### Why are the changes needed?

Here is the way to reproduce:
1. make a file contains 16 radix 'AABBCC' which is not the UTF-8 code.
2. create table test1 (c string) location '$file_path';
3. select hex(c) from test1; // AABBCC
4. craete table test2 (c string) as select c from test1;
5. select hex(c) from test2; // EFBFBDEFBFBDEFBFBD

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

No.

### How was this patch tested?

Closes #26831 from ulysses-you/SPARK-30201.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-17 12:15:53 +08:00
Terry Kim e75d9afb2f [SPARK-30094][SQL] Apply current namespace for the single-part table name
### What changes were proposed in this pull request?

This PR applies the current namespace for the single-part table name if the current catalog is a non-session catalog.

Note that the reason the current namespace is not applied for the session catalog is that the single-part name could be referencing a temp view which doesn't belong to any namespaces. The empty namespace for a table inside the session catalog is resolved by the session catalog implementation.

### Why are the changes needed?

It's fixing the following bug where the current namespace is not respected:
```
sql("CREATE TABLE testcat.ns.t USING foo AS SELECT 1 AS id")
sql("USE testcat.ns")
sql("SHOW CURRENT NAMESPACE").show
+-------+---------+
|catalog|namespace|
+-------+---------+
|testcat|       ns|
+-------+---------+

// `t` is not resolved since the current namespace `ns` is not used.
sql("DESCRIBE t").show
Failed to analyze query: org.apache.spark.sql.AnalysisException: Table not found: t;;
```

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

Yes, the above `DESCRIBE` command will succeed.

### How was this patch tested?

Added tests.

Closes #26894 from imback82/current_namespace.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-17 11:13:27 +08:00
Yuming Wang 696288f623 [INFRA] Reverts commit 56dcd79 and c216ef1
### What changes were proposed in this pull request?
1. Revert "Preparing development version 3.0.1-SNAPSHOT": 56dcd79

2. Revert "Preparing Spark release v3.0.0-preview2-rc2": c216ef1

### Why are the changes needed?
Shouldn't change master.

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

### How was this patch tested?
manual test:
https://github.com/apache/spark/compare/5de5e46..wangyum:revert-master

Closes #26915 from wangyum/revert-master.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-12-16 19:57:44 -07:00
Yuming Wang 56dcd79992 Preparing development version 3.0.1-SNAPSHOT 2019-12-17 01:57:27 +00:00
Yuming Wang c216ef1d03 Preparing Spark release v3.0.0-preview2-rc2 2019-12-17 01:57:21 +00:00
Maxim Gekk b03ce63c05 [SPARK-30258][TESTS] Eliminate warnings of deprecated Spark APIs in tests
### What changes were proposed in this pull request?
In the PR, I propose to move all tests that use deprecated Spark APIs to separate test classes, and add the annotation:
```scala
deprecated("This test suite will be removed.", "3.0.0")
```
The annotation suppress warnings from already deprecated methods and classes.

### Why are the changes needed?
The warnings about deprecated Spark APIs in tests does not indicate any issues because the tests use such APIs intentionally. Eliminating the warnings allows to highlight other warnings that could show real problems.

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

### How was this patch tested?
By existing test suites and by
- DeprecatedAvroFunctionsSuite
- DeprecatedDateFunctionsSuite
- DeprecatedDatasetAggregatorSuite
- DeprecatedStreamingAggregationSuite
- DeprecatedWholeStageCodegenSuite

Closes #26885 from MaxGekk/eliminate-deprecate-warnings.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-16 18:24:32 -06:00
Niranjan Artal dddfeca175 [SPARK-30209][SQL][WEB-UI] Display stageId, attemptId and taskId for max metrics in Spark UI
### What changes were proposed in this pull request?

SPARK-30209 discusses about adding additional metrics such as stageId, attempId and taskId for max metrics. We have the data required to display in LiveStageMetrics. Need to capture and pass these metrics to display on the UI. To minimize memory used for variables, we are saving maximum of each metric id per stage. So per stage additional memory usage is (#metrics * 4 * sizeof(Long)).
Then max is calculated for each metric id among all stages which is passed in the stringValue method. Memory used is minimal. Ran the benchmark for runtime. Stage.Proc time has increased to around 1.5-2.5x but the Aggregate time has decreased.

### Why are the changes needed?

These additional metrics stageId, attemptId and taskId could help in debugging the jobs quicker.  For a  given operator, it will be easy to identify the task which is taking maximum time to complete from the SQL tab itself.

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

Yes. stageId, attemptId and taskId is shown only for executor side metrics. For driver metrics, "(driver)" is displayed on UI.
![image (3)](https://user-images.githubusercontent.com/50492963/70763041-929d9980-1d07-11ea-940f-88ac6bdce9b5.png)

"Driver"
![image (4)](https://user-images.githubusercontent.com/50492963/70763043-94675d00-1d07-11ea-95ab-3478728cb435.png)

### How was this patch tested?

Manually tested, ran benchmark script for runtime.

Closes #26843 from nartal1/SPARK-30209.

Authored-by: Niranjan Artal <nartal@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-12-16 15:27:34 -06:00
HyukjinKwon 23b1312324 [SPARK-30200][DOCS][FOLLOW-UP] Add documentation for explain(mode: String)
### What changes were proposed in this pull request?

This PR adds the documentation of the new `mode` added to `Dataset.explain`.

### Why are the changes needed?

To let users know the new modes.

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

No (doc-only change).

### How was this patch tested?

Manually built the doc:
![Screen Shot 2019-12-16 at 3 34 28 PM](https://user-images.githubusercontent.com/6477701/70884617-d64f1680-2019-11ea-9336-247ade7f8768.png)

Closes #26903 from HyukjinKwon/SPARK-30200-doc.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-16 21:35:37 +09:00
Wenchen Fan fdcd0e71b9 [SPARK-30192][SQL] support column position in DS v2
### What changes were proposed in this pull request?

update DS v2 API to support add/alter column with column position

### Why are the changes needed?

We have a parser rule for column position, but we fail the query if it's specified, because the builtin catalog can't support add/alter column with column position.

Since we have the catalog plugin API now, we should let the catalog implementation to decide if it supports column position or not.

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

not yet

### How was this patch tested?

new tests

Closes #26817 from cloud-fan/parser.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-16 18:55:17 +08:00
Terry Kim 72f5597ce2 [SPARK-30104][SQL][FOLLOWUP] Remove LookupCatalog.AsTemporaryViewIdentifier
### What changes were proposed in this pull request?

As discussed in https://github.com/apache/spark/pull/26741#discussion_r357504518, `LookupCatalog.AsTemporaryViewIdentifier` is no longer used and can be removed.

### Why are the changes needed?

Code clean up

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

No

### How was this patch tested?

Removed tests that were testing solely `AsTemporaryViewIdentifier` extractor.

Closes #26897 from imback82/30104-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-16 17:43:01 +08:00
Boris Boutkov 3bf5498b4a [MINOR][DOCS] Fix documentation for slide function
### What changes were proposed in this pull request?

This PR proposes to fix documentation for slide function. Fixed the spacing issue and added some parameter related info.

### Why are the changes needed?

Documentation improvement

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

No (doc-only change).

### How was this patch tested?

Manually tested by documentation build.

Closes #26896 from bboutkov/pyspark_doc_fix.

Authored-by: Boris Boutkov <boris.boutkov@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-16 16:29:09 +09:00
HyukjinKwon 0a2afcec7d [SPARK-30200][SQL][FOLLOW-UP] Expose only explain(mode: String) in Scala side, and clean up related codes
### What changes were proposed in this pull request?

This PR mainly targets:

1. Expose only explain(mode: String) in Scala side
2. Clean up related codes
    - Hide `ExplainMode` under private `execution` package. No particular reason but just because `ExplainUtils` exists there
    - Use `case object` + `trait` pattern in `ExplainMode` to look after `ParseMode`.
    -  Move `Dataset.toExplainString` to `QueryExecution.explainString` to look after `QueryExecution.simpleString`, and deduplicate the codes at `ExplainCommand`.
    - Use `ExplainMode` in `ExplainCommand` too.
    - Add `explainString` to `PythonSQLUtils` to avoid unexpected test failure of PySpark during refactoring Scala codes side.

### Why are the changes needed?

To minimised exposed APIs, deduplicate, and clean up.

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

`Dataset.explain(mode: ExplainMode)` will be removed (which only exists in master).

### How was this patch tested?

Manually tested and existing tests should cover.

Closes #26898 from HyukjinKwon/SPARK-30200-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-16 14:42:35 +09:00
Maxim Gekk 67b644c3d7 [SPARK-30166][SQL] Eliminate compilation warnings in JSONOptions
### What changes were proposed in this pull request?
In the PR, I propose to replace `setJacksonOptions()` in `JSONOptions` by `buildJsonFactory()` which builds `JsonFactory` using `JsonFactoryBuilder`. This allows to avoid using **deprecated** feature configurations from `JsonParser.Feature`.

### Why are the changes needed?
- The changes eliminate the following compilation warnings in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala`:
```
    Warning:Warning:line (137)Java enum ALLOW_NUMERIC_LEADING_ZEROS in Java enum Feature is deprecated: see corresponding Javadoc for more information.
    factory.configure(JsonParser.Feature.ALLOW_NUMERIC_LEADING_ZEROS, allowNumericLeadingZeros)
    Warning:Warning:line (138)Java enum ALLOW_NON_NUMERIC_NUMBERS in Java enum Feature is deprecated: see corresponding Javadoc for more information.
    factory.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, allowNonNumericNumbers)
    Warning:Warning:line (139)Java enum ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER in Java enum Feature is deprecated: see corresponding Javadoc for more information.
    factory.configure(JsonParser.Feature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER,
    Warning:Warning:line (141)Java enum ALLOW_UNQUOTED_CONTROL_CHARS in Java enum Feature is deprecated: see corresponding Javadoc for more information.
    factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS, allowUnquotedControlChars)
```
- This put together building JsonFactory and set options from JSONOptions. So, we will not forget to call `setJacksonOptions` in the future.

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

### How was this patch tested?
By `JsonSuite`, `JsonFunctionsSuite`, `JsonExpressionsSuite`.

Closes #26797 from MaxGekk/eliminate-warning.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-15 08:45:57 -06:00
fuwhu 4cbef8988e [SPARK-30259][SQL] Fix CREATE TABLE behavior when session catalog is specified explicitly
### What changes were proposed in this pull request?
Fix bug : CREATE TABLE throw error when session catalog specified explicitly.

### Why are the changes needed?
Currently, Spark throw error when the session catalog is specified explicitly in "CREATE TABLE" and "CREATE TABLE AS SELECT" command, eg. 
> CREATE TABLE spark_catalog.tbl USING json AS SELECT 1 AS i;

the error message is like below:
> 19/12/14 10:56:08 INFO HiveMetaStore: 0: get_table : db=spark_catalog tbl=tbl
> 19/12/14 10:56:08 INFO audit: ugi=fuwhu ip=unknown-ip-addr      cmd=get_table : db=spark_catalog tbl=tbl
> 19/12/14 10:56:08 INFO HiveMetaStore: 0: get_database: spark_catalog
> 19/12/14 10:56:08 INFO audit: ugi=fuwhu ip=unknown-ip-addr      cmd=get_database: spark_catalog
> 19/12/14 10:56:08 WARN ObjectStore: Failed to get database spark_catalog, returning NoSuchObjectException
> Error in query: Database 'spark_catalog' not found;

### Does this PR introduce any user-facing change?
Yes, after this PR, "CREATE TALBE" and "CREATE TABLE AS SELECT" can complete successfully when session catalog "spark_catalog" specified explicitly.

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

Closes #26887 from fuwhu/SPARK-30259.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-14 15:36:14 -08:00
Takeshi Yamamuro f483a13d4a [SPARK-30231][SQL][PYTHON][FOLLOWUP] Make error messages clear in PySpark df.explain
### What changes were proposed in this pull request?

This pr is a followup of #26861 to address minor comments from viirya.

### Why are the changes needed?

For better error messages.

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

No.

### How was this patch tested?

Manually tested.

Closes #26886 from maropu/SPARK-30231-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-14 14:26:50 -08:00
Kent Yao d3ec8b1735 [SPARK-30066][SQL] Support columnar execution on interval types
### What changes were proposed in this pull request?

Columnar execution support for interval types

### Why are the changes needed?

support cache tables with interval columns
improve performance too

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

Yes cache table with accept interval columns

### How was this patch tested?

add ut

Closes #26699 from yaooqinn/SPARK-30066.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-14 13:10:46 -08:00
John Ayad f197204f03 [SPARK-30236][SQL][DOCS] Clarify date and time patterns supported in docs
### What changes were proposed in this pull request?

Link to appropriate Java Class with list of date/time patterns supported

### Why are the changes needed?

Avoid confusion on the end-user's side of things, as seen in questions like [this](https://stackoverflow.com/questions/54496878/date-format-conversion-is-adding-1-year-to-the-border-dates) on StackOverflow

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

Yes, Docs are updated.

### How was this patch tested?

`date_format`:
![image](https://user-images.githubusercontent.com/2394761/70796647-b5c55900-1d9a-11ea-89f9-7a8661641c09.png)

`to_unix_timestamp`:
![image](https://user-images.githubusercontent.com/2394761/70796664-c07fee00-1d9a-11ea-9029-e82d899e3f59.png)

`unix_timestamp`:
![image](https://user-images.githubusercontent.com/2394761/70796688-caa1ec80-1d9a-11ea-8868-a18c437a5d49.png)

`from_unixtime`:
![image](https://user-images.githubusercontent.com/2394761/70796703-d4c3eb00-1d9a-11ea-85fe-3c672e0cda28.png)

`to_date`:
![image](https://user-images.githubusercontent.com/2394761/70796718-dd1c2600-1d9a-11ea-81f4-a0966eeb0f1d.png)

`to_timestamp`:
![image](https://user-images.githubusercontent.com/2394761/70796735-e6a58e00-1d9a-11ea-8ef7-d3e1d9b5370f.png)

Closes #26864 from johnhany97/SPARK-30236.

Authored-by: John Ayad <johnhany97@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-14 13:08:15 -08:00
Burak Yavuz 4c37a8a3f4 [SPARK-30143][SS] Add a timeout on stopping a streaming query
### What changes were proposed in this pull request?

Add a timeout configuration for StreamingQuery.stop()

### Why are the changes needed?

The stop() method on a Streaming Query awaits the termination of the stream execution thread. However, the stream execution thread may block forever depending on the streaming source implementation (like in Kafka, which runs UninterruptibleThreads).

This causes control flow applications to hang indefinitely as well. We'd like to introduce a timeout to stop the execution thread, so that the control flow thread can decide to do an action if a timeout is hit.

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

By default, no. If the timeout configuration is set, then a TimeoutException will be thrown if a stream cannot be stopped within the given timeout.

### How was this patch tested?

Unit tests

Closes #26771 from brkyvz/stopTimeout.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-12-13 15:16:00 -08:00
Gengliang Wang 4da9780bc0 Revert "[SPARK-30230][SQL] Like ESCAPE syntax can not use '_' and '%'"
This reverts commit cada5beef7.

Closes #26883 from gengliangwang/revert.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-13 11:23:55 -08:00
Terry Kim ac9b1881a2 [SPARK-30248][SQL] Fix DROP TABLE behavior when session catalog name is provided in the identifier
### What changes were proposed in this pull request?

If a table name is qualified with session catalog name `spark_catalog`, the `DROP TABLE` command fails.

For example, the following

```
sql("CREATE TABLE tbl USING json AS SELECT 1 AS i")
sql("DROP TABLE spark_catalog.tbl")
```
fails with:
```
org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database 'spark_catalog' not found;
   at org.apache.spark.sql.catalyst.catalog.ExternalCatalog.requireDbExists(ExternalCatalog.scala:42)
   at org.apache.spark.sql.catalyst.catalog.ExternalCatalog.requireDbExists$(ExternalCatalog.scala:40)
   at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.requireDbExists(InMemoryCatalog.scala:45)
   at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.tableExists(InMemoryCatalog.scala:336)
```

This PR correctly resolves `spark_catalog` as a catalog.

### Why are the changes needed?

It's fixing a bug.

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

Yes, now, the `spark_catalog.tbl` in the above example is dropped as expected.

### How was this patch tested?

Added a test.

Closes #26878 from imback82/fix_drop_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-13 21:45:35 +08:00
Takeshi Yamamuro 64c7b94d64 [SPARK-30231][SQL][PYTHON] Support explain mode in PySpark df.explain
### What changes were proposed in this pull request?

This pr intends to support explain modes implemented in #26829 for PySpark.

### Why are the changes needed?

For better debugging info. in PySpark dataframes.

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

No.

### How was this patch tested?

Added UTs.

Closes #26861 from maropu/ExplainModeInPython.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-13 17:44:23 +09:00
Jungtaek Lim (HeartSaVioR) 94eb66593a [SPARK-30227][SQL] Add close() on DataWriter interface
### What changes were proposed in this pull request?

This patch adds close() method to the DataWriter interface, which will become the place to cleanup the resource.

### Why are the changes needed?

The lifecycle of DataWriter instance ends at either commit() or abort(). That makes datasource implementors to feel they can place resource cleanup in both sides, but abort() can be called when commit() fails; so they have to ensure they don't do double-cleanup if cleanup is not idempotent.

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

Depends on the definition of user; if they're developers of custom DSv2 source, they have to add close() in their DataWriter implementations. It's OK to just add close() with empty content as they should have already dealt with resource cleanup in commit/abort, but they would love to migrate the resource cleanup logic to close() as it avoids double cleanup. If they're just end users using the provided DSv2 source (regardless of built-in/3rd party), no change.

### How was this patch tested?

Existing tests.

Closes #26855 from HeartSaVioR/SPARK-30227.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-13 16:12:41 +08:00
Pablo Langa cb6d2b3f83 [SPARK-30040][SQL] DROP FUNCTION should do multi-catalog resolution
### What changes were proposed in this pull request?

Add DropFunctionStatement and make DROP FUNCTION go through the same catalog/table resolution framework of v2 commands.

### Why are the changes needed?

It's important to make all the commands have the same table resolution behavior, to avoid confusing
DROP FUNCTION namespace.function

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

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

### How was this patch tested?

Unit tests.

Closes #26854 from planga82/feature/SPARK-30040_DropFunctionV2Catalog.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-12 15:15:54 -08:00
Anton Okolnychyi 5114389aef [SPARK-30107][SQL] Expose nested schema pruning to all V2 sources
### What changes were proposed in this pull request?

This PR exposes the existing logic for nested schema pruning to all sources, which is in line with the description of `SupportsPushDownRequiredColumns` .

Right now, `SchemaPruning` (rule, not helper utility) is applied in the optimizer directly on certain instances of `Table` ignoring `SupportsPushDownRequiredColumns` that is part of `ScanBuilder`. I think it would be cleaner to perform schema pruning and filter push-down in one place. Therefore, this PR moves all the logic into `V2ScanRelationPushDown`.

### Why are the changes needed?

This change allows all V2 data sources to benefit from nested column pruning (if they support it).

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

No.

### How was this patch tested?

This PR mostly relies on existing tests. On top, it adds one test to verify that top-level schema pruning works as well as one test for predicates with subqueries.

Closes #26751 from aokolnychyi/nested-schema-pruning-ds-v2.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2019-12-12 13:40:46 -08:00
Wenchen Fan 982f72f4c3 [SPARK-30238][SQL] hive partition pruning can only support string and integral types
### What changes were proposed in this pull request?

Check the partition column data type and only allow string and integral types in hive partition pruning.

### Why are the changes needed?

Currently we only support string and integral types in hive partition pruning, but the check is done for literals. If the predicate is `InSet`, then there is no literal and we may pass an unsupported partition predicate to Hive and cause problems.

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

yes. fix a bug. A query fails before and can run now.

### How was this patch tested?

a new test

Closes #26871 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-12 13:07:20 -08:00
ulysses cada5beef7 [SPARK-30230][SQL] Like ESCAPE syntax can not use '_' and '%'
### What changes were proposed in this pull request?

Since [25001](https://github.com/apache/spark/pull/25001), spark support like escape syntax.
But '%' and '_' is the reserve char in `Like` expression. We can not use them as escape char.

### Why are the changes needed?

Avoid some unexpect problem when using like escape syntax.

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

No.

### How was this patch tested?

Add UT.

Closes #26860 from ulysses-you/SPARK-30230.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2019-12-12 09:52:27 -08:00
HyukjinKwon cc087a3ac5 [SPARK-30162][SQL] Add PushedFilters to metadata in Parquet DSv2 implementation
### What changes were proposed in this pull request?

This PR proposes to add `PushedFilters` into metadata to show the pushed filters in Parquet DSv2 implementation. In case of ORC, it is already added at https://github.com/apache/spark/pull/24719/files#diff-0fc82694b20da3cd2cbb07206920eef7R62-R64

### Why are the changes needed?

In order for users to be able to debug, and to match with ORC.

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

```scala
spark.range(10).write.mode("overwrite").parquet("/tmp/foo")
spark.read.parquet("/tmp/foo").filter("5 > id").explain()
```

**Before:**

```
== Physical Plan ==
*(1) Project [id#20L]
+- *(1) Filter (isnotnull(id#20L) AND (5 > id#20L))
   +- *(1) ColumnarToRow
      +- BatchScan[id#20L] ParquetScan Location: InMemoryFileIndex[file:/tmp/foo], ReadSchema: struct<id:bigint>
```

**After:**

```
== Physical Plan ==
*(1) Project [id#13L]
+- *(1) Filter (isnotnull(id#13L) AND (5 > id#13L))
   +- *(1) ColumnarToRow
      +- BatchScan[id#13L] ParquetScan Location: InMemoryFileIndex[file:/tmp/foo], ReadSchema: struct<id:bigint>, PushedFilters: [IsNotNull(id), LessThan(id,5)]
```

### How was this patch tested?
Unittest were added and manually tested.

Closes #26857 from HyukjinKwon/SPARK-30162.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-12 08:33:33 -08:00
Aaron Lau fd39b6db34 [SQL] Typo in HashedRelation error
### What changes were proposed in this pull request?

Fixed typo in exception message of HashedRelations

### Why are the changes needed?

Better exception messages

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

No

### How was this patch tested?

No tests needed

Closes #26822 from aaron-lau/master.

Authored-by: Aaron Lau <aaron.lau@datadoghq.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-12 08:42:18 -06:00
Maxim Gekk 25de90e762 [SPARK-30170][SQL][MLLIB][TESTS] Eliminate compilation warnings: part 1
### What changes were proposed in this pull request?
- Replace `Seq[String]` by `Seq[_]` in `StopWordsRemoverSuite` because `String` type is unchecked due erasure.
- Throw an exception for default case in `MLTest.checkNominalOnDF` because we don't expect other attribute types currently.
- Explicitly cast float to double in `BigDecimal(y)`. This is what the `apply()` method does for `float`s.
- Replace deprecated `verifyZeroInteractions` by `verifyNoInteractions`.
- Equivalent replacement of `\0` by `\u0000` in `CSVExprUtilsSuite`
- Import `scala.language.implicitConversions` in `CollectionExpressionsSuite`, `HashExpressionsSuite` and in `ExpressionParserSuite`.

### Why are the changes needed?
The changes fix compiler warnings showed in the JIRA ticket https://issues.apache.org/jira/browse/SPARK-30170 . Eliminating the warning highlights other warnings which could take more attention to real problems.

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

### How was this patch tested?
By existing test suites `StopWordsRemoverSuite`, `AnalysisExternalCatalogSuite`, `CSVExprUtilsSuite`, `CollectionExpressionsSuite`, `HashExpressionsSuite`, `ExpressionParserSuite` and sub-tests of `MLTest`.

Closes #26799 from MaxGekk/eliminate-warning-2.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-12 08:38:15 -06:00
root1 2936507f94 [SPARK-30150][SQL] ADD FILE, ADD JAR, LIST FILE & LIST JAR Command do not accept quoted path
### What changes were proposed in this pull request?
`add file "abc.txt"` and `add file 'abc.txt'` are not supported.
For these two spark sql gives `FileNotFoundException`.
Only `add file abc.txt` is supported currently.

After these changes path can be given as quoted text for ADD FILE, ADD JAR, LIST FILE, LIST JAR commands in spark-sql

### Why are the changes needed?

In many of the spark-sql commands (like create table ,etc )we write path in quoted format only.  To maintain this consistency we should support quoted format with this command as well.

### Does this PR introduce any user-facing change?
Yes. Now users can write path with quotes.

### How was this patch tested?
Manually tested.

Closes #26779 from iRakson/SPARK-30150.

Authored-by: root1 <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-12 17:11:21 +08:00
Terry Kim 3741a36ebf [SPARK-30104][SQL][FOLLOWUP] V2 catalog named 'global_temp' should always be masked
### What changes were proposed in this pull request?

This is a follow up to #26741 to address the following:
1. V2 catalog named `global_temp` should always be masked.
2. #26741 introduces `CatalogAndIdentifer` that supersedes `CatalogObjectIdentfier`. This PR removes `CatalogObjectIdentfier` and its usages and replace them with `CatalogAndIdentifer`.
3. `CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog)` and `CatalogObjectIdentifier(catalog, ident) if isSessionCatalog(catalog)` are replaced with `NonSessionCatalogAndIdentifier` and `SessionCatalogAndIdentifier` respectively.

### Why are the changes needed?

To fix an existing with handling v2 catalog named `global_temp` and to simplify the code base.

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

No

### How was this patch tested?

Added new tests.

Closes #26853 from imback82/lookup_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-12 14:47:20 +08:00
jiake 1ced6c1544 [SPARK-30213][SQL] Remove the mutable status in ShuffleQueryStageExec
### What changes were proposed in this pull request?
Currently `ShuffleQueryStageExec `contain the mutable status, eg `mapOutputStatisticsFuture `variable. So It is not easy to pass when we copy `ShuffleQueryStageExec`. This PR will put the `mapOutputStatisticsFuture ` variable from `ShuffleQueryStageExec` to `ShuffleExchangeExec`. And then we can pass the value of `mapOutputStatisticsFuture ` when copying.

### Why are the changes needed?
In order to remove the mutable status in `ShuffleQueryStageExec`

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

### How was this patch tested?
Existing uts

Closes #26846 from JkSelf/removeMutableVariable.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-11 19:39:31 -08:00
Pablo Langa 9cf9304e17 [SPARK-30038][SQL] DESCRIBE FUNCTION should do multi-catalog resolution
### What changes were proposed in this pull request?

Add DescribeFunctionsStatement and make DESCRIBE FUNCTIONS go through the same catalog/table resolution framework of v2 commands.

### Why are the changes needed?

It's important to make all the commands have the same table resolution behavior, to avoid confusing
DESCRIBE FUNCTIONS namespace.function

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

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

### How was this patch tested?

Unit tests.

Closes #26840 from planga82/feature/SPARK-30038_DescribeFunction_V2Catalog.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-11 14:02:58 -08:00
Sean Owen 33f53cb2d5 [SPARK-30195][SQL][CORE][ML] Change some function, import definitions to work with stricter compiler in Scala 2.13
### What changes were proposed in this pull request?

See https://issues.apache.org/jira/browse/SPARK-30195 for the background; I won't repeat it here. This is sort of a grab-bag of related issues.

### Why are the changes needed?

To cross-compile with Scala 2.13 later.

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

No.

### How was this patch tested?

Existing tests for 2.12. I've been manually checking that this actually resolves the compile problems in 2.13 separately.

Closes #26826 from srowen/SPARK-30195.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-11 12:33:58 -08:00
Maxim Gekk e933539cdd [SPARK-29864][SPARK-29920][SQL] Strict parsing of day-time strings to intervals
### What changes were proposed in this pull request?
In the PR, I propose new implementation of `fromDayTimeString` which strictly parses strings in day-time formats to intervals. New implementation accepts only strings that match to a pattern defined by the `from` and `to`. Here is the mapping of user's bounds and patterns:
- `[+|-]D+ H[H]:m[m]:s[s][.SSSSSSSSS]` for **DAY TO SECOND**
- `[+|-]D+ H[H]:m[m]` for **DAY TO MINUTE**
- `[+|-]D+ H[H]` for **DAY TO HOUR**
- `[+|-]H[H]:m[m]s[s][.SSSSSSSSS]` for **HOUR TO SECOND**
- `[+|-]H[H]:m[m]` for **HOUR TO MINUTE**
- `[+|-]m[m]:s[s][.SSSSSSSSS]` for **MINUTE TO SECOND**

Closes #26327
Closes #26358

### Why are the changes needed?
- Improve user experience with Spark SQL, and respect to the bound specified by users.
- Behave the same as other broadly used DBMS - Oracle and MySQL.

### Does this PR introduce any user-facing change?
Yes, before:
```sql
spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE;
interval 1 weeks 3 days 11 hours 12 minutes
```
After:
```sql
spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE;
Error in query:
requirement failed: Interval string must match day-time format of '^(?<sign>[+|-])?(?<hour>\d{1,2}):(?<minute>\d{1,2})$': 10 11:12:13.123(line 1, pos 16)

== SQL ==
SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE
----------------^^^
```

### How was this patch tested?
- Added tests to `IntervalUtilsSuite`
- By `ExpressionParserSuite`
- Updated `literals.sql`

Closes #26473 from MaxGekk/strict-from-daytime-string.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-12 01:08:53 +08:00
Takeshi Yamamuro a59cb13cda [SPARK-30200][SQL][FOLLOWUP] Fix typo in ExplainMode
### What changes were proposed in this pull request?

This pr is a follow-up of #26829 to fix typos in ExplainMode.

### Why are the changes needed?

For better docs.

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

No.

### How was this patch tested?

N/A

Closes #26851 from maropu/SPARK-30200-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-11 08:17:53 -08:00
Terry Kim beae14d5ed [SPARK-30104][SQL] Fix catalog resolution for 'global_temp'
### What changes were proposed in this pull request?

`global_temp` is used as a database name to access global temp views. The current catalog lookup logic considers only the first element of multi-part name when it resolves a catalog. This results in using the session catalog even `global_temp` is used as a table name under v2 catalog. This PR addresses this by making sure multi-part name has two elements before using the session catalog.

### Why are the changes needed?

Currently, 'global_temp' can be used as a table name in certain commands (CREATE) but not in others (DESCRIBE):
```
// Assume "spark.sql.globalTempDatabase" is set to "global_temp".
sql(s"CREATE TABLE testcat.t (id bigint, data string) USING foo")
sql(s"CREATE TABLE testcat.global_temp (id bigint, data string) USING foo")
sql("USE testcat")

sql(s"DESCRIBE TABLE t").show
+---------------+---------+-------+
|       col_name|data_type|comment|
+---------------+---------+-------+
|             id|   bigint|       |
|           data|   string|       |
|               |         |       |
| # Partitioning|         |       |
|Not partitioned|         |       |
+---------------+---------+-------+

sql(s"DESCRIBE TABLE global_temp").show
org.apache.spark.sql.AnalysisException: Table not found: global_temp;;
  'DescribeTable 'UnresolvedV2Relation [global_temp], org.apache.spark.sql.connector.InMemoryTableSessionCatalog2f1af64f, `global_temp`, false
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:47)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:46)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:122)
```

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

Yes, `sql(s"DESCRIBE TABLE global_temp").show` in the above example now displays:
```
+---------------+---------+-------+
|       col_name|data_type|comment|
+---------------+---------+-------+
|             id|   bigint|       |
|           data|   string|       |
|               |         |       |
| # Partitioning|         |       |
|Not partitioned|         |       |
+---------------+---------+-------+
```
instead of throwing an exception.

### How was this patch tested?

Added new tests.

Closes #26741 from imback82/global_temp.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-11 16:56:42 +08:00
Sean Owen 3cc55f6a0a [SPARK-29392][CORE][SQL][FOLLOWUP] More removal of 'foo Symbol syntax for Scala 2.13
### What changes were proposed in this pull request?

Another continuation of https://github.com/apache/spark/pull/26748

### Why are the changes needed?

To cleanly cross compile with Scala 2.13.

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

None.

### How was this patch tested?

Existing tests

Closes #26842 from srowen/SPARK-29392.4.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-10 19:41:24 -08:00
Kent Yao 8f0eb7dc86 [SPARK-29587][SQL] Support SQL Standard type real as float(4) numeric as decimal
### What changes were proposed in this pull request?
The types decimal and numeric are equivalent. Both types are part of the SQL standard.

the real type is  4 bytes, variable-precision, inexact, 6 decimal digits precision, same as our float, part of the SQL standard.

### Why are the changes needed?

improve sql standard support
other dbs
https://www.postgresql.org/docs/9.3/datatype-numeric.html
https://prestodb.io/docs/current/language/types.html#floating-point
http://www.sqlservertutorial.net/sql-server-basics/sql-server-data-types/
MySQL treats REAL as a synonym for DOUBLE PRECISION (a nonstandard variation), unless the REAL_AS_FLOAT SQL mode is enabled.
In MySQL, NUMERIC is implemented as DECIMAL, so the following remarks about DECIMAL apply equally to NUMERIC.

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

no
### How was this patch tested?

add ut

Closes #26537 from yaooqinn/SPARK-29587.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-11 02:22:08 +08:00
Kent Yao 24c4ce1e64 [SPARK-28351][SQL][FOLLOWUP] Remove 'DELETE FROM' from unsupportedHiveNativeCommands
### What changes were proposed in this pull request?

Minor change, rm `DELETE FROM` from unsupported hive native operation, because it is supported in parser.

### Why are the changes needed?
clear ambiguous ambiguous

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

no

### How was this patch tested?

no

Closes #26836 from yaooqinn/SPARK-28351.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-10 09:54:50 -08:00
Takeshi Yamamuro 6103cf1960 [SPARK-30200][SQL] Add ExplainMode for Dataset.explain
### What changes were proposed in this pull request?

This pr intends to add `ExplainMode` for explaining `Dataset/DataFrame` with a given format mode (`ExplainMode`). `ExplainMode` has four types along with the SQL EXPLAIN command: `Simple`, `Extended`, `Codegen`, `Cost`, and `Formatted`.

For example, this pr enables users to explain DataFrame/Dataset with the `FORMATTED` format implemented in #24759;
```
scala> spark.range(10).groupBy("id").count().explain(ExplainMode.Formatted)
== Physical Plan ==
* HashAggregate (3)
+- * HashAggregate (2)
   +- * Range (1)

(1) Range [codegen id : 1]
Output: [id#0L]

(2) HashAggregate [codegen id : 1]
Input: [id#0L]

(3) HashAggregate [codegen id : 1]
Input: [id#0L, count#8L]
```

This comes from [the cloud-fan suggestion.](https://github.com/apache/spark/pull/24759#issuecomment-560211270)

### Why are the changes needed?

To follow the SQL EXPLAIN command.

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

No, this is just for a new API in Dataset.

### How was this patch tested?

Add tests in `ExplainSuite`.

Closes #26829 from maropu/DatasetExplain.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-10 09:51:29 -08:00
Yuanjian Li d9b3069412 [SPARK-30125][SQL] Remove PostgreSQL dialect
### What changes were proposed in this pull request?
Reprocess all PostgreSQL dialect related PRs, listing in order:

- #25158: PostgreSQL integral division support [revert]
- #25170: UT changes for the integral division support [revert]
- #25458: Accept "true", "yes", "1", "false", "no", "0", and unique prefixes as input and trim input for the boolean data type. [revert]
- #25697: Combine below 2 feature tags into "spark.sql.dialect" [revert]
- #26112: Date substraction support [keep the ANSI-compliant part]
- #26444: Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled" [revert]
- #26463: Cast to boolean support for PostgreSQL dialect [revert]
- #26584: Make the behavior of Postgre dialect independent of ansi mode config [keep the ANSI-compliant part]

### Why are the changes needed?
As the discussion in http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-PostgreSQL-dialect-td28417.html, we need to remove PostgreSQL dialect form code base for several reasons:
1. The current approach makes the codebase complicated and hard to maintain.
2. Fully migrating PostgreSQL workloads to Spark SQL is not our focus for now.

### Does this PR introduce any user-facing change?
Yes, the config `spark.sql.dialect` will be removed.

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

Closes #26763 from xuanyuanking/SPARK-30125.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-11 01:22:34 +08:00
Anton Okolnychyi a9f1809a2a [SPARK-30206][SQL] Rename normalizeFilters in DataSourceStrategy to be generic
### What changes were proposed in this pull request?

This PR renames `normalizeFilters` in `DataSourceStrategy` to be more generic as the logic is not specific to filters.

### Why are the changes needed?

These changes are needed to support PR #26751.

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

No.

### How was this patch tested?

Existing tests.

Closes #26830 from aokolnychyi/rename-normalize-exprs.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-10 07:49:22 -08:00
yi.wu aa9da9365f [SPARK-30151][SQL] Issue better error message when user-specified schema mismatched
### What changes were proposed in this pull request?

Issue better error message when user-specified schema and not match relation schema

### Why are the changes needed?

Inspired by https://github.com/apache/spark/pull/25248#issuecomment-559594305, user could get a weird error message when type mapping behavior change between Spark schema and datasource schema(e.g. JDBC). Instead of saying "SomeProvider does not allow user-specified schemas.", we'd better tell user what is really happening here to make user be more clearly about the error.

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

Yes, user will see error message changes.

### How was this patch tested?

Updated existed tests.

Closes #26781 from Ngone51/dev-mismatch-schema.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-10 20:56:21 +08:00
Luan 3d98c9f985 [SPARK-30179][SQL][TESTS] Improve test in SingleSessionSuite
### What changes were proposed in this pull request?

improve the temporary functions test in SingleSessionSuite by verifying the result in a query

### Why are the changes needed?

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

### How was this patch tested?

Closes #26812 from leoluan2009/SPARK-30179.

Authored-by: Luan <xuluan@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-10 10:57:32 +09:00
Sean Owen 36fa1980c2 [SPARK-30158][SQL][CORE] Seq -> Array for sc.parallelize for 2.13 compatibility; remove WrappedArray
### What changes were proposed in this pull request?

Use Seq instead of Array in sc.parallelize, with reference types.
Remove usage of WrappedArray.

### Why are the changes needed?

These both enable building on Scala 2.13.

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

None

### How was this patch tested?

Existing tests

Closes #26787 from srowen/SPARK-30158.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-09 14:41:48 -06:00
Jungtaek Lim (HeartSaVioR) 538b8d101c [SPARK-30159][SQL][FOLLOWUP] Fix lint-java via removing unnecessary imports
### What changes were proposed in this pull request?

This patch fixes the Java code style violations in SPARK-30159 (#26788) which are caught by lint-java (Github Action caught it and I can reproduce it locally). Looks like Jenkins build may have different policy on checking Java style check or less accurate.

### Why are the changes needed?

Java linter starts complaining.

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

No.

### How was this patch tested?

lint-java passed locally

This closes #26819

Closes #26818 from HeartSaVioR/SPARK-30159-FOLLOWUP.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-09 08:57:20 -08:00
Gengliang Wang a717d219a6 [SPARK-30159][SQL][TESTS] Fix the method calls of QueryTest.checkAnswer
### What changes were proposed in this pull request?

Before this PR, the method `checkAnswer` in Object `QueryTest` returns an optional string. It doesn't throw exceptions when errors happen.
The actual exceptions are thrown in the trait `QueryTest`.

However, there are some test suites(`StreamSuite`, `SessionStateSuite`, `BinaryFileFormatSuite`, etc.) that use the no-op method `QueryTest.checkAnswer` and expect it to fail test cases when the execution results don't match the expected answers.

After this PR:
1. the method `checkAnswer` in Object `QueryTest` will fail tests on errors or unexpected results.
2. add a new method `getErrorMessageInCheckAnswer`, which is exactly the same as the previous version of `checkAnswer`. There are some test suites use this one to customize the test failure message.
3. for the test suites that extend the trait `QueryTest`, we should use the method `checkAnswer` directly, instead of calling the method from Object `QueryTest`.

### Why are the changes needed?

We should fix these method calls to perform actual validations in test suites.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #26788 from gengliangwang/fixCheckAnswer.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-09 22:19:08 +09:00
fuwhu c2f29d5ea5 [SPARK-30138][SQL] Separate configuration key of max iterations for analyzer and optimizer
### What changes were proposed in this pull request?
separate the configuration keys "spark.sql.optimizer.maxIterations" and "spark.sql.analyzer.maxIterations".

### Why are the changes needed?
Currently, both Analyzer and Optimizer use conf "spark.sql.optimizer.maxIterations" to set the max iterations to run, which is a little confusing.
It is clearer to add a new conf "spark.sql.analyzer.maxIterations" for analyzer max iterations.

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

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

Closes #26766 from fuwhu/SPARK-30138.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-09 19:43:32 +09:00
Aman Omer dcea7a4c9a [SPARK-29883][SQL] Implement a helper method for aliasing bool_and() and bool_or()
### What changes were proposed in this pull request?
This PR introduces a method `expressionWithAlias` in class `FunctionRegistry` which is used to register function's constructor. Currently, `expressionWithAlias` is used to register `BoolAnd` & `BoolOr`.

### Why are the changes needed?
Error message is wrong when alias name is used for `BoolAnd` & `BoolOr`.

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

### How was this patch tested?
Tested manually.

For query,
`select every('true');`

Output before this PR,

> Error in query: cannot resolve 'bool_and('true')' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [string].; line 1 pos 7;

After this PR,

> Error in query: cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7;

Closes #26712 from amanomer/29883.

Authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-09 13:23:16 +08:00
Pablo Langa bca9de6684 [SPARK-29922][SQL] SHOW FUNCTIONS should do multi-catalog resolution
### What changes were proposed in this pull request?

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

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

### Why are the changes needed?

It's important to make all the commands have the same table resolution behavior, to avoid confusing
`SHOW FUNCTIONS LIKE namespace.function`

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

Yes. When running SHOW FUNCTIONS LIKE namespace.function Spark fails the command if the current catalog is set to a v2 catalog.

### How was this patch tested?

Unit tests.

Closes #26667 from planga82/feature/SPARK-29922_ShowFunctions_V2Catalog.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2019-12-08 20:15:09 -08:00
Kent Yao e88d74052b [SPARK-30147][SQL] Trim the string when cast string type to booleans
### What changes were proposed in this pull request?

Now, we trim the string when casting string value to those `canCast` types values, e.g. int, double, decimal, interval, date, timestamps, except for boolean.
This behavior makes type cast and coercion inconsistency in Spark.
Not fitting ANSI SQL standard either.
```
If TD is boolean, then
Case:
a) If SD is character string, then SV is replaced by
    TRIM ( BOTH ' ' FROM VE )
    Case:
    i) If the rules for literal in Subclause 5.3, “literal”, can be applied to SV to determine a valid
value of the data type TD, then let TV be that value.
   ii) Otherwise, an exception condition is raised: data exception — invalid character value for cast.
b) If SD is boolean, then TV is SV
```
In this pull request, we trim all the whitespaces from both ends of the string before converting it to a bool value. This behavior is as same as others, but a bit different from sql standard, which trim only spaces.

### Why are the changes needed?

Type cast/coercion consistency

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

yes, string with whitespaces in both ends will be trimmed before converted to booleans.

e.g. `select cast('\t true' as boolean)` results `true` now, before this pr it's `null`
### How was this patch tested?

add unit tests

Closes #26776 from yaooqinn/SPARK-30147.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-07 15:03:51 +09:00
Aman Omer 51aa7a920e [SPARK-30148][SQL] Optimize writing plans if there is an analysis exception
### What changes were proposed in this pull request?
Optimized QueryExecution.scala#writePlans().

### Why are the changes needed?
If any query fails in Analysis phase and gets AnalysisException, there is no need to execute further phases since those will return a same result i.e, AnalysisException.

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

### How was this patch tested?
Manually

Closes #26778 from amanomer/optExplain.

Authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-07 10:58:02 +09:00
Sean Owen a30ec19a73 [SPARK-30155][SQL] Rename parse() to parseString() to avoid conflict in Scala 2.13
### What changes were proposed in this pull request?

Rename internal method LegacyTypeStringParser.parse() to parseString().

### Why are the changes needed?

In Scala 2.13, the parse() definition clashes with supertype declarations.

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

No

### How was this patch tested?

Existing tests.

Closes #26784 from srowen/SPARK-30155.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-06 16:16:28 -08:00
wuyi 58be82ad4b [SPARK-30098][SQL] Use default datasource as provider for CREATE TABLE syntax
### What changes were proposed in this pull request?

In this PR, we propose to use the value of `spark.sql.source.default` as the provider for `CREATE TABLE` syntax instead of `hive` in Spark 3.0.

And to help the migration, we introduce a legacy conf `spark.sql.legacy.respectHiveDefaultProvider.enabled` and set its default to `false`.

### Why are the changes needed?

1. Currently, `CREATE TABLE` syntax use hive provider to create table while `DataFrameWriter.saveAsTable` API using the value of `spark.sql.source.default` as a provider to create table. It would be better to make them consistent.

2. User may gets confused in some cases. For example:

```
CREATE TABLE t1 (c1 INT) USING PARQUET;
CREATE TABLE t2 (c1 INT);
```

In these two DDLs, use may think that `t2` should also use parquet as default provider since Spark always advertise parquet as the default format. However, it's hive in this case.

On the other hand, if we omit the USING clause in a CTAS statement, we do pick parquet by default if `spark.sql.hive.convertCATS=true`:

```
CREATE TABLE t3 USING PARQUET AS SELECT 1 AS VALUE;
CREATE TABLE t4 AS SELECT 1 AS VALUE;
```
And these two cases together can be really confusing.

3. Now, Spark SQL is very independent and popular. We do not need to be fully consistent with Hive's behavior.

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

Yes, before this PR, using `CREATE TABLE` syntax will use hive provider. But now, it use the value of `spark.sql.source.default` as its provider.

### How was this patch tested?

Added tests in `DDLParserSuite` and `HiveDDlSuite`.

Closes #26736 from Ngone51/dev-create-table-using-parquet-by-default.

Lead-authored-by: wuyi <yi.wu@databricks.com>
Co-authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-07 02:15:25 +08:00
Liang-Chi Hsieh c1a5f94973 [SPARK-30112][SQL] Allow insert overwrite same table if using dynamic partition overwrite
### What changes were proposed in this pull request?

This patch proposes to allow insert overwrite same table if using dynamic partition overwrite.

### Why are the changes needed?

Currently, Insert overwrite cannot overwrite to same table even it is dynamic partition overwrite. But for dynamic partition overwrite, we do not delete partition directories ahead. We write to staging directories and move data to final partition directories. We should be able to insert overwrite to same table under dynamic partition overwrite.

This enables users to read data from a table and insert overwrite to same table by using dynamic partition overwrite. Because this is not allowed for now, users need to write to other temporary location and move it back to the table.

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

Yes. Users can insert overwrite same table if using dynamic partition overwrite.

### How was this patch tested?

Unit test.

Closes #26752 from viirya/dynamic-overwrite-same-table.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-06 09:22:16 -08:00
Sean Owen c8ed71b3cd [SPARK-30011][SQL] Inline 2.12 "AsIfIntegral" classes, not present in 2.13
### What changes were proposed in this pull request?

Classes like DoubleAsIfIntegral are not found in Scala 2.13, but used in the current build. This change 'inlines' the 2.12 implementation and makes it work with both 2.12 and 2.13.

### Why are the changes needed?

To cross-compile with 2.13.

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

It should not as it copies in 2.12's existing behavior.

### How was this patch tested?

Existing tests.

Closes #26769 from srowen/SPARK-30011.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-06 08:15:38 -08:00
gengjiaan 187f3c1773 [SPARK-28083][SQL] Support LIKE ... ESCAPE syntax
## What changes were proposed in this pull request?

The syntax 'LIKE predicate: ESCAPE clause' is a ANSI SQL.
For example:

```
select 'abcSpark_13sd' LIKE '%Spark\\_%';             //true
select 'abcSpark_13sd' LIKE '%Spark/_%';              //false
select 'abcSpark_13sd' LIKE '%Spark"_%';              //false
select 'abcSpark_13sd' LIKE '%Spark/_%' ESCAPE '/';   //true
select 'abcSpark_13sd' LIKE '%Spark"_%' ESCAPE '"';   //true
select 'abcSpark%13sd' LIKE '%Spark\\%%';             //true
select 'abcSpark%13sd' LIKE '%Spark/%%';              //false
select 'abcSpark%13sd' LIKE '%Spark"%%';              //false
select 'abcSpark%13sd' LIKE '%Spark/%%' ESCAPE '/';   //true
select 'abcSpark%13sd' LIKE '%Spark"%%' ESCAPE '"';   //true
select 'abcSpark\\13sd' LIKE '%Spark\\\\_%';          //true
select 'abcSpark/13sd' LIKE '%Spark//_%';             //false
select 'abcSpark"13sd' LIKE '%Spark""_%';             //false
select 'abcSpark/13sd' LIKE '%Spark//_%' ESCAPE '/';  //true
select 'abcSpark"13sd' LIKE '%Spark""_%' ESCAPE '"';  //true
```
But Spark SQL only supports 'LIKE predicate'.

Note: If the input string or pattern string is null, then the result is null too.

There are some mainstream database support the syntax.

**PostgreSQL:**
https://www.postgresql.org/docs/11/functions-matching.html

**Vertica:**
https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Predicates/LIKE-predicate.htm?zoom_highlight=like%20escape

**MySQL:**
https://dev.mysql.com/doc/refman/5.6/en/string-comparison-functions.html

**Oracle:**
https://docs.oracle.com/en/database/oracle/oracle-database/19/jjdbc/JDBC-reference-information.html#GUID-5D371A5B-D7F6-42EB-8C0D-D317F3C53708
https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/Pattern-matching-Conditions.html#GUID-0779657B-06A8-441F-90C5-044B47862A0A

## How was this patch tested?

Exists UT and new UT.

This PR merged to my production environment and runs above sql:
```
spark-sql> select 'abcSpark_13sd' LIKE '%Spark\\_%';
true
Time taken: 0.119 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark_13sd' LIKE '%Spark/_%';
false
Time taken: 0.103 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark_13sd' LIKE '%Spark"_%';
false
Time taken: 0.096 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark_13sd' LIKE '%Spark/_%' ESCAPE '/';
true
Time taken: 0.096 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark_13sd' LIKE '%Spark"_%' ESCAPE '"';
true
Time taken: 0.092 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark\\%%';
true
Time taken: 0.109 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark/%%';
false
Time taken: 0.1 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark"%%';
false
Time taken: 0.081 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark/%%' ESCAPE '/';
true
Time taken: 0.095 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark%13sd' LIKE '%Spark"%%' ESCAPE '"';
true
Time taken: 0.113 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark\\13sd' LIKE '%Spark\\\\_%';
true
Time taken: 0.078 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark/13sd' LIKE '%Spark//_%';
false
Time taken: 0.067 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark"13sd' LIKE '%Spark""_%';
false
Time taken: 0.084 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark/13sd' LIKE '%Spark//_%' ESCAPE '/';
true
Time taken: 0.091 seconds, Fetched 1 row(s)
spark-sql> select 'abcSpark"13sd' LIKE '%Spark""_%' ESCAPE '"';
true
Time taken: 0.091 seconds, Fetched 1 row(s)
```
I create a table and its schema is:
```
spark-sql> desc formatted gja_test;
key     string  NULL
value   string  NULL
other   string  NULL

# Detailed Table Information
Database        test
Table   gja_test
Owner   test
Created Time    Wed Apr 10 11:06:15 CST 2019
Last Access     Thu Jan 01 08:00:00 CST 1970
Created By      Spark 2.4.1-SNAPSHOT
Type    MANAGED
Provider        hive
Table Properties        [transient_lastDdlTime=1563443838]
Statistics      26 bytes
Location        hdfs://namenode.xxx:9000/home/test/hive/warehouse/test.db/gja_test
Serde Library   org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat     org.apache.hadoop.mapred.TextInputFormat
OutputFormat    org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Storage Properties      [field.delim=   , serialization.format= ]
Partition Provider      Catalog
Time taken: 0.642 seconds, Fetched 21 row(s)
```
Table `gja_test` exists three rows of data.
```
spark-sql> select * from gja_test;
a       A       ao
b       B       bo
"__     """__   "
Time taken: 0.665 seconds, Fetched 3 row(s)
```
At finally, I test this function:
```
spark-sql> select * from gja_test where key like value escape '"';
"__     """__   "
Time taken: 0.687 seconds, Fetched 1 row(s)
```

Closes #25001 from beliefer/ansi-sql-like.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2019-12-06 00:07:38 -08:00
Terry Kim b86d4bb931 [SPARK-30001][SQL] ResolveRelations should handle both V1 and V2 tables
### What changes were proposed in this pull request?

This PR makes `Analyzer.ResolveRelations` responsible for looking up both v1 and v2 tables from the session catalog and create an appropriate relation.

### Why are the changes needed?

Currently there are two issues:
1. As described in [SPARK-29966](https://issues.apache.org/jira/browse/SPARK-29966), the logic for resolving relation can load a table twice, which is a perf regression (e.g., Hive metastore can be accessed twice).
2. As described in [SPARK-30001](https://issues.apache.org/jira/browse/SPARK-30001), if a catalog name is specified for v1 tables, the query fails:
```
scala> sql("create table t using csv as select 1 as i")
res2: org.apache.spark.sql.DataFrame = []

scala> sql("select * from t").show
+---+
|  i|
+---+
|  1|
+---+

scala> sql("select * from spark_catalog.t").show
org.apache.spark.sql.AnalysisException: Table or view not found: spark_catalog.t; line 1 pos 14;
'Project [*]
+- 'UnresolvedRelation [spark_catalog, t]
```

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

Yes. Now the catalog name is resolved correctly:
```
scala> sql("create table t using csv as select 1 as i")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("select * from t").show
+---+
|  i|
+---+
|  1|
+---+

scala> sql("select * from spark_catalog.t").show
+---+
|  i|
+---+
|  1|
+---+
```

### How was this patch tested?

Added new tests.

Closes #26684 from imback82/resolve_relation.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-06 15:45:13 +08:00
madianjun a5ccbced8c [SPARK-30067][CORE] Fix fragment offset comparison in getBlockHosts
### What changes were proposed in this pull request?

A bug fixed about the code in getBlockHosts() function. In the case "The fragment ends at a position within this block", the end of fragment should be before the end of block,where the "end of block" means `b.getOffset + b.getLength`,not `b.getLength`.

### Why are the changes needed?

When comparing the fragment end and the block end,we should use fragment's `offset + length`,and then compare to the block's `b.getOffset + b.getLength`, not the block's length.

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

No.

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

Closes #26650 from mdianjun/fix-getBlockHosts.

Authored-by: madianjun <madianjun@jd.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-05 23:39:49 -08:00
Jungtaek Lim (HeartSaVioR) 25431d79f7
[SPARK-29953][SS] Don't clean up source files for FileStreamSource if the files belong to the output of FileStreamSink
### What changes were proposed in this pull request?

This patch prevents the cleanup operation in FileStreamSource if the source files belong to the FileStreamSink. This is needed because the output of FileStreamSink can be read with multiple Spark queries and queries will read the files based on the metadata log, which won't reflect the cleanup.

To simplify the logic, the patch only takes care of the case of when the source path without glob pattern refers to the output directory of FileStreamSink, via checking FileStreamSource to see whether it leverages metadata directory or not to list the source files.

### Why are the changes needed?

Without this patch, if end users turn on cleanup option with the path which is the output of FileStreamSink, there may be out of sync between metadata and available files which may break other queries reading the path.

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

No

### How was this patch tested?

Added UT.

Closes #26590 from HeartSaVioR/SPARK-29953.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-12-05 21:46:28 -08:00
Sean Owen 7782b61a31 [SPARK-29392][CORE][SQL][FOLLOWUP] Avoid deprecated (in 2.13) Symbol syntax 'foo in favor of simpler expression, where it generated deprecation warnings
TL;DR - this is more of the same change in https://github.com/apache/spark/pull/26748

I told you it'd be iterative!

Closes #26765 from srowen/SPARK-29392.3.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-05 13:48:29 -08:00
Kent Yao b9cae37750 [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres
# What changes were proposed in this pull request?
Add an analyzer rule to convert unresolved `Add`, `Subtract`, etc. to `TimeAdd`, `DateAdd`, etc. according to the following policy:
```scala
 /**
   * For [[Add]]:
   * 1. if both side are interval, stays the same;
   * 2. else if one side is interval, turns it to [[TimeAdd]];
   * 3. else if one side is date, turns it to [[DateAdd]] ;
   * 4. else stays the same.
   *
   * For [[Subtract]]:
   * 1. if both side are interval, stays the same;
   * 2. else if the right side is an interval, turns it to [[TimeSub]];
   * 3. else if one side is timestamp, turns it to [[SubtractTimestamps]];
   * 4. else if the right side is date, turns it to [[DateDiff]]/[[SubtractDates]];
   * 5. else if the left side is date, turns it to [[DateSub]];
   * 6. else turns it to stays the same.
   *
   * For [[Multiply]]:
   * 1. If one side is interval, turns it to [[MultiplyInterval]];
   * 2. otherwise, stays the same.
   *
   * For [[Divide]]:
   * 1. If the left side is interval, turns it to [[DivideInterval]];
   * 2. otherwise, stays the same.
   */
```
Besides, we change datetime functions from implicit cast types to strict ones, all available type coercions happen in `DateTimeOperations` coercion rule.
### Why are the changes needed?

Feature Parity between PostgreSQL and Spark, and make the null semantic consistent with Spark.

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

1. date_add/date_sub functions only accept int/tinynit/smallint as the second arg, double/string etc, are forbidden like hive, which produce weird results.

### How was this patch tested?

add ut

Closes #26412 from yaooqinn/SPARK-29774.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-05 22:03:44 +08:00
Kent Yao 332e252a14 [SPARK-29425][SQL] The ownership of a database should be respected
### What changes were proposed in this pull request?

Keep the owner of a database when executing alter database commands

### Why are the changes needed?

Spark will inadvertently delete the owner of a database for executing databases ddls

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

NO

### How was this patch tested?

add and modify uts

Closes #26080 from yaooqinn/SPARK-29425.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-05 16:14:27 +08:00
turbofei 0ab922c1eb [SPARK-29860][SQL] Fix dataType mismatch issue for InSubquery
### What changes were proposed in this pull request?
There is an issue for InSubquery expression.
For example, there are two tables `ta` and `tb` created by the below statements.
```
 sql("create table ta(id Decimal(18,0)) using parquet")
 sql("create table tb(id Decimal(19,0)) using parquet")
```
This statement below would thrown dataType mismatch exception.

```
 sql("select * from ta where id in (select id from tb)").show()
```
However, this similar statement could execute successfully.

```
 sql("select * from ta where id in ((select id from tb))").show()
```
The root cause is that, for `InSubquery` expression, it does not find a common type for two decimalType like `In` expression.
Besides that, for `InSubquery` expression, it also does not find a common type for DecimalType and double/float/bigInt.
In this PR, I fix this issue by finding widerType for `InSubquery` expression when DecimalType is involved.

### Why are the changes needed?
Some InSubquery would throw dataType mismatch exception.

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

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

Closes #26485 from turboFei/SPARK-29860-in-subquery.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-05 16:00:16 +08:00
Aman Omer 0bd8b995d6 [SPARK-30093][SQL] Improve error message for creating view
### What changes were proposed in this pull request?
Improved error message while creating views.

### Why are the changes needed?
Error message should suggest user to use TEMPORARY keyword while creating permanent view referred by temporary view.
https://github.com/apache/spark/pull/26317#discussion_r352377363

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

### How was this patch tested?
Updated test case.

Closes #26731 from amanomer/imp_err_msg.

Authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-05 15:28:07 +08:00
Sean Owen ebd83a544e [SPARK-30009][CORE][SQL][FOLLOWUP] Remove OrderingUtil and Utils.nanSafeCompare{Doubles,Floats} and use java.lang.{Double,Float}.compare directly
### What changes were proposed in this pull request?

Follow up on https://github.com/apache/spark/pull/26654#discussion_r353826162
Instead of OrderingUtil or Utils.nanSafeCompare{Doubles,Floats}, just use java.lang.{Double,Float}.compare directly. All work identically w.r.t. NaN when used to `compare`.

### Why are the changes needed?

Simplification of the previous change, which existed to support Scala 2.13 migration.

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

No.

### How was this patch tested?

Existing tests

Closes #26761 from srowen/SPARK-30009.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-05 11:27:25 +08:00
Sean Owen 2ceed6f32c [SPARK-29392][CORE][SQL][FOLLOWUP] Avoid deprecated (in 2.13) Symbol syntax 'foo in favor of simpler expression, where it generated deprecation warnings
### What changes were proposed in this pull request?

Where it generates a deprecation warning in Scala 2.13, replace Symbol shorthand syntax `'foo` with an equivalent.

### Why are the changes needed?

Symbol syntax `'foo` is deprecated in Scala 2.13. The lines changed below otherwise generate about 440 warnings when building for 2.13.

The previous PR directly replaced many usages with `Symbol("foo")`. But it's also used to specify Columns via implicit conversion (`.select('foo)`) or even where simple Strings are used (`.as('foo)`), as it's kind of an abstraction for interned Strings.

While I find this syntax confusing and would like to deprecate it, here I just replaced it where it generates a build warning (not sure why all occurrences don't): `$"foo"` or just `"foo"`.

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

Should not change behavior.

### How was this patch tested?

Existing tests.

Closes #26748 from srowen/SPARK-29392.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-04 15:03:26 -08:00
07ARB a2102c81ee [SPARK-29453][WEBUI] Improve tooltips information for SQL tab
### What changes were proposed in this pull request?
Adding tooltip to SQL tab for better usability.

### Why are the changes needed?
There are a few common points of confusion in the UI that could be clarified with tooltips. We
 should add tooltips to explain.

### Does this PR introduce any user-facing change?
yes.
![Screenshot 2019-11-23 at 9 47 41 AM](https://user-images.githubusercontent.com/8948111/69472963-aaec5980-0dd6-11ea-881a-fe6266171054.png)

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

Closes #26641 from 07ARB/SPARK-29453.

Authored-by: 07ARB <ankitrajboudh@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-12-04 12:33:43 -06:00
Aman Omer 55132ae9c9 [SPARK-30099][SQL] Improve Analyzed Logical Plan
### What changes were proposed in this pull request?
Avoid duplicate error message in Analyzed Logical plan.

### Why are the changes needed?
Currently, when any query throws `AnalysisException`, same error message will be repeated because of following code segment.
04a5b8f5f8/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala (L157-L166)

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

### How was this patch tested?
Manually. Result of `explain extended select * from wrong;`
BEFORE
> == Parsed Logical Plan ==
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>
> == Analyzed Logical Plan ==
> org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31;
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>
> org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31;
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>
> == Optimized Logical Plan ==
> org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31;
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>
> == Physical Plan ==
> org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31;
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>

AFTER
> == Parsed Logical Plan ==
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>
> == Analyzed Logical Plan ==
> org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31;
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>
> == Optimized Logical Plan ==
> org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31;
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>
> == Physical Plan ==
> org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31;
> 'Project [*]
> +- 'UnresolvedRelation [wrong]
>

Closes #26734 from amanomer/cor_APlan.

Authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-04 13:51:40 +08:00
xiaodeshan 196ea936c3 [SPARK-30106][SQL][TEST] Fix the test of DynamicPartitionPruningSuite
### What changes were proposed in this pull request?
Changed the test **DPP triggers only for certain types of query** in **DynamicPartitionPruningSuite**.

### Why are the changes needed?
The sql has no partition key. The description "no predicate on the dimension table" is not right. So fix it.
```
      Given("no predicate on the dimension table")
      withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") {
        val df = sql(
          """
            |SELECT * FROM fact_sk f
            |JOIN dim_store s
            |ON f.date_id = s.store_id
          """.stripMargin)
```

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

### How was this patch tested?
Updated UT

Closes #26744 from deshanxiao/30106.

Authored-by: xiaodeshan <xiaodeshan@xiaomi.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-03 14:27:48 -08:00
Sean Owen 4193d2f4cc [SPARK-30012][CORE][SQL] Change classes extending scala collection classes to work with 2.13
### What changes were proposed in this pull request?

Move some classes extending Scala collections into parallel source trees, to support 2.13; other minor collection-related modifications.

Modify some classes extending Scala collections to work with 2.13 as well as 2.12. In many cases, this means introducing parallel source trees, as the type hierarchy changed in ways that one class can't support both.

### Why are the changes needed?

To support building for Scala 2.13 in the future.

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

There should be no behavior change.

### How was this patch tested?

Existing tests. Note that the 2.13 changes are not tested by the PR builder, of course. They compile in 2.13 but can't even be tested locally. Later, once the project can be compiled for 2.13, thus tested, it's possible the 2.13 implementations will need updates.

Closes #26728 from srowen/SPARK-30012.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-03 08:59:43 -08:00
John Ayad 8c2849a695 [SPARK-30082][SQL] Do not replace Zeros when replacing NaNs
### What changes were proposed in this pull request?
Do not cast `NaN` to an `Integer`, `Long`, `Short` or `Byte`. This is because casting `NaN` to those types results in a `0` which erroneously replaces `0`s while only `NaN`s should be replaced.

### Why are the changes needed?
This Scala code snippet:
```
import scala.math;

println(Double.NaN.toLong)
```
returns `0` which is problematic as if you run the following Spark code, `0`s get replaced as well:
```
>>> df = spark.createDataFrame([(1.0, 0), (0.0, 3), (float('nan'), 0)], ("index", "value"))
>>> df.show()
+-----+-----+
|index|value|
+-----+-----+
|  1.0|    0|
|  0.0|    3|
|  NaN|    0|
+-----+-----+
>>> df.replace(float('nan'), 2).show()
+-----+-----+
|index|value|
+-----+-----+
|  1.0|    2|
|  0.0|    3|
|  2.0|    2|
+-----+-----+
```

### Does this PR introduce any user-facing change?
Yes, after the PR, running the same above code snippet returns the correct expected results:
```
>>> df = spark.createDataFrame([(1.0, 0), (0.0, 3), (float('nan'), 0)], ("index", "value"))
>>> df.show()
+-----+-----+
|index|value|
+-----+-----+
|  1.0|    0|
|  0.0|    3|
|  NaN|    0|
+-----+-----+

>>> df.replace(float('nan'), 2).show()
+-----+-----+
|index|value|
+-----+-----+
|  1.0|    0|
|  0.0|    3|
|  2.0|    0|
+-----+-----+
```

### How was this patch tested?

Added unit tests to verify replacing `NaN` only affects columns of type `Float` and `Double`

Closes #26738 from johnhany97/SPARK-30082.

Lead-authored-by: John Ayad <johnhany97@gmail.com>
Co-authored-by: John Ayad <jayad@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-04 00:04:55 +08:00
Kent Yao 65552a81d1 [SPARK-30083][SQL] visitArithmeticUnary should wrap PLUS case with UnaryPositive for type checking
### What changes were proposed in this pull request?

`UnaryPositive` only accepts numeric and interval as we defined, but what we do for this in  `AstBuider.visitArithmeticUnary` is just bypassing it.

This should not be omitted for the type checking requirement.

### Why are the changes needed?

bug fix, you can find a pre-discussion here https://github.com/apache/spark/pull/26578#discussion_r347350398

### Does this PR introduce any user-facing change?
yes,  +non-numeric-or-interval is now invalid.
```
-- !query 14
select +date '1900-01-01'
-- !query 14 schema
struct<DATE '1900-01-01':date>
-- !query 14 output
1900-01-01

-- !query 15
select +timestamp '1900-01-01'
-- !query 15 schema
struct<TIMESTAMP '1900-01-01 00:00:00':timestamp>
-- !query 15 output
1900-01-01 00:00:00

-- !query 16
select +map(1, 2)
-- !query 16 schema
struct<map(1, 2):map<int,int>>
-- !query 16 output
{1:2}

-- !query 17
select +array(1,2)
-- !query 17 schema
struct<array(1, 2):array<int>>
-- !query 17 output
[1,2]

-- !query 18
select -'1'
-- !query 18 schema
struct<(- CAST(1 AS DOUBLE)):double>
-- !query 18 output
-1.0

-- !query 19
select -X'1'
-- !query 19 schema
struct<>
-- !query 19 output
org.apache.spark.sql.AnalysisException
cannot resolve '(- X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7

-- !query 20
select +X'1'
-- !query 20 schema
struct<X'01':binary>
-- !query 20 output
```

### How was this patch tested?

add ut check

Closes #26716 from yaooqinn/SPARK-30083.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-03 23:42:21 +08:00
Kent Yao 39291cff95 [SPARK-30048][SQL] Enable aggregates with interval type values for RelationalGroupedDataset
### What changes were proposed in this pull request?

Now the min/max/sum/avg are support for intervals, we should also enable it in RelationalGroupedDataset

### Why are the changes needed?

API consistency improvement

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

yes, Dataset support min/max/sum/avg(mean) on intervals
### How was this patch tested?

add ut

Closes #26681 from yaooqinn/SPARK-30048.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-03 18:40:14 +08:00
herman d7b268ab32 [SPARK-29348][SQL] Add observable Metrics for Streaming queries
### What changes were proposed in this pull request?
Observable metrics are named arbitrary aggregate functions that can be defined on a query (Dataframe). As soon as the execution of a Dataframe reaches a completion point (e.g. finishes batch query or reaches streaming epoch) a named event is emitted that contains the metrics for the data processed since the last completion point.

A user can observe these metrics by attaching a listener to spark session, it depends on the execution mode which listener to attach:
- Batch: `QueryExecutionListener`. This will be called when the query completes. A user can access the metrics by using the `QueryExecution.observedMetrics` map.
- (Micro-batch) Streaming: `StreamingQueryListener`. This will be called when the streaming query completes an epoch. A user can access the metrics by using the `StreamingQueryProgress.observedMetrics` map. Please note that we currently do not support continuous execution streaming.

### Why are the changes needed?
This enabled observable metrics.

### Does this PR introduce any user-facing change?
Yes. It adds the `observe` method to `Dataset`.

### How was this patch tested?
- Added unit tests for the `CollectMetrics` logical node to the `AnalysisSuite`.
- Added unit tests for `StreamingProgress` JSON serialization to the `StreamingQueryStatusAndProgressSuite`.
- Added integration tests for streaming to the `StreamingQueryListenerSuite`.
- Added integration tests for batch to the `DataFrameCallbackSuite`.

Closes #26127 from hvanhovell/SPARK-29348.

Authored-by: herman <herman@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-12-03 11:25:49 +01:00
wuyi 075ae1eeaf [SPARK-29537][SQL] throw exception when user defined a wrong base path
### What changes were proposed in this pull request?

When user defined a base path which is not an ancestor directory for all the input paths,
throw exception immediately.

### Why are the changes needed?

Assuming that we have a DataFrame[c1, c2] be written out in parquet and partitioned by c1.

When using `spark.read.parquet("/path/to/data/c1=1")` to read the data, we'll have a DataFrame with column c2 only.

But if we use `spark.read.option("basePath", "/path/from").parquet("/path/to/data/c1=1")` to
read the data, we'll have a DataFrame with column c1 and c2.

This's happens because a wrong base path does not actually work in `parsePartition()`, so paring would continue until it reaches a directory without "=".

And I think the result of the second read way doesn't make sense.

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

Yes, with this change, user would hit `IllegalArgumentException ` when given a wrong base path while previous behavior doesn't.

### How was this patch tested?

Added UT.

Closes #26195 from Ngone51/dev-wrong-basePath.

Lead-authored-by: wuyi <ngone_5451@163.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-03 17:02:50 +08:00
sychen 332e593093 [SPARK-29943][SQL] Improve error messages for unsupported data type
### What changes were proposed in this pull request?
Improve error messages for unsupported data type.

### Why are the changes needed?
When the spark reads the hive table and encounters an unsupported field type, the exception message has only one unsupported type, and the user cannot know which field of which table.

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

### How was this patch tested?
```create view t AS SELECT STRUCT('a' AS `$a`, 1 AS b) as q;```
current:
org.apache.spark.SparkException: Cannot recognize hive type string: struct<$a:string,b:int>
change:
org.apache.spark.SparkException: Cannot recognize hive type string: struct<$a:string,b:int>, column: q

```select * from t,t_normal_1,t_normal_2```
current:
org.apache.spark.SparkException: Cannot recognize hive type string: struct<$a:string,b:int>
change:
org.apache.spark.SparkException: Cannot recognize hive type string: struct<$a:string,b:int>, column: q, db: default, table: t

Closes #26577 from cxzl25/unsupport_data_type_msg.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-03 10:07:09 +09:00
Ali Afroozeh 68034a8056 [SPARK-30072][SQL] Create dedicated planner for subqueries
### What changes were proposed in this pull request?

This PR changes subquery planning by calling the planner and plan preparation rules on the subquery plan directly. Before we were creating a `QueryExecution` instance for subqueries to get the executedPlan. This would re-run analysis and optimization on the subqueries plan. Running the analysis again on an optimized query plan can have unwanted consequences, as some rules, for example `DecimalPrecision`, are not idempotent.

As an example, consider the expression `1.7 * avg(a)` which after applying the `DecimalPrecision` rule becomes:

```
promote_precision(1.7) * promote_precision(avg(a))
```

After the optimization, more specifically the constant folding rule, this expression becomes:

```
1.7 * promote_precision(avg(a))
```

Now if we run the analyzer on this optimized query again, we will get:

```
promote_precision(1.7) * promote_precision(promote_precision(avg(a)))
```

Which will later optimized as:

```
1.7 * promote_precision(promote_precision(avg(a)))
```

As can be seen, re-running the analysis and optimization on this expression results in an expression with extra nested promote_preceision nodes. Adding unneeded nodes to the plan is problematic because it can eliminate situations where we can reuse the plan.

We opted to introduce dedicated planners for subuqueries, instead of making the DecimalPrecision rule idempotent, because this eliminates this entire category of problems. Another benefit is that planning time for subqueries is reduced.

### How was this patch tested?
Unit tests

Closes #26705 from dbaliafroozeh/CreateDedicatedPlannerForSubqueries.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-12-02 20:56:40 +01:00
Jungtaek Lim (HeartSaVioR) 54edaee586 [MINOR][SS] Add implementation note on overriding serialize/deserialize in HDFSMetadataLog methods' scaladoc
### What changes were proposed in this pull request?

The patch adds scaladoc on `HDFSMetadataLog.serialize` and `HDFSMetadataLog.deserialize` for adding implementation note when overriding - HDFSMetadataLog calls `serialize` and `deserialize` inside try-finally and caller will do the resource (input stream, output stream) cleanup, so resource cleanup should not be performed in these methods, but there's no note on this (only code comment, not scaladoc) which is easy to be missed.

### Why are the changes needed?

Contributors who are unfamiliar with the intention seem to think it as a bug if the resource is not cleaned up in serialize/deserialize of subclass of HDFSMetadataLog, and they couldn't know about the intention without reading the code of HDFSMetadataLog. Adding the note as scaladoc would expand the visibility.

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

No

### How was this patch tested?

Just a doc change.

Closes #26732 from HeartSaVioR/MINOR-SS-HDFSMetadataLog-serde-scaladoc.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: dz <953396112@qq.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-12-02 09:01:45 -06:00
Wenchen Fan e271664a01 [MINOR][SQL] Rename config name to spark.sql.analyzer.failAmbiguousSelfJoin.enabled
### What changes were proposed in this pull request?

add `.enabled` postfix to `spark.sql.analyzer.failAmbiguousSelfJoin`.

### Why are the changes needed?

to follow the existing naming style

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

no

### How was this patch tested?

not needed

Closes #26694 from cloud-fan/conf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 21:05:06 +08:00
Kent Yao 4e073f3c50 [SPARK-30047][SQL] Support interval types in UnsafeRow
### What changes were proposed in this pull request?

Optimize aggregates on interval values from sort-based to hash-based, and we can use the `org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch` for better performance.

### Why are the changes needed?

improve aggerates

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

### How was this patch tested?

add ut and existing ones

Closes #26680 from yaooqinn/SPARK-30047.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 20:47:23 +08:00
LantaoJin 04a5b8f5f8 [SPARK-29839][SQL] Supporting STORED AS in CREATE TABLE LIKE
### What changes were proposed in this pull request?
In SPARK-29421 (#26097) , we can specify a different table provider for `CREATE TABLE LIKE` via `USING provider`.
Hive support `STORED AS` new file format syntax:
```sql
CREATE TABLE tbl(a int) STORED AS TEXTFILE;
CREATE TABLE tbl2 LIKE tbl STORED AS PARQUET;
```
For Hive compatibility, we should also support `STORED AS` in `CREATE TABLE LIKE`.

### Why are the changes needed?
See https://github.com/apache/spark/pull/26097#issue-327424759

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

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

Closes #26466 from LantaoJin/SPARK-29839.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 16:11:58 +08:00
Yuanjian Li 169415ffac [SPARK-30025][CORE] Continuous shuffle block fetching should be disabled by default when the old fetch protocol is used
### What changes were proposed in this pull request?
Disable continuous shuffle block fetching when the old fetch protocol in use.

### Why are the changes needed?
The new feature of continuous shuffle block fetching depends on the latest version of the shuffle fetch protocol. We should keep this constraint in `BlockStoreShuffleReader.fetchContinuousBlocksInBatch`.

### Does this PR introduce any user-facing change?
Users will not get the exception related to continuous shuffle block fetching when old version of the external shuffle service is used.

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

Closes #26663 from xuanyuanking/SPARK-30025.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 15:59:12 +08:00
Liang-Chi Hsieh 85cb388ae3 [SPARK-30050][SQL] analyze table and rename table should not erase hive table bucketing info
### What changes were proposed in this pull request?

This patch adds Hive provider into table metadata in `HiveExternalCatalog.alterTableStats`. When we call `HiveClient.alterTable`, `alterTable` will erase if it can not find hive provider in given table metadata.

Rename table also has this issue.

### Why are the changes needed?

Because running `ANALYZE TABLE` on a Hive table, if the table has bucketing info, will erase existing bucket info.

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

Yes. After this PR, running `ANALYZE TABLE` on Hive table, won't erase existing bucketing info.

### How was this patch tested?

Unit test.

Closes #26685 from viirya/fix-hive-bucket.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 13:40:11 +08:00
HyukjinKwon 51e69feb49 [SPARK-29851][SQL][FOLLOW-UP] Use foreach instead of misusing map
### What changes were proposed in this pull request?

This PR proposes to use foreach instead of misusing map as a small followup of #26476. This could cause some weird errors potentially and it's not a good practice anyway. See also SPARK-16694

### Why are the changes needed?
To avoid potential issues like SPARK-16694

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

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

Closes #26729 from HyukjinKwon/SPARK-29851.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-02 13:40:00 +09:00
Yuanjian Li d1465a1b0d [SPARK-30074][SQL] The maxNumPostShufflePartitions config should obey reducePostShufflePartitions enabled
### What changes were proposed in this pull request?
1. Make maxNumPostShufflePartitions config obey reducePostShfflePartitions config.
2. Update the description for all the SQLConf affected by `spark.sql.adaptive.enabled`.

### Why are the changes needed?
Make the relation between these confs clearer.

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

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

Closes #26664 from xuanyuanking/SPARK-9853-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 12:37:06 +08:00
Terry Kim 5a1896adcb [SPARK-30065][SQL] DataFrameNaFunctions.drop should handle duplicate columns
### What changes were proposed in this pull request?

`DataFrameNaFunctions.drop` doesn't handle duplicate columns even when column names are not specified.

```Scala
val left = Seq(("1", null), ("3", "4")).toDF("col1", "col2")
val right = Seq(("1", "2"), ("3", null)).toDF("col1", "col2")
val df = left.join(right, Seq("col1"))
df.printSchema
df.na.drop("any").show
```
produces
```
root
 |-- col1: string (nullable = true)
 |-- col2: string (nullable = true)
 |-- col2: string (nullable = true)

org.apache.spark.sql.AnalysisException: Reference 'col2' is ambiguous, could be: col2, col2.;
  at org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:240)
```
The reason for the above failure is that columns are resolved by name and if there are multiple columns with the same name, it will fail due to ambiguity.

This PR updates `DataFrameNaFunctions.drop` such that if the columns to drop are not specified, it will resolve ambiguity gracefully by applying `drop` to all the eligible columns. (Note that if the user specifies the columns, it will still continue to fail due to ambiguity).

### Why are the changes needed?

If column names are not specified, `drop` should not fail due to ambiguity since it should still be able to apply `drop` to the eligible columns.

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

Yes, now all the rows with nulls are dropped in the above example:
```
scala> df.na.drop("any").show
+----+----+----+
|col1|col2|col2|
+----+----+----+
+----+----+----+
```

### How was this patch tested?

Added new unit tests.

Closes #26700 from imback82/na_drop.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 12:25:28 +08:00
wuyi 87ebfaf003 [SPARK-29956][SQL] A literal number with an exponent should be parsed to Double
### What changes were proposed in this pull request?

For a literal number with an exponent(e.g. 1e-45, 1E2), we'd parse it to Double by default rather than Decimal. And user could still use  `spark.sql.legacy.exponentLiteralToDecimal.enabled=true` to fall back to previous behavior.

### Why are the changes needed?

According to ANSI standard of SQL, we see that the (part of) definition of `literal` :

```
<approximate numeric literal> ::=
    <mantissa> E <exponent>
```
which indicates that a literal number with an exponent should be approximate numeric(e.g. Double) rather than exact numeric(e.g. Decimal).

And when we test Presto, we found that Presto also conforms to this standard:

```
presto:default> select typeof(1E2);
 _col0
--------
 double
(1 row)
```

```
presto:default> select typeof(1.2);
    _col0
--------------
 decimal(2,1)
(1 row)
```

We also find that, actually, literals like `1E2` are parsed as Double before Spark2.1, but changed to Decimal after #14828 due to *The difference between the two confuses most users* as it said. But we also see support(from DB2 test) of original behavior at #14828 (comment).

Although, we also see that PostgreSQL has its own implementation:

```
postgres=# select pg_typeof(1E2);
 pg_typeof
-----------
 numeric
(1 row)

postgres=# select pg_typeof(1.2);
 pg_typeof
-----------
 numeric
(1 row)
```

We still think that Spark should also conform to this standard while considering SQL standard and Spark own history and majority DBMS and also user experience.

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

Yes.

For `1E2`, before this PR:

```
scala> spark.sql("select 1E2")
res0: org.apache.spark.sql.DataFrame = [1E+2: decimal(1,-2)]
```

After this PR:

```
scala> spark.sql("select 1E2")
res0: org.apache.spark.sql.DataFrame = [100.0: double]
```

And for `1E-45`, before this PR:

```
org.apache.spark.sql.catalyst.parser.ParseException:
decimal can only support precision up to 38
== SQL ==
select 1E-45
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:131)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605)
  ... 47 elided
```

after this PR:

```
scala> spark.sql("select 1E-45");
res1: org.apache.spark.sql.DataFrame = [1.0E-45: double]
```

And before this PR, user may feel super weird to see that `select 1e40` works but `select 1e-40 fails`. And now, both of them work well.

### How was this patch tested?

updated `literals.sql.out` and `ansi/literals.sql.out`

Closes #26595 from Ngone51/SPARK-29956.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 11:34:56 +08:00
Yuming Wang 708ab57f37 [SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column
## What changes were proposed in this pull request?

[HIVE-12063](https://issues.apache.org/jira/browse/HIVE-12063) improved pad decimal numbers with trailing zeros to the scale of the column. The following description is copied from the description of HIVE-12063.

> HIVE-7373 was to address the problems of trimming tailing zeros by Hive, which caused many problems including treating 0.0, 0.00 and so on as 0, which has different precision/scale. Please refer to HIVE-7373 description. However, HIVE-7373 was reverted by HIVE-8745 while the underlying problems remained. HIVE-11835 was resolved recently to address one of the problems, where 0.0, 0.00, and so on cannot be read into decimal(1,1).
 However, HIVE-11835 didn't address the problem of showing as 0 in query result for any decimal values such as 0.0, 0.00, etc. This causes confusion as 0 and 0.0 have different precision/scale than 0.
The proposal here is to pad zeros for query result to the type's scale. This not only removes the confusion described above, but also aligns with many other DBs. Internal decimal number representation doesn't change, however.

**Spark SQL**:
```sql
// bin/spark-sql
spark-sql> select cast(1 as decimal(38, 18));
1
spark-sql>

// bin/beeline
0: jdbc:hive2://localhost:10000/default> select cast(1 as decimal(38, 18));
+----------------------------+--+
| CAST(1 AS DECIMAL(38,18))  |
+----------------------------+--+
| 1.000000000000000000       |
+----------------------------+--+

// bin/spark-shell
scala> spark.sql("select cast(1 as decimal(38, 18))").show(false)
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|1.000000000000000000     |
+-------------------------+

// bin/pyspark
>>> spark.sql("select cast(1 as decimal(38, 18))").show()
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|     1.000000000000000000|
+-------------------------+

// bin/sparkR
> showDF(sql("SELECT cast(1 as decimal(38, 18))"))
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|     1.000000000000000000|
+-------------------------+
```

**PostgreSQL**:
```sql
postgres=# select cast(1 as decimal(38, 18));
       numeric
----------------------
 1.000000000000000000
(1 row)
```
**Presto**:
```sql
presto> select cast(1 as decimal(38, 18));
        _col0
----------------------
 1.000000000000000000
(1 row)
```

## How was this patch tested?

unit tests and manual test:
```sql
spark-sql> select cast(1 as decimal(38, 18));
1.000000000000000000
```
Spark SQL Upgrading Guide:
![image](https://user-images.githubusercontent.com/5399861/69649620-4405c380-10a8-11ea-84b1-6ee675663b98.png)

Closes #26697 from wangyum/SPARK-28461.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-02 09:02:39 +09:00