Commit graph

2392 commits

Author SHA1 Message Date
Yuming Wang 789d19cab5 [SPARK-33319][SQL][TEST] Add all built-in SerDes to HiveSerDeReadWriteSuite
### What changes were proposed in this pull request?

This pr add all built-in SerDes to `HiveSerDeReadWriteSuite`.

https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-RowFormats&SerDe

### Why are the changes needed?

We will upgrade Parquet, ORC and Avro, need to ensure compatibility.

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

No.

### How was this patch tested?

N/A

Closes #30228 from wangyum/SPARK-33319.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-11-02 08:34:50 -08:00
xuewei.linxuewei 537a49fc09 [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]
### What changes were proposed in this pull request?

Since Issue [SPARK-33139](https://issues.apache.org/jira/browse/SPARK-33139) has been done, and SQLConf.get and SparkSession.active are more reliable. We are trying to refine the existing code usage of passing SQLConf and SparkSession into sub-class of Rule[QueryPlan].

In this PR.

* remove SQLConf from ctor-parameter of all sub-class of Rule[QueryPlan].
* using SQLConf.get to replace the original SQLConf instance.
* remove SparkSession from ctor-parameter of all sub-class of Rule[QueryPlan].
* using SparkSession.active to replace the original SparkSession instance.

### Why are the changes needed?

Code refine.

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

### How was this patch tested?

Existing test

Closes #30097 from leanken/leanken-SPARK-33140.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-27 12:40:57 +00:00
angerszhu e43cd8ccef [SPARK-32388][SQL] TRANSFORM with schema-less mode should keep the same with hive
### What changes were proposed in this pull request?
In current Spark script transformation with hive serde mode, in case of schema less, result is different with hive.
This pr to keep result same with hive script transform  serde.

#### Hive Scrip Transform with serde in schemaless
```
hive> create table t (c0 int, c1 int, c2 int);
hive> INSERT INTO t VALUES (1, 1, 1);
hive> INSERT INTO t VALUES (2, 2, 2);
hive> CREATE VIEW v AS SELECT TRANSFORM(c0, c1, c2) USING 'cat' FROM t;

hive> DESCRIBE v;
key                 	string
value               	string

hive> SELECT * FROM v;
1	1	1
2	2	2

hive> SELECT key FROM v;
1
2

hive> SELECT value FROM v;
1	1
2	2
```

#### Spark script transform with hive serde in schema less.
```
hive> create table t (c0 int, c1 int, c2 int);
hive> INSERT INTO t VALUES (1, 1, 1);
hive> INSERT INTO t VALUES (2, 2, 2);
hive> CREATE VIEW v AS SELECT TRANSFORM(c0, c1, c2) USING 'cat' FROM t;

hive> SELECT * FROM v;
1   1
2   2
```

**No serde mode in hive (ROW FORMATTED DELIMITED)**
![image](https://user-images.githubusercontent.com/46485123/90088770-55841e00-dd52-11ea-92dd-7fe52d93f0b3.png)

### Why are the changes needed?
Keep same behavior with hive script transform

### Does this PR introduce _any_ user-facing change?
Before this pr with hive serde script transform
```
select transform(*)
USING 'cat'
from (
select 1, 2, 3, 4
) tmp

key     value
1         2
```
After
```
select transform(*)
USING 'cat'
from (
select 1, 2, 3, 4
) tmp

key     value
1         2   3  4
```
### How was this patch tested?
UT

Closes #29421 from AngersZhuuuu/SPARK-32388.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-27 09:25:53 +09:00
HyukjinKwon 369cc614f3 Revert "[SPARK-32388][SQL] TRANSFORM with schema-less mode should keep the same with hive"
This reverts commit 56ab60fb7a.
2020-10-26 11:38:48 +09:00
angerszhu 56ab60fb7a [SPARK-32388][SQL] TRANSFORM with schema-less mode should keep the same with hive
### What changes were proposed in this pull request?
In current Spark script transformation with hive serde mode, in case of schema less, result is different with hive.
This pr to keep result same with hive script transform  serde.

#### Hive Scrip Transform with serde in schemaless
```
hive> create table t (c0 int, c1 int, c2 int);
hive> INSERT INTO t VALUES (1, 1, 1);
hive> INSERT INTO t VALUES (2, 2, 2);
hive> CREATE VIEW v AS SELECT TRANSFORM(c0, c1, c2) USING 'cat' FROM t;

hive> DESCRIBE v;
key                 	string
value               	string

hive> SELECT * FROM v;
1	1	1
2	2	2

hive> SELECT key FROM v;
1
2

hive> SELECT value FROM v;
1	1
2	2
```

#### Spark script transform with hive serde in schema less.
```
hive> create table t (c0 int, c1 int, c2 int);
hive> INSERT INTO t VALUES (1, 1, 1);
hive> INSERT INTO t VALUES (2, 2, 2);
hive> CREATE VIEW v AS SELECT TRANSFORM(c0, c1, c2) USING 'cat' FROM t;

hive> SELECT * FROM v;
1   1
2   2
```

**No serde mode in hive (ROW FORMATTED DELIMITED)**
![image](https://user-images.githubusercontent.com/46485123/90088770-55841e00-dd52-11ea-92dd-7fe52d93f0b3.png)

### Why are the changes needed?
Keep same behavior with hive script transform

### Does this PR introduce _any_ user-facing change?
Before this pr with hive serde script transform
```
select transform(*)
USING 'cat'
from (
select 1, 2, 3, 4
) tmp

key     value
1         2
```
After
```
select transform(*)
USING 'cat'
from (
select 1, 2, 3, 4
) tmp

key     value
1         2   3  4
```
### How was this patch tested?
UT

Closes #29421 from AngersZhuuuu/SPARK-32388.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-26 11:20:29 +09:00
Kent Yao e21bb710e5 [SPARK-32991][SQL] Use conf in shared state as the original configuraion for RESET
### What changes were proposed in this pull request?

####  case

the case here covers the static and dynamic SQL configs behavior in `sharedState` and `sessionState`,  and the specially handled config `spark.sql.warehouse.dir`
the case can be found here - https://github.com/yaooqinn/sugar/blob/master/src/main/scala/com/netease/mammut/spark/training/sql/WarehouseSCBeforeSS.scala

```scala

import java.lang.reflect.Field

import org.apache.spark.sql.SparkSession
import org.apache.spark.{SparkConf, SparkContext}

object WarehouseSCBeforeSS extends App {

  val wh = "spark.sql.warehouse.dir"
  val td = "spark.sql.globalTempDatabase"
  val custom = "spark.sql.custom"

  val conf = new SparkConf()
    .setMaster("local")
    .setAppName("SPARK-32991")
    .set(wh, "./data1")
    .set(td, "bob")

  val sc = new SparkContext(conf)

  val spark = SparkSession.builder()
    .config(wh, "./data2")
    .config(td, "alice")
    .config(custom, "kyao")
    .getOrCreate()

  val confField: Field = spark.sharedState.getClass.getDeclaredField("conf")
  confField.setAccessible(true)
  private val shared: SparkConf = confField.get(spark.sharedState).asInstanceOf[SparkConf]
  println()
  println(s"=====> SharedState: $wh=${shared.get(wh)}")
  println(s"=====> SharedState: $td=${shared.get(td)}")
  println(s"=====> SharedState: $custom=${shared.get(custom, "")}")

  println(s"=====> SessionState: $wh=${spark.conf.get(wh)}")
  println(s"=====> SessionState: $td=${spark.conf.get(td)}")
  println(s"=====> SessionState: $custom=${spark.conf.get(custom, "")}")

  val spark2 = SparkSession.builder().config(td, "fred").getOrCreate()

  println(s"=====> SessionState 2: $wh=${spark2.conf.get(wh)}")
  println(s"=====> SessionState 2: $td=${spark2.conf.get(td)}")
  println(s"=====> SessionState 2: $custom=${spark2.conf.get(custom, "")}")

  SparkSession.setActiveSession(spark)
  spark.sql("RESET")

  println(s"=====> SessionState RESET: $wh=${spark.conf.get(wh)}")
  println(s"=====> SessionState RESET: $td=${spark.conf.get(td)}")
  println(s"=====> SessionState RESET: $custom=${spark.conf.get(custom, "")}")

  val spark3 = SparkSession.builder().getOrCreate()

  println(s"=====> SessionState 3: $wh=${spark2.conf.get(wh)}")
  println(s"=====> SessionState 3: $td=${spark2.conf.get(td)}")
  println(s"=====> SessionState 3: $custom=${spark2.conf.get(custom, "")}")
}
```

#### outputs and analysis
```
// 1. Make the cloned spark conf in shared state respect the warehouse dir from the 1st SparkSession
//=====> SharedState: spark.sql.warehouse.dir=./data1
// 2. 
//=====> SharedState: spark.sql.globalTempDatabase=alice
//=====> SharedState: spark.sql.custom=kyao
//=====> SessionState: spark.sql.warehouse.dir=./data2
//=====> SessionState: spark.sql.globalTempDatabase=alice
//=====> SessionState: spark.sql.custom=kyao
//=====> SessionState 2: spark.sql.warehouse.dir=./data2
//=====> SessionState 2: spark.sql.globalTempDatabase=alice
//=====> SessionState 2: spark.sql.custom=kyao
// 2'.🔼 OK until here
// 3. Make the below 3 ones respect the cloned spark conf in shared state with issue 1 fixed
//=====> SessionState RESET: spark.sql.warehouse.dir=./data1
//=====> SessionState RESET: spark.sql.globalTempDatabase=bob
//=====> SessionState RESET: spark.sql.custom=
// 4. Then the SparkSessions created after RESET will be corrected.
//=====> SessionState 3: spark.sql.warehouse.dir=./data1
//=====> SessionState 3: spark.sql.globalTempDatabase=bob
//=====> SessionState 3: spark.sql.custom=
```

In this PR, we gather all valid config to the cloned conf of `sharedState` during being constructed, well, actually only `spark.sql.warehouse.dir` is missing. Then we use this conf as defaults for `RESET` Command.

`SparkSession.clearActiveSession/clearDefaultSession` will make the shared state invisible and unsharable. They will be internal only soon (confirmed with Wenchen), so cases with them called will not be a problem.

### Why are the changes needed?

bugfix for programming API to call RESET while users creating SparkContext first and config SparkSession later.

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

yes, before this change when you use programming API and call RESET, all configs will be reset to  SparkContext.conf, now they go to SparkSession.sharedState.conf

### How was this patch tested?

new tests

Closes #30045 from yaooqinn/SPARK-32991.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-23 05:52:38 +00:00
Max Gekk a03d77d326 [SPARK-33160][SQL][FOLLOWUP] Replace the parquet metadata key org.apache.spark.int96NoRebase by org.apache.spark.legacyINT96
### What changes were proposed in this pull request?
1. Replace the metadata key `org.apache.spark.int96NoRebase` by `org.apache.spark.legacyINT96`.
2. Change the condition when new key should be saved to parquet metadata: it should be saved when the SQL config `spark.sql.legacy.parquet.int96RebaseModeInWrite` is set to `LEGACY`.
3. Change handling the metadata key in read:
    - If there is no the key in parquet metadata, take the rebase mode from the SQL config: `spark.sql.legacy.parquet.int96RebaseModeInRead`
    - If parquet files were saved by Spark < 3.1.0, use the `LEGACY` rebasing mode for INT96 type.
    - For files written by Spark >= 3.1.0, if the `org.apache.spark.legacyINT96` presents in metadata, perform rebasing otherwise don't.

### Why are the changes needed?
- To not increase parquet size by default when `spark.sql.legacy.parquet.int96RebaseModeInWrite` is `EXCEPTION` after https://github.com/apache/spark/pull/30121.
- To have the implementation similar to `org.apache.spark.legacyDateTime`
- To minimise impact on other subsystems that are based on file sizes like gathering statistics.

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

### How was this patch tested?
Modified test in `ParquetIOSuite`

Closes #30132 from MaxGekk/int96-flip-metadata-rebase-key.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-22 15:57:03 +00:00
angerszhu a1629b4a57 [SPARK-32852][SQL] spark.sql.hive.metastore.jars support HDFS location
### What changes were proposed in this pull request?
Support `spark.sql.hive.metastore.jars` use HDFS location.

When user need to use path to set hive metastore jars, you should set
`spark.sql.hive.metasstore.jars=path` and set real path in `spark.sql.hive.metastore.jars.path`
since we use `File.pathSeperator` to split path, but `FIle.pathSeparator` is `:` in unix, it will split hdfs location `hdfs://nameservice/xx`. So add new config `spark.sql.hive.metastore.jars.path` to set comma separated paths.
To keep both two way supported

### Why are the changes needed?
All spark app can fetch internal version hive jars in HDFS location, not need distribute to all node.

### Does this PR introduce _any_ user-facing change?
User can use HDFS location to store hive metastore jars

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

Closes #29881 from AngersZhuuuu/SPARK-32852.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-22 13:53:01 +00:00
Chao Sun cb3fa6c936 [SPARK-33212][BUILD] Move to shaded clients for Hadoop 3.x profile
### What changes were proposed in this pull request?

This switches Spark to use shaded Hadoop clients, namely hadoop-client-api and hadoop-client-runtime, for Hadoop 3.x. For Hadoop 2.7, we'll still use the same modules such as hadoop-client.

In order to still keep default Hadoop profile to be hadoop-3.2, this defines the following Maven properties:

```
hadoop-client-api.artifact
hadoop-client-runtime.artifact
hadoop-client-minicluster.artifact
```

which default to:
```
hadoop-client-api
hadoop-client-runtime
hadoop-client-minicluster
```
but all switch to `hadoop-client` when the Hadoop profile is hadoop-2.7. A side affect from this is we'll import the same dependency multiple times. For this I have to disable Maven enforcer `banDuplicatePomDependencyVersions`.

Besides above, there are the following changes:
- explicitly add a few dependencies which are imported via transitive dependencies from Hadoop jars, but are removed from the shaded client jars.
- removed the use of `ProxyUriUtils.getPath` from `ApplicationMaster` which is a server-side/private API.
- modified `IsolatedClientLoader` to exclude `hadoop-auth` jars when Hadoop version is 3.x. This change should only matter when we're not sharing Hadoop classes with Spark (which is _mostly_ used in tests).

### Why are the changes needed?

This serves two purposes:
- to unblock Spark from upgrading to Hadoop 3.2.2/3.3.0+. Latest Hadoop versions have upgraded to use Guava 27+ and in order to adopt the latest Hadoop versions in Spark, we'll need to resolve the Guava conflicts. This takes the approach by switching to shaded client jars provided by Hadoop.
- avoid pulling 3rd party dependencies from Hadoop and avoid potential future conflicts.

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

When people use Spark with `hadoop-provided` option, they should make sure class path contains `hadoop-client-api` and `hadoop-client-runtime` jars. In addition, they may need to make sure these jars appear before other Hadoop jars in the order. Otherwise, classes may be loaded from the other non-shaded Hadoop jars and cause potential conflicts.

### How was this patch tested?

Relying on existing tests.

Closes #29843 from sunchao/SPARK-29250.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-10-22 03:21:34 +00:00
Max Gekk ba13b94f6b [SPARK-33210][SQL] Set the rebasing mode for parquet INT96 type to EXCEPTION by default
### What changes were proposed in this pull request?
1. Set the default value for the SQL configs `spark.sql.legacy.parquet.int96RebaseModeInWrite` and `spark.sql.legacy.parquet.int96RebaseModeInRead` to `EXCEPTION`.
2. Update the SQL migration guide.

### Why are the changes needed?
Current default value `LEGACY` may lead to shifting timestamps in read or in write. We should leave the decision about rebasing to users.

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

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

Closes #30121 from MaxGekk/int96-exception-by-default.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-22 03:04:29 +00:00
Nan Zhu 35133901f7 [SPARK-32351][SQL] Show partially pushed down partition filters in explain()
### What changes were proposed in this pull request?

Currently, actual non-dynamic partition pruning is executed in the optimizer phase (PruneFileSourcePartitions) if an input relation has a catalog file index. The current code assumes the same partition filters are generated again in FileSourceStrategy and passed into FileSourceScanExec. FileSourceScanExec uses the partition filters when listing files, but these non-dynamic partition filters do nothing because unnecessary partitions are already pruned in advance, so the filters are mainly used for explain output in this case. If a WHERE clause has DNF-ed predicates, FileSourceStrategy cannot extract the same filters with PruneFileSourcePartitions and then PartitionFilters is not shown in explain output.

This patch proposes to extract partition filters in FileSourceStrategy and HiveStrategy with `extractPredicatesWithinOutputSet` added in https://github.com/apache/spark/pull/29101/files#diff-6be42cfa3c62a7536b1eb1d6447c073c again, then It will show the partially pushed down partition filter in explain().

### Why are the changes needed?

without the patch, the explained plan is inconsistent with what is actually executed

<b>without the change </b> the explained plan of `"SELECT * FROM t WHERE p = '1' OR (p = '2' AND i = 1)"` for datasource and hive tables are like the following respectively (missing pushed down partition filters)

```
== Physical Plan ==
*(1) Filter ((p#21 = 1) OR ((p#21 = 2) AND (i#20 = 1)))
+- *(1) ColumnarToRow
   +- FileScan parquet default.t[i#20,p#21] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/nanzhu/code/spark/sql/hive/target/tmp/hive_execution_test_group/war..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<i:int>
```

```
   == Physical Plan ==
   *(1) Filter ((p#33 = 1) OR ((p#33 = 2) AND (i#32 = 1)))
   +- Scan hive default.t [i#32, p#33], HiveTableRelation [`default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [i#32], Partition Cols: [p#33], Pruned Partitions: [(p=1), (p=2)]]
```

<b> with change </b> the  plan looks like (the actually executed partition filters are exhibited)

```
== Physical Plan ==
*(1) Filter ((p#21 = 1) OR ((p#21 = 2) AND (i#20 = 1)))
+- *(1) ColumnarToRow
   +- FileScan parquet default.t[i#20,p#21] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/nanzhu/code/spark/sql/hive/target/tmp/hive_execution_test_group/war..., PartitionFilters: [((p#21 = 1) OR (p#21 = 2))], PushedFilters: [], ReadSchema: struct<i:int>
```

```
== Physical Plan ==
*(1) Filter ((p#37 = 1) OR ((p#37 = 2) AND (i#36 = 1)))
+- Scan hive default.t [i#36, p#37], HiveTableRelation [`default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [i#36], Partition Cols: [p#37], Pruned Partitions: [(p=1), (p=2)]], [((p#37 = 1) OR (p#37 = 2))]
```

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

no

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

Closes #29831 from CodingCat/SPARK-32351.

Lead-authored-by: Nan Zhu <nanzhu@uber.com>
Co-authored-by: Nan Zhu <CodingCat@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-20 11:13:16 +09:00
angerszhu f8277d3aa3 [SPARK-32069][CORE][SQL] Improve error message on reading unexpected directory
### What changes were proposed in this pull request?
Improve error message on reading unexpected directory

### Why are the changes needed?
Improve error message on reading unexpected directory

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

### How was this patch tested?
Ut

Closes #30027 from AngersZhuuuu/SPARK-32069.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-18 19:02:21 -07:00
xuewei.linxuewei 306872eefa [SPARK-33139][SQL] protect setActionSession and clearActiveSession
### What changes were proposed in this pull request?

This PR is a sub-task of [SPARK-33138](https://issues.apache.org/jira/browse/SPARK-33138). In order to make SQLConf.get reliable and stable, we need to make sure user can't pollute the SQLConf and SparkSession Context via calling setActiveSession and clearActiveSession.

Change of the PR:

* add legacy config spark.sql.legacy.allowModifyActiveSession to fallback to old behavior if user do need to call these two API.
* by default, if user call these two API, it will throw exception
* add extra two internal and private API setActiveSessionInternal and clearActiveSessionInternal for current internal usage
* change all internal reference to new internal API except for SQLContext.setActive and SQLContext.clearActive

### Why are the changes needed?

Make SQLConf.get reliable and stable.

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

### How was this patch tested?

* Add UT in SparkSessionBuilderSuite to test the legacy config
* Existing test

Closes #30042 from leanken/leanken-SPARK-33139.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-16 06:05:17 +00:00
Dongjoon Hyun ec34a001ad [SPARK-33153][SQL][TESTS] Ignore Spark 2.4 in HiveExternalCatalogVersionsSuite on Python 3.8/3.9
### What changes were proposed in this pull request?

This PR aims to ignore Apache Spark 2.4.x distribution in HiveExternalCatalogVersionsSuite if Python version is 3.8 or 3.9.

### Why are the changes needed?

Currently, `HiveExternalCatalogVersionsSuite` is broken on the latest OS like `Ubuntu 20.04` because its default Python version is 3.8. PySpark 2.4.x doesn't work on Python 3.8 due to SPARK-29536.

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

No.

### How was this patch tested?

Manually.
```
$ python3 --version
Python 3.8.5

$ build/sbt "hive/testOnly *.HiveExternalCatalogVersionsSuite"
...
[info] All tests passed.
[info] Passed: Total 1, Failed 0, Errors 0, Passed 1
```

Closes #30044 from dongjoon-hyun/SPARK-33153.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-14 20:48:13 -07:00
xuewei.linxuewei dc697a8b59 [SPARK-13860][SQL] Change statistical aggregate function to return null instead of Double.NaN when divideByZero
### What changes were proposed in this pull request?

As [SPARK-13860](https://issues.apache.org/jira/browse/SPARK-13860) stated, TPCDS Query 39 returns wrong results using SparkSQL. The root cause is that when stddev_samp is applied to a single element set, with TPCDS answer, it return null; as in SparkSQL, it return Double.NaN which caused the wrong result.

Add an extra legacy config to fallback into the NaN logical, and return null by default to align with TPCDS standard.

### Why are the changes needed?

SQL correctness issue.

### Does this PR introduce any user-facing change?
Yes. See sql-migration-guide

In Spark 3.1, statistical aggregation function includes `std`, `stddev`, `stddev_samp`, `variance`, `var_samp`, `skewness`, `kurtosis`, `covar_samp`, `corr` will return `NULL` instead of `Double.NaN` when `DivideByZero` occurs during expression evaluation, for example, when `stddev_samp` applied on a single element set. In Spark version 3.0 and earlier, it will return `Double.NaN` in such case. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.statisticalAggregate` to `true`.

### How was this patch tested?
Updated DataFrameAggregateSuite/DataFrameWindowFunctionsSuite to test both default and legacy behavior.
Adjust DataFrameWindowFunctionsSuite/SQLQueryTestSuite and some R case to update to the default return null behavior.

Closes #29983 from leanken/leanken-SPARK-13860.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-13 13:21:45 +00:00
Yuming Wang 543d59dfbf [SPARK-33107][BUILD][FOLLOW-UP] Remove com.twitter:parquet-hadoop-bundle:1.6.0 and orc.classifier
### What changes were proposed in this pull request?

This pr removes `com.twitter:parquet-hadoop-bundle:1.6.0` and `orc.classifier`.

### Why are the changes needed?

To make code more clear and readable.

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

No.

### How was this patch tested?

Existing test.

Closes #30005 from wangyum/SPARK-33107.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-11 21:54:56 -07:00
Yuming Wang 5e170140b0 [SPARK-33107][SQL] Remove hive-2.3 workaround code
### What changes were proposed in this pull request?

This pr remove `hive-2.3` workaround code.

### Why are the changes needed?

Make code more clear and readable.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #29996 from wangyum/SPARK-33107.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-10 16:41:42 -07:00
HyukjinKwon 2e07ed3041 [SPARK-33082][SPARK-20202][BUILD][SQL][FOLLOW-UP] Remove Hive 1.2 workarounds and Hive 1.2 profile in Jenkins script
### What changes were proposed in this pull request?

This PR removes the leftover of Hive 1.2 workarounds and Hive 1.2 profile in Jenkins script.

- `test-hive1.2` title is not used anymore in Jenkins
- Remove some comments related to Hive 1.2
- Remove unused codes in `OrcFilters.scala`  Hive
- Test `spark.sql.hive.convertMetastoreOrc` disabled case for the tests added at SPARK-19809 and SPARK-22267

### Why are the changes needed?

To remove unused codes & improve test coverage

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

No, dev-only.

### How was this patch tested?

Manually ran the unit tests. Also It will be tested in CI in this PR.

Closes #29973 from HyukjinKwon/SPARK-33082-SPARK-20202.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-09 03:04:26 -07:00
Max Gekk c5f6af9f17 [SPARK-33094][SQL] Make ORC format propagate Hadoop config from DS options to underlying HDFS file system
### What changes were proposed in this pull request?
Propagate ORC options to Hadoop configs in Hive `OrcFileFormat` and in the regular ORC datasource.

### Why are the changes needed?
There is a bug that when running:
```scala
spark.read.format("orc").options(conf).load(path)
```
The underlying file system will not receive the conf options.

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

### How was this patch tested?
Added UT to `OrcSourceSuite`.

Closes #29976 from MaxGekk/orc-option-propagation.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-08 11:59:30 -07:00
Dongjoon Hyun a127387a53 [SPARK-33082][SQL] Remove hive-1.2 workaround code
### What changes were proposed in this pull request?

This PR removes old Hive-1.2 profile related workaround code.

### Why are the changes needed?

To simply the code.
### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CI.

Closes #29961 from dongjoon-hyun/SPARK-HIVE12.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-07 12:27:23 -07:00
Terry Kim 7e99fcd64e [SPARK-33004][SQL] Migrate DESCRIBE column to use UnresolvedTableOrView to resolve the identifier
### What changes were proposed in this pull request?

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

### Why are the changes needed?

The current behavior is not consistent between v1 and v2 commands when resolving a temp view.
In v2, the `t` in the following example is resolved to a table:
```scala
sql("CREATE TABLE testcat.ns.t (id bigint) USING foo")
sql("CREATE TEMPORARY VIEW t AS SELECT 2 as i")
sql("USE testcat.ns")
sql("DESCRIBE t i") // 't' is resolved to testcat.ns.t

Describing columns is not supported for v2 tables.;
org.apache.spark.sql.AnalysisException: Describing columns is not supported for v2 tables.;
```
whereas in v1, the `t` is resolved to a temp view:
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint) USING csv")
sql("CREATE TEMPORARY VIEW t AS SELECT 2 as i")
sql("USE spark_catalog.test")
sql("DESCRIBE t i").show // 't' is resolved to a temp view

+---------+----------+
|info_name|info_value|
+---------+----------+
| col_name|         i|
|data_type|       int|
|  comment|      NULL|
+---------+----------+
```

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

After this PR, `DESCRIBE t i` is resolved to a temp view `t` instead of `testcat.ns.t`.

### How was this patch tested?

Added a new test

Closes #29880 from imback82/describe_column_consistent.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-07 06:33:20 +00:00
Wenchen Fan ec6fccb922 [SPARK-32243][SQL][FOLLOWUP] Fix compilation in HiveSessionCatalog
Fix a mistake when merging https://github.com/apache/spark/pull/29054

Closes #29955 from cloud-fan/hot-fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-06 14:33:34 +00:00
angerszhu ddc7012b3d [SPARK-32243][SQL] HiveSessionCatalog call super.makeFunctionExpression should throw earlier when got Spark UDAF Invalid arguments number error
### What changes were proposed in this pull request?
When we create a UDAF function use class extended `UserDefinedAggregeteFunction`,  when we call the function,  in support hive mode, in HiveSessionCatalog, it will call super.makeFunctionExpression, 

but it will catch error  such as the function need 2 parameter and we only give 1, throw exception only show 
```
No handler for UDF/UDAF/UDTF xxxxxxxx
```
This is confused for develop , we should show error thrown by super method too,

For this pr's UT :
Before change, throw Exception like
```
No handler for UDF/UDAF/UDTF 'org.apache.spark.sql.hive.execution.LongProductSum'; line 1 pos 7
```
After this pr, throw exception
```
Spark UDAF Error: Invalid number of arguments for function longProductSum. Expected: 2; Found: 1;
Hive UDF/UDAF/UDTF Error: No handler for UDF/UDAF/UDTF 'org.apache.spark.sql.hive.execution.LongProductSum'; line 1 pos 7
```

### Why are the changes needed?
Show more detail error message when define UDAF

### Does this PR introduce _any_ user-facing change?
People will see more detail error message when use spark sql's UDAF  in hive support Mode

### How was this patch tested?
Added UT

Closes #29054 from AngersZhuuuu/SPARK-32243.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-06 09:09:19 +00:00
Dongjoon Hyun 008a2ad1f8 [SPARK-20202][BUILD][SQL] Remove references to org.spark-project.hive (Hive 1.2.1)
### What changes were proposed in this pull request?

As of today,
- SPARK-30034 Apache Spark 3.0.0 switched its default Hive execution engine from Hive 1.2 to Hive 2.3. This removes the direct dependency to the forked Hive 1.2.1 in maven repository.
- SPARK-32981 Apache Spark 3.1.0(`master` branch) removed Hive 1.2 related artifacts from Apache Spark binary distributions.

This PR(SPARK-20202) aims to remove the following usage of unofficial Apache Hive fork completely from Apache Spark master for Apache Spark 3.1.0.
```
<hive.group>org.spark-project.hive</hive.group>
<hive.version>1.2.1.spark2</hive.version>
```

For the forked Hive 1.2.1.spark2 users, Apache Spark 2.4(LTS) and 3.0 (~ 2021.12) will provide it.

### Why are the changes needed?

- First, Apache Spark community should not use the unofficial forked release of another Apache project.
- Second, Apache Hive 1.2.1 was released at 2015-06-26 and the forked Hive `1.2.1.spark2` exposed many unfixable bugs in Apache because the forked `1.2.1.spark2` is not maintained at all. Apache Hive 2.3.0 was released at 2017-07-19 and it has been used with less number of bugs compared with `1.2.1.spark2`. Many bugs still exist in `hive-1.2` profile and new Apache Spark unit tests are added with `HiveUtils.isHive23` condition so far.

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

No. This is a dev-only change. PRBuilder will not accept `[test-hive1.2]` on master and `branch-3.1`.

### How was this patch tested?

1. SBT/Hadoop 3.2/Hive 2.3 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129366)
2. SBT/Hadoop 2.7/Hive 2.3 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129382)
3. SBT/Hadoop 3.2/Hive 1.2 (This has not been supported already due to Hive 1.2 doesn't work with Hadoop 3.2.)
4. SBT/Hadoop 2.7/Hive 1.2 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129383, This is rejected)

Closes #29936 from dongjoon-hyun/SPARK-REMOVE-HIVE1.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-05 15:29:56 -07:00
Cheng Su d6f3138352 [SPARK-32859][SQL] Introduce physical rule to decide bucketing dynamically
### What changes were proposed in this pull request?

This PR is to add support to decide bucketed table scan dynamically based on actual query plan. Currently bucketing is enabled by default (`spark.sql.sources.bucketing.enabled`=true), so for all bucketed tables in the query plan, we will use bucket table scan (all input files per the bucket will be read by same task). This has the drawback that if the bucket table scan is not benefitting at all (no join/groupby/etc in the query), we don't need to use bucket table scan as it would restrict the # of tasks to be # of buckets and might hurt parallelism.

The feature is to add a physical plan rule right after `EnsureRequirements`:

The rule goes through plan nodes. For all operators which has "interesting partition" (i.e., require `ClusteredDistribution` or `HashClusteredDistribution`), check if the sub-plan for operator has `Exchange` and bucketed table scan (and only allow certain operators in plan (i.e. `Scan/Filter/Project/Sort/PartialAgg/etc`.), see details in `DisableUnnecessaryBucketedScan.disableBucketWithInterestingPartition`). If yes, disable the bucketed table scan in the sub-plan. In addition, disabling bucketed table scan if there's operator with interesting partition along the sub-plan.

Why the algorithm works is that if there's a shuffle between the bucketed table scan and operator with interesting partition, then bucketed table scan partitioning will be destroyed by the shuffle operator in the middle, and we don't need bucketed table scan for sure.

The idea of "interesting partition" is inspired from "interesting order" in "Access Path Selection in a Relational Database Management System"(http://www.inf.ed.ac.uk/teaching/courses/adbs/AccessPath.pdf), after discussion with cloud-fan .

### Why are the changes needed?

To avoid unnecessary bucketed scan in the query, and this is prerequisite for https://github.com/apache/spark/pull/29625 (decide bucketed sorted scan dynamically will be added later in that PR).

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

A new config `spark.sql.sources.bucketing.autoBucketedScan.enabled` is introduced which set to false by default (the rule is disabled by default as it can regress cached bucketed table query, see discussion in https://github.com/apache/spark/pull/29804#issuecomment-701151447). User can opt-in/opt-out by enabling/disabling the config, as we found in prod, some users rely on assumption of # of tasks == # of buckets when reading bucket table to precisely control # of tasks. This is a bad assumption but it does happen on our side, so leave a config here to allow them opt-out for the feature.

### How was this patch tested?

Added unit tests in `DisableUnnecessaryBucketedScanSuite.scala`

Closes #29804 from c21/bucket-rule.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-02 09:01:15 +09:00
Yuanjian Li 9e6882feca [SPARK-32885][SS] Add DataStreamReader.table API
### What changes were proposed in this pull request?
This pr aims to add a new `table` API in DataStreamReader, which is similar to the table API in DataFrameReader.

### Why are the changes needed?
Users can directly use this API to get a Streaming DataFrame on a table. Below is a simple example:

Application 1 for initializing and starting the streaming job:

```
val path = "/home/yuanjian.li/runtime/to_be_deleted"
val tblName = "my_table"

// Write some data to `my_table`
spark.range(3).write.format("parquet").option("path", path).saveAsTable(tblName)

// Read the table as a streaming source, write result to destination directory
val table = spark.readStream.table(tblName)
table.writeStream.format("parquet").option("checkpointLocation", "/home/yuanjian.li/runtime/to_be_deleted_ck").start("/home/yuanjian.li/runtime/to_be_deleted_2")
```

Application 2 for appending new data:

```
// Append new data into the path
spark.range(5).write.format("parquet").option("path", "/home/yuanjian.li/runtime/to_be_deleted").mode("append").save()
```

Check result:
```
// The desitination directory should contains all written data
spark.read.parquet("/home/yuanjian.li/runtime/to_be_deleted_2").show()
```

### Does this PR introduce _any_ user-facing change?
Yes, a new API added.

### How was this patch tested?
New UT added and integrated testing.

Closes #29756 from xuanyuanking/SPARK-32885.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-25 06:50:24 +00:00
ulysses f2fc966674 [SPARK-32877][SQL][TEST] Add test for Hive UDF complex decimal type
### What changes were proposed in this pull request?

Add test to cover Hive UDF whose input contains complex decimal type.
Add comment to explain why we can't make `HiveSimpleUDF` extend `ImplicitTypeCasts`.

### Why are the changes needed?

For better test coverage with Hive which we compatible or not.

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

No.

### How was this patch tested?

Add test.

Closes #29863 from ulysses-you/SPARK-32877-test.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-24 22:16:05 -07:00
Dongjoon Hyun d7aa3b56e8 [SPARK-32889][SQL][TESTS][FOLLOWUP] Skip special column names test in Hive 1.2
### What changes were proposed in this pull request?

This PR is a followup of SPARK-32889 in order to ignore the special column names test in `hive-1.2` profile.

### Why are the changes needed?

Hive 1.2 is too old to support special column names because it doesn't use Apache ORC. This will recover our `hive-1.2` Jenkins job.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7-hive-1.2/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-1.2/

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

No.

### How was this patch tested?

Pass the test with Hive 1.2 profile.

Closes #29867 from dongjoon-hyun/SPARK-32889-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-24 16:22:08 -07:00
angerszhu c336ddfdb8 [SPARK-32867][SQL] When explain, HiveTableRelation show limited message
### What changes were proposed in this pull request?
In current mode, when explain a SQL plan with HiveTableRelation, it will show so many info about HiveTableRelation's prunedPartition,  this make plan hard to read, this pr make this information simpler.

Before:
![image](https://user-images.githubusercontent.com/46485123/93012078-aeeca080-f5cf-11ea-9286-f5c15eadbee3.png)

For UT
```
 test("Make HiveTableScanExec message simple") {
  withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") {
      withTable("df") {
        spark.range(30)
          .select(col("id"), col("id").as("k"))
          .write
          .partitionBy("k")
          .format("hive")
          .mode("overwrite")
          .saveAsTable("df")

        val df = sql("SELECT df.id, df.k FROM df WHERE df.k < 2")
        df.explain(true)
      }
    }
  }
```

After this pr will show
```
== Parsed Logical Plan ==
'Project ['df.id, 'df.k]
+- 'Filter ('df.k < 2)
   +- 'UnresolvedRelation [df], []

== Analyzed Logical Plan ==
id: bigint, k: bigint
Project [id#11L, k#12L]
+- Filter (k#12L < cast(2 as bigint))
   +- SubqueryAlias spark_catalog.default.df
      +- HiveTableRelation [`default`.`df`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [id#11L], Partition Cols: [k#12L]]

== Optimized Logical Plan ==
Filter (isnotnull(k#12L) AND (k#12L < 2))
+- HiveTableRelation [`default`.`df`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [id#11L], Partition Cols: [k#12L], Pruned Partitions: [(k=0), (k=1)]]

== Physical Plan ==
Scan hive default.df [id#11L, k#12L], HiveTableRelation [`default`.`df`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [id#11L], Partition Cols: [k#12L], Pruned Partitions: [(k=0), (k=1)]], [isnotnull(k#12L), (k#12L < 2)]

```

In my pr, I will construct `HiveTableRelation`'s `simpleString` method to avoid show too much unnecessary info in explain plan. compared to what we had before,I decrease the detail metadata of each partition and only retain the partSpec to show each partition was pruned. Since for detail information, we always don't see this in Plan but to use DESC EXTENDED statement.

### Why are the changes needed?
Make plan about HiveTableRelation more readable

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

### How was this patch tested?
No

Closes #29739 from AngersZhuuuu/HiveTableScan-meta-location-info.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-21 09:15:12 +00:00
William Hyun 7892887981 [SPARK-32930][CORE] Replace deprecated isFile/isDirectory methods
### What changes were proposed in this pull request?

This PR aims to replace deprecated `isFile` and `isDirectory` methods.

```diff
- fs.isDirectory(hadoopPath)
+ fs.getFileStatus(hadoopPath).isDirectory
```

```diff
- fs.isFile(new Path(inProgressLog))
+ fs.getFileStatus(new Path(inProgressLog)).isFile
```

### Why are the changes needed?

It shows deprecation warnings.

- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-3.2-hive-2.3/1244/consoleFull

```
[warn] /home/jenkins/workspace/spark-master-test-sbt-hadoop-3.2-hive-2.3/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala:815: method isFile in class FileSystem is deprecated: see corresponding Javadoc for more information.
[warn]             if (!fs.isFile(new Path(inProgressLog))) {
```

```
[warn] /home/jenkins/workspace/spark-master-test-sbt-hadoop-3.2-hive-2.3/core/src/main/scala/org/apache/spark/SparkContext.scala:1884: method isDirectory in class FileSystem is deprecated: see corresponding Javadoc for more information.
[warn]           if (fs.isDirectory(hadoopPath)) {
```

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

No.

### How was this patch tested?

Pass the Jenkins.

Closes #29796 from williamhyun/filesystem.

Authored-by: William Hyun <williamhyun3@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-18 18:13:11 +09:00
jzc ea3b979e95 [SPARK-32889][SQL] orc table column name supports special characters
### What changes were proposed in this pull request?
make orc table column name support special characters like `$`

### Why are the changes needed?
Special characters like `$` are allowed in orc table column name by Hive.
But it's error when execute command "CREATE TABLE tbl(`$` INT, b INT) using orc" in spark. it's not compatible with Hive.

`Column name "$" contains invalid character(s). Please use alias to rename it.;Column name "$" contains invalid character(s). Please use alias to rename it.;org.apache.spark.sql.AnalysisException: Column name "$" contains invalid character(s). Please use alias to rename it.;
at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$.checkFieldName(OrcFileFormat.scala:51)
at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$.$anonfun$checkFieldNames$1(OrcFileFormat.scala:59)
at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$.$anonfun$checkFieldNames$1$adapted(OrcFileFormat.scala:59)
at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:38) `

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

### How was this patch tested?
Add unit test

Closes #29761 from jzc928/orcColSpecialChar.

Authored-by: jzc <jzc@jzcMacBookPro.local>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-17 14:50:47 -07:00
sychen 92b75dc260 [SPARK-32508][SQL] Disallow empty part col values in partition spec before static partition writing
### What changes were proposed in this pull request?
Write to static partition, check in advance that the partition field is empty.

### Why are the changes needed?
When writing to the current static partition, the partition field is empty, and an error will be reported when all tasks are completed.

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

### How was this patch tested?
add ut

Closes #29316 from cxzl25/SPARK-32508.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-17 06:50:30 +00:00
yangjie01 7fdb571963 [SPARK-32890][SQL] Pass all sql/hive module UTs in Scala 2.13
### What changes were proposed in this pull request?
This pr fix failed cases in sql hive module in Scala 2.13 as follow:

- HiveSchemaInferenceSuite (1 FAILED -> PASS)
- HiveSparkSubmitSuite (1 FAILED-> PASS)
- StatisticsSuite (1 FAILED-> PASS)
- HiveDDLSuite (1 FAILED-> PASS)

After this patch all test passed in sql hive module in Scala 2.13.

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

### How was this patch tested?
- Scala 2.12: Pass the Jenkins or GitHub Action

- Scala 2.13: All tests passed.

Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests -pl sql/hive -am -Pscala-2.13 -Phive
mvn clean test -pl sql/hive -Pscala-2.13 -Phive
```

**Before**

```
Tests: succeeded 3662, failed 4, canceled 0, ignored 601, pending 0
*** 4 TESTS FAILED ***
```

**After**

```
Tests: succeeded 3666, failed 0, canceled 0, ignored 601, pending 0
All tests passed.
```

Closes #29760 from LuciferYang/sql-hive-test.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-16 13:42:04 -05:00
herman c8baab1a1f [SPARK-32879][SQL] Refactor SparkSession initial options
### What changes were proposed in this pull request?
This PR refactors the way we propagate the options from the `SparkSession.Builder` to the` SessionState`. This currently done via a mutable map inside the SparkSession. These setting settings are then applied **after** the Session. This is a bit confusing when you expect something to be set when constructing the `SessionState`. This PR passes the options as a constructor parameter to the `SessionStateBuilder` and this will set the options when the configuration is created.

### Why are the changes needed?
It makes it easier to reason about the configurations set in a SessionState than before. We recently had an incident where someone was using `SparkSessionExtensions` to create a planner rule that relied on a conf to be set. While this is in itself probably incorrect usage, it still illustrated this somewhat funky behavior.

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

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

Closes #29752 from hvanhovell/SPARK-32879.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 06:24:54 +00:00
HyukjinKwon 0696f04672 [SPARK-32876][SQL] Change default fallback versions to 3.0.1 and 2.4.7 in HiveExternalCatalogVersionsSuite
### What changes were proposed in this pull request?

The Jenkins job fails to get the versions. This was fixed by adding temporary fallbacks at https://github.com/apache/spark/pull/28536.
This still doesn't work without the temporary fallbacks. See https://github.com/apache/spark/pull/29694

This PR adds new fallbacks since 2.3 is EOL and Spark 3.0.1 and 2.4.7 are released.

### Why are the changes needed?

To test correctly in Jenkins.

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

No, dev-only

### How was this patch tested?

Jenkins and GitHub Actions builds should test.

Closes #29748 from HyukjinKwon/SPARK-32876.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 13:54:21 -07:00
sandeep.katta 2009f95340 [SPARK-32779][SQL][FOLLOW-UP] Delete Unused code
### What changes were proposed in this pull request?
Follow-up PR as per the review comments in [29649](8d45542e91 (r487140171))

### Why are the changes needed?
Delete the un used code

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

### How was this patch tested?
Existing UT

Closes #29736 from sandeep-katta/deadlockfollowup.

Authored-by: sandeep.katta <sandeep.katta2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 13:22:54 -07:00
Takeshi Yamamuro 4269c2c252 [SPARK-32851][SQL][TEST] Tests should fail if errors happen when generating projection code
### What changes were proposed in this pull request?

This PR intends to set `CODEGEN_ONLY` at `CODEGEN_FACTORY_MODE` in test spark context so that tests can fail if errors happen when generating expr code.

### Why are the changes needed?

I noticed that the code generation of `SafeProjection` failed in the existing test (https://issues.apache.org/jira/browse/SPARK-32828) but it passed because `FALLBACK` was set at `CODEGEN_FACTORY_MODE` (by default) in `SharedSparkSession`. To get aware of these failures quickly, I think its worth setting `CODEGEN_ONLY` at `CODEGEN_FACTORY_MODE`.

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

No.

### How was this patch tested?

Existing tests.

Closes #29721 from maropu/ExprCodegenTest.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-12 08:42:07 +09:00
Wenchen Fan f7995c576a Revert "[SPARK-32677][SQL] Load function resource before create"
This reverts commit 05fcf26b79.
2020-09-09 18:15:22 +00:00
sandeep.katta b0322bf05a [SPARK-32779][SQL] Avoid using synchronized API of SessionCatalog in withClient flow, this leads to DeadLock
### What changes were proposed in this pull request?

No need of using database name in `loadPartition` API of `Shim_v3_0` to get the hive table, in hive there is a overloaded method which gives hive table using table name. By using this API dependency with `SessionCatalog` can be removed in Shim layer

### Why are the changes needed?
To avoid deadlock when communicating with Hive metastore 3.1.x
```
Found one Java-level deadlock:
=============================
"worker3":
  waiting to lock monitor 0x00007faf0be602b8 (object 0x00000007858f85f0, a org.apache.spark.sql.hive.HiveSessionCatalog),
  which is held by "worker0"
"worker0":
  waiting to lock monitor 0x00007faf0be5fc88 (object 0x0000000785c15c80, a org.apache.spark.sql.hive.HiveExternalCatalog),
  which is held by "worker3"

Java stack information for the threads listed above:
===================================================
"worker3":
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.getCurrentDatabase(SessionCatalog.scala:256)
  - waiting to lock <0x00000007858f85f0> (a org.apache.spark.sql.hive.HiveSessionCatalog)
  at org.apache.spark.sql.hive.client.Shim_v3_0.loadPartition(HiveShim.scala:1332)
  at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$loadPartition$1(HiveClientImpl.scala:870)
  at org.apache.spark.sql.hive.client.HiveClientImpl$$Lambda$4459/1387095575.apply$mcV$sp(Unknown Source)
  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
  at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:294)
  at org.apache.spark.sql.hive.client.HiveClientImpl$$Lambda$2227/313239499.apply(Unknown Source)
  at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:227)
  at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:226)
  - locked <0x0000000785ef9d78> (a org.apache.spark.sql.hive.client.IsolatedClientLoader)
  at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:276)
  at org.apache.spark.sql.hive.client.HiveClientImpl.loadPartition(HiveClientImpl.scala:860)
  at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$loadPartition$1(HiveExternalCatalog.scala:911)
  at org.apache.spark.sql.hive.HiveExternalCatalog$$Lambda$4457/2037578495.apply$mcV$sp(Unknown Source)
  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
  at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
  - locked <0x0000000785c15c80> (a org.apache.spark.sql.hive.HiveExternalCatalog)
  at org.apache.spark.sql.hive.HiveExternalCatalog.loadPartition(HiveExternalCatalog.scala:890)
  at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.loadPartition(ExternalCatalogWithListener.scala:179)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.loadPartition(SessionCatalog.scala:512)
  at org.apache.spark.sql.execution.command.LoadDataCommand.run(tables.scala:383)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  - locked <0x00000007b1690ff8> (a org.apache.spark.sql.execution.command.ExecutedCommandExec)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$$Lambda$2084/428667685.apply(Unknown Source)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3616)
  at org.apache.spark.sql.Dataset$$Lambda$2085/559530590.apply(Unknown Source)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:100)
  at org.apache.spark.sql.execution.SQLExecution$$$Lambda$2093/139449177.apply(Unknown Source)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:87)
  at org.apache.spark.sql.execution.SQLExecution$$$Lambda$2086/1088974677.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3614)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.Dataset$$$Lambda$1959/1977822284.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:606)
  at org.apache.spark.sql.SparkSession$$Lambda$1899/424830920.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
  at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anon$1.run(<console>:45)
  at java.lang.Thread.run(Thread.java:748)
"worker0":
  at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
  - waiting to lock <0x0000000785c15c80
  > (a org.apache.spark.sql.hive.HiveExternalCatalog)
  at org.apache.spark.sql.hive.HiveExternalCatalog.tableExists(HiveExternalCatalog.scala:851)
  at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.tableExists(ExternalCatalogWithListener.scala:146)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.tableExists(SessionCatalog.scala:432)
  - locked <0x00000007858f85f0> (a org.apache.spark.sql.hive.HiveSessionCatalog)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.requireTableExists(SessionCatalog.scala:185)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.loadPartition(SessionCatalog.scala:509)
  at org.apache.spark.sql.execution.command.LoadDataCommand.run(tables.scala:383)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  - locked <0x00000007b529af58> (a org.apache.spark.sql.execution.command.ExecutedCommandExec)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$$Lambda$2084/428667685.apply(Unknown Source)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3616)
  at org.apache.spark.sql.Dataset$$Lambda$2085/559530590.apply(Unknown Source)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:100)
  at org.apache.spark.sql.execution.SQLExecution$$$Lambda$2093/139449177.apply(Unknown Source)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:87)
  at org.apache.spark.sql.execution.SQLExecution$$$Lambda$2086/1088974677.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3614)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.Dataset$$$Lambda$1959/1977822284.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:606)
  at org.apache.spark.sql.SparkSession$$Lambda$1899/424830920.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
  at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anon$1.run(<console>:45)
  at java.lang.Thread.run(Thread.java:748)

Found 1 deadlock.
```

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

### How was this patch tested?
Tested using below script by executing in spark-shell and I found no dead lock

launch spark-shell using ./bin/spark-shell --conf "spark.sql.hive.metastore.jars=maven" --conf spark.sql.hive.metastore.version=3.1 --conf spark.hadoop.datanucleus.schema.autoCreateAll=true

**code**
```
def testHiveDeadLock = {
      import scala.collection.mutable.ArrayBuffer
      import scala.util.Random
      println("test hive DeadLock")
      spark.sql("drop database if exists testDeadLock cascade")
      spark.sql("create database testDeadLock")
      spark.sql("use testDeadLock")
      val tableCount = 100
      val tableNamePrefix = "testdeadlock"
      for (i <- 0 until tableCount) {
        val tableName = s"$tableNamePrefix${i + 1}"
        spark.sql(s"drop table if exists $tableName")
        spark.sql(s"create table $tableName (a bigint) partitioned by (b bigint) stored as orc")
      }

      val threads = new ArrayBuffer[Thread]
      for (i <- 0 until tableCount) {
        threads.append(new Thread( new Runnable {
          override def run: Unit = {
            val tableName = s"$tableNamePrefix${i + 1}"
            val rand = Random
            val df = spark.range(0, 20000).toDF("a")
            val location = s"/tmp/${rand.nextLong.abs}"
            df.write.mode("overwrite").orc(location)
            spark.sql(
              s"""
        LOAD DATA LOCAL INPATH '$location' INTO TABLE $tableName partition (b=$i)""")
          }
        }, s"worker$i"))
        threads(i).start()
      }

      for (i <- 0 until tableCount) {
        println(s"Joining with thread $i")
        threads(i).join()
      }
      for (i <- 0 until tableCount) {
        val tableName = s"$tableNamePrefix${i + 1}"
        spark.sql(s"select count(*) from $tableName").show(false)
      }
      println("All done")
    }

    for(i <- 0 until 100) {
      testHiveDeadLock
      println(s"completed {$i}th iteration")
    }
  }
```

Closes #29649 from sandeep-katta/metastore3.1DeadLock.

Authored-by: sandeep.katta <sandeep.katta2007@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-07 15:10:33 +09:00
ulysses 05fcf26b79 [SPARK-32677][SQL] Load function resource before create
### What changes were proposed in this pull request?

Change `CreateFunctionCommand` code that add class check before create function.

### Why are the changes needed?

We have different behavior between create permanent function and temporary function when function class is invaild. e.g.,
```
create function f as 'test.non.exists.udf';
-- Time taken: 0.104 seconds

create temporary function f as 'test.non.exists.udf'
-- Error in query: Can not load class 'test.non.exists.udf' when registering the function 'f', please make sure it is on the classpath;
```

And Hive also fails both of them.

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

Yes, user will get exception when create a invalid udf.

### How was this patch tested?

New test.

Closes #29502 from ulysses-you/function.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-07 06:00:23 +00:00
angerszhu 55ce49ed28 [SPARK-32400][SQL][TEST][FOLLOWUP][TEST-MAVEN] Fix resource loading error in HiveScripTransformationSuite
### What changes were proposed in this pull request?
#29401 move `test_script.py` from sql/hive module to sql/core module, cause HiveScripTransformationSuite load resource issue.

### Why are the changes needed?
This issue cause jenkins test failed in mvn

spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/
spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11/
spark-master-test-maven-hadoop-3.2-hive-2.3:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3/
![image](https://user-images.githubusercontent.com/46485123/91681585-71285a80-eb81-11ea-8519-99fc9783d6b9.png)

![image](https://user-images.githubusercontent.com/46485123/91681010-aaf86180-eb7f-11ea-8dbb-61365a3b0ab4.png)

Error as below:
```
 Exception thrown while executing Spark plan:
 HiveScriptTransformation [a#349299, b#349300, c#349301, d#349302, e#349303], python /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/hive/file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test_script.py, [a#349309, b#349310, c#349311, d#349312, e#349313], ScriptTransformationIOSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim, )),List((field.delim, )),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
+- Project [_1#349288 AS a#349299, _2#349289 AS b#349300, _3#349290 AS c#349301, _4#349291 AS d#349302, _5#349292 AS e#349303]
   +- LocalTableScan [_1#349288, _2#349289, _3#349290, _4#349291, _5#349292]

 == Exception ==
 org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 18021.0 failed 1 times, most recent failure: Lost task 0.0 in stage 18021.0 (TID 37324) (192.168.10.31 executor driver): org.apache.spark.SparkException: Subprocess exited with status 2. Error: python: can't open file '/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/hive/file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test_script.py': [Errno 2] No such file or directory

 at org.apache.spark.sql.execution.BaseScriptTransformationExec.checkFailureAndPropagate(BaseScriptTransformationExec.scala:180)
 at org.apache.spark.sql.execution.BaseScriptTransformationExec.checkFailureAndPropagate$(BaseScriptTransformationExec.scala:157)
 at org.apache.spark.sql.hive.execution.HiveScriptTransformationExec.checkFailureAndPropagate(HiveScriptTransformationExec.scala:49)
 at org.apache.spark.sql.hive.execution.HiveScriptTransformationExec$$anon$1.hasNext(HiveScriptTransformationExec.scala:110)
 at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
 at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
 at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
 at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
 at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
 at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
 at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
 at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
 at org.apache.spark.scheduler.Task.run(Task.scala:127)
 at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:480)
 at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1426)
 at o
```
### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
Existed UT

Closes #29588 from AngersZhuuuu/SPARK-32400-FOLLOWUP.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-02 18:27:29 +09:00
Huaxin Gao 806140de40 [SPARK-32592][SQL] Make DataFrameReader.table take the specified options
### What changes were proposed in this pull request?
pass specified options in DataFrameReader.table to JDBCTableCatalog.loadTable

### Why are the changes needed?
Currently, `DataFrameReader.table` ignores the specified options. The options specified like the following are lost.
```
    val df = spark.read
      .option("partitionColumn", "id")
      .option("lowerBound", "0")
      .option("upperBound", "3")
      .option("numPartitions", "2")
      .table("h2.test.people")
```
We need to make `DataFrameReader.table` take the specified options.

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

### How was this patch tested?
Manually test for now. Will add a test after V2 JDBC read is implemented.

Closes #29535 from huaxingao/table_options.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-31 13:21:15 +00:00
ulysses 339eec5f32 [SPARK-20680][SQL][FOLLOW-UP] Add HiveVoidType in HiveClientImpl
### What changes were proposed in this pull request?

Discussion with [comment](https://github.com/apache/spark/pull/29244#issuecomment-671746329).

Add `HiveVoidType` class in `HiveClientImpl` then we can replace `NullType` to `HiveVoidType` before we call hive client.

### Why are the changes needed?

Better compatible with hive.

More details in [#29244](https://github.com/apache/spark/pull/29244).

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

Yes, user can create view with null type in Hive.

### How was this patch tested?

New test.

Closes #29423 from ulysses-you/add-HiveVoidType.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-14 06:59:15 +00:00
Max Gekk f664aaaab1 [SPARK-32599][SQL][TESTS] Check the TEXTFILE file format in HiveSerDeReadWriteSuite
### What changes were proposed in this pull request?
- Test TEXTFILE together with the PARQUET and ORC file formats in `HiveSerDeReadWriteSuite`
- Remove the "SPARK-32594: insert dates to a Hive table" added by #29409

### Why are the changes needed?
- To improve test coverage, and test other row SerDe - `org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe`.
- The removed test is not needed anymore because the bug reported in SPARK-32594 is triggered by the TEXTFILE file format too.

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

### How was this patch tested?
By running the modified test suite `HiveSerDeReadWriteSuite`.

Closes #29417 from MaxGekk/textfile-HiveSerDeReadWriteSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-12 20:59:59 +09:00
angerszhu 4cf8c1d07d [SPARK-32400][SQL] Improve test coverage of HiveScriptTransformationExec
### What changes were proposed in this pull request?

1. Extract common test case (no serde) to BasicScriptTransformationExecSuite
2. Add more test case for no serde mode about supported data type and behavior in `BasicScriptTransformationExecSuite`
3. Add more test case for hive serde mode about supported type and behavior in `HiveScriptTransformationExecSuite`

### Why are the changes needed?
Improve test coverage of Script Transformation

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

### How was this patch tested?
Added UT

Closes #29401 from AngersZhuuuu/SPARK-32400.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-12 06:02:42 +00:00
Max Gekk 0477d23467 [SPARK-32594][SQL] Fix serialization of dates inserted to Hive tables
### What changes were proposed in this pull request?
Fix `DaysWritable` by overriding parent's method `def get(doesTimeMatter: Boolean): Date` from `DateWritable` instead of `Date get()` because the former one uses the first one. The bug occurs because `HiveOutputWriter.write()` call `def get(doesTimeMatter: Boolean): Date` transitively with default implementation from the parent class  `DateWritable` which doesn't respect date rebases and uses not initialized `daysSinceEpoch` (0 which `1970-01-01`).

### Why are the changes needed?
The changes fix the bug:
```sql
spark-sql> CREATE TABLE table1 (d date);
spark-sql> INSERT INTO table1 VALUES (date '2020-08-11');
spark-sql> SELECT * FROM table1;
1970-01-01
```
The expected result of the last SQL statement must be **2020-08-11** but got **1970-01-01**.

### Does this PR introduce _any_ user-facing change?
Yes. After the fix, `INSERT` work correctly:
```sql
spark-sql> SELECT * FROM table1;
2020-08-11
```

### How was this patch tested?
Add new test to `HiveSerDeReadWriteSuite`

Closes #29409 from MaxGekk/insert-date-into-hive-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-12 13:32:16 +09:00
angerszhu d251443a02 [SPARK-32403][SQL] Refactor current ScriptTransformationExec
# What changes were proposed in this pull request?

This PR comes from the comment: #29085 (comment)

- Extract common Script IOSchema `ScriptTransformationIOSchema`
- avoid repeated judgement extract process output row method `createOutputIteratorWithoutSerde` && `createOutputIteratorWithSerde`
- add default no serde IO schemas `ScriptTransformationIOSchema.defaultIOSchema`

### Why are the changes needed?
Refactor code

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

### How was this patch tested?
NO

Closes #29199 from AngersZhuuuu/spark-32105-followup.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-10 16:37:31 +00:00
GuoPhilipse aa4d3c19fe [SPARK-32560][SQL] Improve exception message at InsertIntoHiveTable.processInsert
### What changes were proposed in this pull request?
improve exception message

### Why are the changes needed?
the before message lack of single quotes, we may improve it to keep consisent.
![image](https://user-images.githubusercontent.com/46367746/89595808-15bbc300-d888-11ea-9914-b05ea7b66461.png)

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

### How was this patch tested?
No ,it is only improving the message.

Closes #29376 from GuoPhilipse/improve-exception-message.

Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Co-authored-by: GuoPhilipse <guofei_ok@126.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-07 14:29:32 +09:00
Max Gekk 6664e282f6 [SPARK-32546][SQL][FOLLOWUP] Add .toSeq to tableNames in HiveClientImpl.listTablesByType
### What changes were proposed in this pull request?
Explicitly convert `tableNames` to `Seq` in `HiveClientImpl.listTablesByType` as it was done by c28a6fa511 (diff-6fd847124f8eae45ba2de1cf7d6296feR769)

### Why are the changes needed?
See this PR https://github.com/apache/spark/pull/29111, to compile by Scala 2.13. The changes were discarded by https://github.com/apache/spark/pull/29363 accidentally.

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

### How was this patch tested?
Compiling by Scala 2.13

Closes #29379 from MaxGekk/fix-listTablesByType-for-views-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-06 14:23:56 +00:00
Max Gekk dc96f2f8d6 [SPARK-32546][SQL] Get table names directly from Hive tables
### What changes were proposed in this pull request?
Get table names directly from a sequence of Hive tables in `HiveClientImpl.listTablesByType()` by skipping conversions Hive tables to Catalog tables.

### Why are the changes needed?
A Hive metastore can be shared across many clients. A client can create tables using a SerDe which is not available on other clients, for instance `ROW FORMAT SERDE "com.ibm.spss.hive.serde2.xml.XmlSerDe"`. In the current implementation, other clients get the following exception while getting views:
```
java.lang.RuntimeException: MetaException(message:java.lang.ClassNotFoundException Class com.ibm.spss.hive.serde2.xml.XmlSerDe not found)
```
when `com.ibm.spss.hive.serde2.xml.XmlSerDe` is not available.

### Does this PR introduce _any_ user-facing change?
Yes. For example, `SHOW VIEWS` returns a list of views instead of throwing an exception.

### How was this patch tested?
- By existing test suites like:
```
$ build/sbt -Phive-2.3 "test:testOnly org.apache.spark.sql.hive.client.VersionsSuite"
```
- And manually:

1. Build Spark with Hive 1.2: `./build/sbt package -Phive-1.2 -Phive -Dhadoop.version=2.8.5`

2. Run spark-shell with a custom Hive SerDe, for instance download [json-serde-1.3.8-jar-with-dependencies.jar](https://github.com/cdamak/Twitter-Hive/blob/master/json-serde-1.3.8-jar-with-dependencies.jar) from https://github.com/cdamak/Twitter-Hive:
```
$ ./bin/spark-shell --jars ../Downloads/json-serde-1.3.8-jar-with-dependencies.jar
```

3. Create a Hive table using this SerDe:
```scala
scala> :paste
// Entering paste mode (ctrl-D to finish)

sql(s"""
  |CREATE TABLE json_table2(page_id INT NOT NULL)
  |ROW FORMAT SERDE 'org.openx.data.jsonserde.JsonSerDe'
  |""".stripMargin)

// Exiting paste mode, now interpreting.
res0: org.apache.spark.sql.DataFrame = []

scala> sql("SHOW TABLES").show
+--------+-----------+-----------+
|database|  tableName|isTemporary|
+--------+-----------+-----------+
| default|json_table2|      false|
+--------+-----------+-----------+

scala> sql("SHOW VIEWS").show
+---------+--------+-----------+
|namespace|viewName|isTemporary|
+---------+--------+-----------+
+---------+--------+-----------+
```

4. Quit from the current `spark-shell` and run it without jars:
```
$ ./bin/spark-shell
```

5. Show views. Without the fix, it throws the exception:
```scala
scala> sql("SHOW VIEWS").show
20/08/06 10:53:36 ERROR log: error in initSerDe: java.lang.ClassNotFoundException Class org.openx.data.jsonserde.JsonSerDe not found
java.lang.ClassNotFoundException: Class org.openx.data.jsonserde.JsonSerDe not found
	at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2273)
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:385)
	at org.apache.hadoop.hive.ql.metadata.Table.getDeserializerFromMetaStore(Table.java:276)
	at org.apache.hadoop.hive.ql.metadata.Table.getDeserializer(Table.java:258)
	at org.apache.hadoop.hive.ql.metadata.Table.getCols(Table.java:605)
```

After the fix:
```scala
scala> sql("SHOW VIEWS").show
+---------+--------+-----------+
|namespace|viewName|isTemporary|
+---------+--------+-----------+
+---------+--------+-----------+
```

Closes #29363 from MaxGekk/fix-listTablesByType-for-views.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-06 08:35:59 +00:00