Commit graph

10760 commits

Author SHA1 Message Date
Gabor Somogyi 0a37a95224 [SPARK-31816][SQL][DOCS] Added high level description about JDBC connection providers for users/developers
### What changes were proposed in this pull request?
JDBC connection provider API and embedded connection providers already added to the code but no in-depth description about the internals. In this PR I've added both user and developer documentation and additionally added an example custom JDBC connection provider.

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

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

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

Closes #31384 from gaborgsomogyi/SPARK-31816.

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

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

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

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

### How was this patch tested?

This was tested via unit tests.

cc: MaxGekk

Closes #31073 from MrPowers/SPARK-33995.

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

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

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

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

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

```

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

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

### How was this patch tested?
Added UT

Closes #31485 from AngersZhuuuu/SPARK-34137.

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

This PR did 2 things :

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

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

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

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

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

### How was this patch tested?
Added UT

Closes #31378 from AngersZhuuuu/SPARK-34240.

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

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

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

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

### Why are the changes needed?

To provide better support with PostgreSQL.

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

Yes. PostgresDialect can handle arrays of types shown above.

### How was this patch tested?

New test.

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

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

### Why are the changes needed?
Keep consistence

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

### How was this patch tested?
Not need

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

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

Delegate table name validation to the session catalog

### Why are the changes needed?

Queerying of tables with nested namespaces.

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

SQL queries of nested namespace queries

### How was this patch tested?

Unit tests updated.

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

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

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

### Why are the changes needed?

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

### How was this patch tested?
No

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

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

We can clean the parameter for Code Simplifications.

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

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

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

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

Closes #31510 from yikf/master.

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

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

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

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

### How was this patch tested?
Jenkins test

Closes #31466 from beliefer/SPARK-34352.

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

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

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

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

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

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

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

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

### Why are the changes needed?
Improve performance

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

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

Closes #31258 from JkSelf/supportDPP1.

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

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

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

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

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

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

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

### How was this patch tested?

Added new tests.

Closes #31500 from imback82/udaf_name.

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

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

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

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

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

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

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

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

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

### How was this patch tested?
existing tests

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

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

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

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

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

To fix the build.

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

No, dev-only.

### How was this patch tested?

CI in this PR should test it out.

Closes #31526 from HyukjinKwon/SPARK-34157.

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

### Why are the changes needed?
Keep consistence

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

### How was this patch tested?
Not need

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

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

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

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

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

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

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

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

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

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

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

Closes #31245 from beliefer/SPARK-34157.

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

Add some info log around commit log.

### Why are the changes needed?

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

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

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

Yes, more info log.

### How was this patch tested?

Not need.

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

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

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

### Why are the changes needed?

Make the plan more readable.

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

No.

### How was this patch tested?

Unit test.

Closes #31455 from wangyum/SPARK-34342.

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

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

### Why are the changes needed?

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

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

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

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

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

Fixes the potential value error.

### How was this patch tested?

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

Closes #31501 from attilapiros/SPARK-34370.

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

UT

Closes #31213 from tanelk/SPARK-34141_extract_generator.

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

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

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

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

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

### Why are the changes needed?

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

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

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

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

```

With this fix:

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

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

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

### How was this patch tested?

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

Closes #31133 from attilapiros/SPARK-26836.

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

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

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

Closes #31413 from c21/bucket-pruning.

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

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

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

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

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

Rule                                  Effective Time / Total Time                     Effective Runs / Total Runs

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

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

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

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

Improve backward compatibility

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

No (the legacy config is false by default)

### How was this patch tested?

a new test

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

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

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

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

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

### Why are the changes needed?

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

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

this pr restores silent user face change

### How was this patch tested?

new tests

Closes #31460 from yaooqinn/SPARK-34346.

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

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Modified UTs

Closes #31464 from HeartSaVioR/SPARK-34339.

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

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

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

### Why are the changes needed?

Revert and improvement to sql.Time handling

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

No

### How was this patch tested?

Unit tests and integration tests

Closes #31473 from saikocat/SPARK-34357.

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

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

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

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

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

### Why are the changes needed?

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

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

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

Yes, adds the hint in the exception message.

### How was this patch tested?

Update existing tests to include the hint.

Closes #31452 from imback82/followup_SPARK-34317.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Updated existing tests / added new tests.

Closes #31422 from imback82/v2_alter_table_set_unset_properties.

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

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

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

### How was this patch tested?
existing testsuites

Closes #31409 from zhengruifeng/limit_with_single_partition.

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

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

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

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

### Why are the changes needed?

To avoid breaking the layout of `Dataset#printSchema`

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

No.

### How was this patch tested?

New test.

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

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

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

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

### Why are the changes needed?

To not fail queries unnecessarily.

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

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

### How was this patch tested?

updated test.

Closes #31443 from cloud-fan/improve.

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

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

Closes #31435 from HeartSaVioR/SPARK-34326.

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Existing unit tests.

Closes #31432 from viirya/SPARK-34324.

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

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

### Why are the changes needed?

To keep it easy to maintain.

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

No.

### How was this patch tested?

`ParserUtilsSuite` passes.

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Unit tests

Closes #31434 from gengliangwang/reviseVersion.

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

New test.

Closes #31426 from sarutak/fix-colRegex.

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Updated existing test cases to include the hint.

Closes #31424 from imback82/better_error.

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

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

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

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

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

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

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

Resolve duplicate attributes for `FlatMapCoGroupsInPandas`.

### Why are the changes needed?

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

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

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

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

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

### How was this patch tested?

Adde unit tests.

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Unit test.

Closes #31421 from gengliangwang/legacyNullStringConstant.

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

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

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

### Why are the changes needed?

To keep the consistent naming in APIs.

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

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

### How was this patch tested?

Unittests were added.

Closes #31408 from HyukjinKwon/SPARK-34306.

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

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

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

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

Closes #31416 from LuciferYang/SPARK-34310.

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

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

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

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

### How was this patch tested?
Added UT

Closes #31377 from AngersZhuuuu/SPARK-34239.

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

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

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

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

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

Closes #31403 from MaxGekk/catalogimpl-recoverPartitions.

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

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

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

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

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

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

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

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

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

Closes #31364 from MaxGekk/doc-refreshTable.

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

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

### Why are the changes needed?

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

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

No, it's a change in unreleased branches.

### How was this patch tested?

Existing tests should cover.

Closes #31410 from HyukjinKwon/SPARK-33245.

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

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

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

### Why are the changes needed?

To clean up unused variables.

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

No

### How was this patch tested?

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

Closes #31400 from imback82/cleanup_resolve_session_catalog.

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

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

### Why are the changes needed?

To increase the test coverage.

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

No

### How was this patch tested?

Added a new test.

Closes #31352 from imback82/grouping_expr.

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

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

### Why are the changes needed?

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

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

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

NO

### How was this patch tested?

This PR just adds unit tests.

Closes #31182 from sunchao/SPARK-34108.

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

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

### Why are the changes needed?

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

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

Yes, this corrects metrics collected for stateful join.

### How was this patch tested?

Unit test.

Closes #31369 from viirya/SPARK-34270.

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

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

### Why are the changes needed?

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

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

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

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

No.

### How was this patch tested?

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

Closes #31393 from wangyum/SPARK-34289.

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

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

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

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

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

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

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

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

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

Closes #31293 from beliefer/SPARK-33601.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Added tests

Closes #31264 from cristichircu/SPARK-34144.

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

existing tests

Closes #31368 from cloud-fan/try.

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

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

### Why are the changes needed?

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

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

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

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

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

### How was this patch tested?

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

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

Closes #31340 from c21/object-hash.

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

Add function exists check before load resource.

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Add test.

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

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

This PR upgrade Parquet to 1.11.1.

Parquet 1.11.1 new features:

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

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

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

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

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

Closes #26804 from wangyum/SPARK-26346.

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

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

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

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

### Why are the changes needed?

To avoid unnecessary sort in query. Save resource.

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

No.

### How was this patch tested?

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

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

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

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

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

Unit tests.

### Additional comments

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

Closes #31254 from MrPowers/SPARK-34165.

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

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

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

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

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

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

Closes #31374 from LuciferYang/SPARK-34275.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

### How was this patch tested?
Added UT

Closes #31341 from AngersZhuuuu/SPARK-34238.

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

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

Closes #31370 from wangyum/SPARK-34268.

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

This is a refactoring and relies on existing tests.

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

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

### Why are the changes needed?
bug fix

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

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

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Test added

Closes #31148 from cristichircu/SPARK-33867.

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

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

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

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

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

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

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

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

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

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

### Why are the changes needed?

To make the behavior more sane.

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

Yes, but it's a followup.

### How was this patch tested?

updated test

Closes #31357 from cloud-fan/bug.

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

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

### Why are the changes needed?

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

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

no

### How was this patch tested?

plan stability check

Closes #31012 from yaooqinn/tpcds.

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

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

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

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

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

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

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

### Why are the changes needed?

fix an obvious NPE bug

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

no

### How was this patch tested?

new tests

Closes #31336 from yaooqinn/SPARK-34233.

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

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

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

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

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

### Why are the changes needed?

bugfix

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

no
### How was this patch tested?

an original test to new place

Closes #31339 from yaooqinn/SPARK-34236.

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

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

### Why are the changes needed?

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

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

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

No.

### How was this patch tested?

Existing tests.

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

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

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

### Why are the changes needed?

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

**BEFORE (Simplified example for correctness)**

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

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

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

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

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

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

Yes. This fixes the correctness issue.

### How was this patch tested?

Pass with the newly added test case.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

### How was this patch tested?
Not need

Closes #31342 from AngersZhuuuu/SPARK-34241.

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

This PR comes with a new test suite.

Closes #31083 from aokolnychyi/spark-34026.

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

yes

### How was this patch tested?

new test

Closes #31325 from cloud-fan/bug.

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

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

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

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

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

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

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

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

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

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

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

### How was this patch tested?
Not need

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

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

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

### Why are the changes needed?

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

no
### How was this patch tested?

new tests

Closes #31320 from yaooqinn/SPARK-34223.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 12:05:58 +08:00