Commit graph

28742 commits

Author SHA1 Message Date
Kousuke Saruta c50fcac00e [SPARK-33607][SS][WEBUI] Input Rate timeline/histogram aren't rendered if built with Scala 2.13
### What changes were proposed in this pull request?

This PR fixes an issue that the histogram and timeline aren't rendered in the `Streaming Query Statistics` page if we built Spark with Scala 2.13.

![before-fix-the-issue](https://user-images.githubusercontent.com/4736016/100612855-f543d700-3356-11eb-90d9-ede57b8b3f4f.png)
![NaN_Error](https://user-images.githubusercontent.com/4736016/100612879-00970280-3357-11eb-97cf-43978bbe2d3a.png)

The reason is [`maxRecordRate` can be `NaN`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala#L371) for Scala 2.13.

The `NaN` is the result of [`query.recentProgress.map(_.inputRowsPerSecond).max`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala#L372) when the first element of `query.recentProgress.map(_.inputRowsPerSecond)` is `NaN`.
Actually, the comparison logic for `Double` type was changed in Scala 2.13.
https://github.com/scala/bug/issues/12107
https://github.com/scala/scala/pull/6410

So this issue happens as of Scala 2.13.

The root cause of the `NaN` is [here](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala#L164).
This `NaN` seems to be an initial value of `inputTimeSec` so I think `Double.PositiveInfinity` is suitable rather than `NaN` and this change can resolve this issue.

### Why are the changes needed?

To make sure we can use the histogram/timeline with Scala 2.13.

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

No.

### How was this patch tested?

First, I built with the following commands.
```
$ /dev/change-scala-version.sh 2.13
$ build/sbt -Phive -Phive-thriftserver -Pscala-2.13 package
```

Then, ran the following query (this is brought from #30427 ).
```
import org.apache.spark.sql.streaming.Trigger
val query = spark
  .readStream
  .format("rate")
  .option("rowsPerSecond", 1000)
  .option("rampUpTime", "10s")
  .load()
  .selectExpr("*", "CAST(CAST(timestamp AS BIGINT) - CAST((RAND() * 100000) AS BIGINT) AS TIMESTAMP) AS tsMod")
  .selectExpr("tsMod", "mod(value, 100) as mod", "value")
  .withWatermark("tsMod", "10 seconds")
  .groupBy(window($"tsMod", "1 minute", "10 seconds"), $"mod")
  .agg(max("value").as("max_value"), min("value").as("min_value"), avg("value").as("avg_value"))
  .writeStream
  .format("console")
  .trigger(Trigger.ProcessingTime("5 seconds"))
  .outputMode("append")
  .start()
```

Finally, I confirmed that the timeline and histogram are rendered.
![after-fix-the-issue](https://user-images.githubusercontent.com/4736016/100612736-c9285600-3356-11eb-856d-7e53cc656c36.png)

```

Closes #30546 from sarutak/ss-nan.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-01 11:45:32 +09:00
Weichen Xu 80161238fe [SPARK-33592] Fix: Pyspark ML Validator params in estimatorParamMaps may be lost after saving and reloading
### What changes were proposed in this pull request?
Fix: Pyspark ML Validator params in estimatorParamMaps may be lost after saving and reloading

When saving validator estimatorParamMaps, will check all nested stages in tuned estimator to get correct param parent.

Two typical cases to manually test:
~~~python
tokenizer = Tokenizer(inputCol="text", outputCol="words")
hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features")
lr = LogisticRegression()
pipeline = Pipeline(stages=[tokenizer, hashingTF, lr])

paramGrid = ParamGridBuilder() \
    .addGrid(hashingTF.numFeatures, [10, 100]) \
    .addGrid(lr.maxIter, [100, 200]) \
    .build()
tvs = TrainValidationSplit(estimator=pipeline,
                           estimatorParamMaps=paramGrid,
                           evaluator=MulticlassClassificationEvaluator())

tvs.save(tvsPath)
loadedTvs = TrainValidationSplit.load(tvsPath)

# check `loadedTvs.getEstimatorParamMaps()` restored correctly.
~~~

~~~python
lr = LogisticRegression()
ova = OneVsRest(classifier=lr)
grid = ParamGridBuilder().addGrid(lr.maxIter, [100, 200]).build()
evaluator = MulticlassClassificationEvaluator()
tvs = TrainValidationSplit(estimator=ova, estimatorParamMaps=grid, evaluator=evaluator)

tvs.save(tvsPath)
loadedTvs = TrainValidationSplit.load(tvsPath)

# check `loadedTvs.getEstimatorParamMaps()` restored correctly.
~~~

### Why are the changes needed?
Bug fix.

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

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

Closes #30539 from WeichenXu123/fix_tuning_param_maps_io.

Authored-by: Weichen Xu <weichen.xu@databricks.com>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2020-12-01 09:36:42 +08:00
Bryan Cutler aeb3649fb9 [SPARK-33613][PYTHON][TESTS] Replace deprecated APIs in pyspark tests
### What changes were proposed in this pull request?

This replaces deprecated API usage in PySpark tests with the preferred APIs. These have been deprecated for some time and usage is not consistent within tests.

- https://docs.python.org/3/library/unittest.html#deprecated-aliases

### Why are the changes needed?

For consistency and eventual removal of deprecated APIs.

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

No

### How was this patch tested?

Existing tests

Closes #30557 from BryanCutler/replace-deprecated-apis-in-tests.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-01 10:34:40 +09:00
Weichen Xu 596fbc1d29 [SPARK-33556][ML] Add array_to_vector function for dataframe column
### What changes were proposed in this pull request?

Add array_to_vector function for dataframe column

### Why are the changes needed?
Utility function for array to vector conversion.

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

### How was this patch tested?
scala unit test & doctest.

Closes #30498 from WeichenXu123/array_to_vec.

Lead-authored-by: Weichen Xu <weichen.xu@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-01 09:52:19 +09:00
Jungtaek Lim (HeartSaVioR) f5d2165c95 [SPARK-33440][CORE] Use current timestamp with warning log in HadoopFSDelegationTokenProvider when the issue date for token is not set up properly
### What changes were proposed in this pull request?

This PR proposes to use current timestamp with warning log when the issue date for token is not set up properly. The next section will explain the rationalization with details.

### Why are the changes needed?

Unfortunately not every implementations respect the `issue date` in `AbstractDelegationTokenIdentifier`, which Spark relies on while calculating. The default value of issue date is 0L, which is far from actual issue date, breaking logic on calculating next renewal date under some circumstance, leading to 0 interval (immediate) on rescheduling token renewal.

In HadoopFSDelegationTokenProvider, Spark calculates token renewal interval as below:

2c64b731ae/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala (L123-L134)

The interval is calculated as `token.renew() - identifier.getIssueDate`, which is providing correct interval assuming both `token.renew()` and `identifier.getIssueDate` produce correct value, but it's going to be weird when `identifier.getIssueDate` provides 0L (default value), like below:

```
20/10/13 06:34:19 INFO security.HadoopFSDelegationTokenProvider: Renewal interval is 1603175657000 for token S3ADelegationToken/IDBroker
20/10/13 06:34:19 INFO security.HadoopFSDelegationTokenProvider: Renewal interval is 86400048 for token HDFS_DELEGATION_TOKEN
```

Hopefully we pick the minimum value as safety guard (so in this case, `86400048` is being picked up), but the safety guard leads unintentional bad impact on this case.

2c64b731ae/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala (L58-L71)

Spark leverages the interval being calculated in above, "minimum" value of intervals, and blindly adds the value to token's issue date to calculates the next renewal date for the token, and picks "minimum" value again. In problematic case, the value would be `86400048` (86400048 + 0) which is quite smaller than current timestamp.

2c64b731ae/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala (L228-L234)

The next renewal date is subtracted with current timestamp again to get the interval, and multiplexed by configured ratio to produce the final schedule interval. In problematic case, this value goes to negative.

2c64b731ae/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala (L180-L188)

There's a safety guard to not allow negative value, but that's simply 0 meaning schedule immediately. This triggers next calculation of next renewal date to calculate the schedule interval, lead to the same behavior, hence updating delegation token immediately and continuously.

As we fetch token just before the calculation happens, the actual issue date is likely slightly before, hence it's not that dangerous to use current timestamp as issue date for the token the issue date has not been set up properly. Still, it's better not to leave the token implementation as it is, so we log warn message to let end users consult with token implementer.

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

Yes. End users won't encounter the tight loop of schedule of token renewal after the PR. In end users' perspective of reflection, there's nothing end users need to change.

### How was this patch tested?

Manually tested with problematic environment.

Closes #30366 from HeartSaVioR/SPARK-33440.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-01 06:44:15 +09:00
Dongjoon Hyun c6994354f7
[SPARK-33545][CORE] Support Fallback Storage during Worker decommission
### What changes were proposed in this pull request?

This PR aims to support storage migration to the fallback storage like cloud storage (`S3`) during worker decommission for the corner cases where the exceptions occur or there is no live peer left.

Although this PR focuses on cloud storage like `S3` which has a TTL feature in order to simplify Spark's logic, we can use alternative fallback storages like HDFS/NFS(EFS) if the user provides a clean-up mechanism.

### Why are the changes needed?

Currently, storage migration is not possible when there is no available executor. For example, when there is one executor, the executor cannot perform storage migration because it has no peer.

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

Yes. This is a new feature.

### How was this patch tested?

Pass the CIs with newly added test cases.

Closes #30492 from dongjoon-hyun/SPARK-33545.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-30 13:29:50 -08:00
Erik Krogen f3c2583cc3 [SPARK-33185][YARN][FOLLOW-ON] Leverage RM's RPC API instead of REST to fetch driver log links in yarn.Client
### What changes were proposed in this pull request?
This is a follow-on to PR #30096 which initially added support for printing direct links to the driver stdout/stderr logs from the application report output in `yarn.Client` using the `spark.yarn.includeDriverLogsLink` configuration. That PR made use of the ResourceManager's REST APIs to fetch the necessary information to construct the links. This PR proposes removing the dependency on the REST API, since the new logic is the only place in `yarn.Client` which makes use of this API, and instead leverages the RPC API via `YarnClient`, which brings the code in line with the rest of `yarn.Client`.

### Why are the changes needed?

While the old logic worked okay when running a Spark application in a "standard" environment with full access to Kerberos credentials, it can fail when run in an environment with restricted Kerberos credentials. In our case, this environment is represented by [Azkaban](https://azkaban.github.io/), but it likely affects other job scheduling systems as well. In such an environment, the application has delegation tokens which enabled it to communicate with services such as YARN, but the RM REST API is not typically covered by such delegation tokens (note that although YARN does actually support accessing the RM REST API via a delegation token as documented [here](https://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html#Cluster_Delegation_Tokens_API), it is a new feature in alpha phase, and most deployments are likely not retrieving this token today).

Besides this enhancement, leveraging the `YarnClient` APIs greatly simplifies the processing logic, such as removing all JSON parsing.

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

Very minimal user-facing changes on top of PR #30096. Basically expands the scope of environments in which that feature will operate correctly.

### How was this patch tested?

In addition to redoing the `spark-submit` testing as mentioned in PR #30096, I also tested this logic in a restricted-credentials environment (Azkaban). It succeeds where the previous logic would fail with a 401 error.

Closes #30450 from xkrogen/xkrogen-SPARK-33185-driverlogs-followon.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-11-30 14:40:51 -06:00
Max Gekk 030b3139da [SPARK-33569][SPARK-33452][SQL][FOLLOWUP] Fix a build error in ShowPartitionsExec
### What changes were proposed in this pull request?
Use `listPartitionIdentifiers ` instead of `listPartitionByNames` in `ShowPartitionsExec`. The `listPartitionByNames` was renamed by https://github.com/apache/spark/pull/30514.

### Why are the changes needed?
To fix build error.

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

### How was this patch tested?
By running tests for the `SHOW PARTITIONS` command:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowPartitionsSuite"
```

Closes #30553 from MaxGekk/fix-build-show-partitions-exec.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 16:40:36 +00:00
Max Gekk 6fd148fea8 [SPARK-33569][SQL] Remove getting partitions by an identifier prefix
### What changes were proposed in this pull request?
1. Remove the method `listPartitionIdentifiers()` from the `SupportsPartitionManagement` interface. The method lists partitions by ident prefix.
2. Rename `listPartitionByNames()` to `listPartitionIdentifiers()`.
3. Re-implement the default method `partitionExists()` using new method.

### Why are the changes needed?
Getting partitions by ident prefix only is not used, and it can be removed to improve code maintenance. Also this makes the `SupportsPartitionManagement` interface cleaner.

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

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt "test:testOnly org.apache.spark.sql.connector.catalog.*"
```

Closes #30514 from MaxGekk/remove-listPartitionIdentifiers.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 14:05:49 +00:00
Max Gekk 0a612b6a40 [SPARK-33452][SQL] Support v2 SHOW PARTITIONS
### What changes were proposed in this pull request?
1. Remove V2 logical node `ShowPartitionsStatement `, and replace it by V2 `ShowPartitions`.
2. Implement V2 execution node `ShowPartitionsExec` similar to V1 `ShowPartitionsCommand`.

### Why are the changes needed?
To have feature parity with Datasource V1.

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

Before the change, `SHOW PARTITIONS` fails in V2 table catalogs with the exception:
```
org.apache.spark.sql.AnalysisException: SHOW PARTITIONS is only supported with v1 tables.
   at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.org$apache$spark$sql$catalyst$analysis$ResolveSessionCatalog$$parseV1Table(ResolveSessionCatalog.scala:628)
   at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:466)
```

### How was this patch tested?
By running the following test suites:
1. Modified `ShowPartitionsParserSuite` where `ShowPartitionsStatement` is replaced by V2 `ShowPartitions`.
2. `v2.ShowPartitionsSuite`

Closes #30398 from MaxGekk/show-partitions-exec-v2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 13:45:53 +00:00
Pascal Gillet 6e5446e61f [SPARK-33579][UI] Fix executor blank page behind proxy
### What changes were proposed in this pull request?

Fix some "hardcoded" API urls in Web UI.
More specifically, we avoid the use of `location.origin` when constructing URLs for internal API calls within the JavaScript.
Instead, we use `apiRoot` global variable.

### Why are the changes needed?

On one hand, it allows us to build relative URLs. On the other hand, `apiRoot` reflects the Spark property `spark.ui.proxyBase` which can be set to change the root path of the Web UI.

If `spark.ui.proxyBase` is actually set, original URLs become incorrect, and we end up with an executors blank page.
I encounter this bug when accessing the Web UI behind a proxy (in my case a Kubernetes Ingress).

See the following link for more context:
https://github.com/jupyterhub/jupyter-server-proxy/issues/57#issuecomment-699163115

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

Yes, as all the changes introduced are in the JavaScript for the Web UI.

### How the changes have been tested ?
I modified/debugged the JavaScript as in the commit with the help of the developer tools in Google Chrome, while accessing the Web UI of my Spark app behind my k8s ingress.

Closes #30523 from pgillet/fix-executors-blank-page-behind-proxy.

Authored-by: Pascal Gillet <pascal.gillet@stack-labs.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-11-30 19:31:42 +09:00
Wenchen Fan 5cfbdddefe [SPARK-33480][SQL] Support char/varchar type
### What changes were proposed in this pull request?

This PR adds the char/varchar type which is kind of a variant of string type:
1. Char type is fixed-length string. When comparing char type values, we need to pad the shorter one to the longer length.
2. Varchar type is string with a length limitation.

To implement the char/varchar semantic, this PR:
1. Do string length check when writing to char/varchar type columns.
2. Do string padding when reading char type columns. We don't do it at the writing side to save storage space.
3. Do string padding when comparing char type column with string literal or another char type column. (string literal is fixed length so should be treated as char type as well)

To simplify the implementation, this PR doesn't propagate char/varchar type info through functions/operators(e.g. `substring`). That said, a column can only be char/varchar type if it's a table column, not a derived column like `SELECT substring(col)`.

To be safe, this PR doesn't add char/varchar type to the query engine(expression input check, internal row framework, codegen framework, etc.). We will replace char/varchar type by string type with metadata (`Attribute.metadata` or `StructField.metadata`) that includes the original type string before it goes into the query engine. That said, the existing code will not see char/varchar type but only string type.

char/varchar type may come from several places:
1. v1 table from hive catalog.
2. v2 table from v2 catalog.
3. user-specified schema in `spark.read.schema` and `spark.readStream.schema`
4. `Column.cast`
5. schema string in places like `from_json`, pandas UDF, etc. These places use SQL parser which replaces char/varchar with string already, even before this PR.

This PR covers all the above cases, implements the length check and padding feature by looking at string type with special metadata.

### Why are the changes needed?

char and varchar are standard SQL types. varchar is widely used in other databases instead of string type.

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

For hive tables: now the table insertion fails if the value exceeds char/varchar length. Previously we truncate the value silently.

For other tables:
1. now char type is allowed.
2. now we have length check when inserting to varchar columns. Previously we write the value as it is.

### How was this patch tested?

new tests

Closes #30412 from cloud-fan/char.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 09:23:05 +00:00
gengjiaan b665d58819 [SPARK-28646][SQL] Fix bug of Count so as consistent with mainstream databases
### What changes were proposed in this pull request?
Currently, Spark allows calls to `count` even for non parameterless aggregate function. For example, the following query actually works:
`SELECT count() FROM tenk1;`
On the other hand, mainstream databases will throw an error.
**Oracle**
`> ORA-00909: invalid number of arguments`
**PgSQL**
`ERROR:  count(*) must be used to call a parameterless aggregate function`
**MySQL**
`> 1064 - You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near ')`

### Why are the changes needed?
Fix a bug so that consistent with mainstream databases.
There is an example query output with/without this fix.
`SELECT count() FROM testData;`
The output before this fix:
`0`
The output after this fix:
```
org.apache.spark.sql.AnalysisException
cannot resolve 'count()' due to data type mismatch: count requires at least one argument.; line 1 pos 7
```

### Does this PR introduce _any_ user-facing change?
Yes.
If not specify parameter for `count`, will throw an error.

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

Closes #30541 from beliefer/SPARK-28646.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-30 17:04:38 +09:00
xuewei.linxuewei 225c2e2815 [SPARK-33498][SQL][FOLLOW-UP] Deduplicate the unittest by using checkCastWithParseError
### What changes were proposed in this pull request?

Dup code removed in SPARK-33498 as follow-up.

### Why are the changes needed?

Nit.

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

No.

### How was this patch tested?

Existing UT.

Closes #30540 from leanken/leanken-SPARK-33498.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-30 15:36:26 +09:00
Terry Kim 0fd9f57dd4 [SPARK-33448][SQL] Support CACHE/UNCACHE TABLE commands for v2 tables
### What changes were proposed in this pull request?

This PR proposes to support `CHACHE/UNCACHE TABLE` commands for v2 tables.

In addtion, this PR proposes to migrate `CACHE/UNCACHE TABLE` to use `UnresolvedTableOrView` 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?

To support `CACHE/UNCACHE TABLE` commands for v2 tables.

Note that `CACHE/UNCACHE TABLE` for v1 tables/views go through `SparkSession.table` to resolve identifier, which resolves temp views first, so there is no change in the behavior by moving to the new framework.

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

Yes. Now the user can run `CACHE/UNCACHE TABLE` commands on v2 tables.

### How was this patch tested?

Added/updated existing tests.

Closes #30403 from imback82/cache_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 05:37:10 +00:00
Kent Yao 2da72593c1 [SPARK-32976][SQL] Support column list in INSERT statement
### What changes were proposed in this pull request?

#### JIRA expectations
```
   INSERT currently does not support named column lists.

   INSERT INTO <table> (col1, col2,…) VALUES( 'val1', 'val2', … )
   Note, we assume the column list contains all the column names. Issue an exception if the list is not complete. The column order could be different from the column order defined in the table definition.
```
#### implemetations
In this PR, we add a column list  as an optional part to the `INSERT OVERWRITE/INTO` statements:
```
  /**
   * {{{
   *   INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]? [identifierList] ...
   *   INSERT INTO [TABLE] tableIdentifier [partitionSpec]  [identifierList] ...
   * }}}
   */
```
The column list represents all expected columns with an explicit order that you want to insert to the target table. **Particularly**,  we assume the column list contains all the column names in the current implementation, it will fail when the list is incomplete.

In **Analyzer**, we add a code path to resolve the column list in the `ResolveOutputRelation` rule before it is transformed to v1 or v2 command. It will fail here if the list has any field that not belongs to the target table.

Then, for v2 command, e.g. `AppendData`, we use the resolved column list and output of the target table to resolve the output of the source query `ResolveOutputRelation` rule. If the list has duplicated columns, we fail. If the list is not empty but the list size does not match the target table, we fail. If no other exceptions occur, we use the column list to map the output of the source query to the output of the target table.  The column list will be set to Nil and it will not hit the rule again after it is resolved.

for v1 command, those all happen in the `PreprocessTableInsertion` rule

### Why are the changes needed?
 new feature support

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

yes, insert into/overwrite table support specify column list
### How was this patch tested?

new tests

Closes #29893 from yaooqinn/SPARK-32976.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 05:23:23 +00:00
Josh Soref 485145326a [MINOR] Spelling bin core docs external mllib repl
### What changes were proposed in this pull request?

This PR intends to fix typos in the sub-modules:
* `bin`
* `core`
* `docs`
* `external`
* `mllib`
* `repl`
* `pom.xml`

Split per srowen https://github.com/apache/spark/pull/30323#issuecomment-728981618

NOTE: The misspellings have been reported at 706a726f87 (commitcomment-44064356)

### Why are the changes needed?

Misspelled words make it harder to read / understand content.

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

There are various fixes to documentation, etc...

### How was this patch tested?

No testing was performed

Closes #30530 from jsoref/spelling-bin-core-docs-external-mllib-repl.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-11-30 13:59:51 +09:00
Chao Sun feda7299e3 [SPARK-33567][SQL] DSv2: Use callback instead of passing Spark session and v2 relation for refreshing cache
### What changes were proposed in this pull request?

This replaces Spark session and `DataSourceV2Relation` in V2 write plans by replacing them with a callback `afterWrite`.

### Why are the changes needed?

Per discussion in #30429, it's better to not pass Spark session and `DataSourceV2Relation` through Spark plans. Instead we can use a callback which makes the interface cleaner.

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

No

### How was this patch tested?

N/A

Closes #30491 from sunchao/SPARK-33492-followup.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-30 04:50:50 +00:00
Yuming Wang a5e13acd19 [SPARK-33582][SQL] Hive Metastore support filter by not-equals
### What changes were proposed in this pull request?

This pr make partition predicate pushdown into Hive metastore support not-equals operator.

Hive related changes:
b8bd4594be/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java (L2194-L2207)
https://issues.apache.org/jira/browse/HIVE-2702

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #30534 from wangyum/SPARK-33582.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-30 11:24:15 +09:00
Yuming Wang f93d4395b2 [SPARK-33589][SQL] Close opened session if the initialization fails
### What changes were proposed in this pull request?

This pr add try catch when opening session.

### Why are the changes needed?

Close opened session if the initialization fails.

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

No.

### How was this patch tested?

Manual test.

Before this pr:

```
[rootspark-3267648 spark]#  bin/beeline -u jdbc:hive2://localhost:10000/db_not_exist
NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly.
Connecting to jdbc:hive2://localhost:10000/db_not_exist
log4j:WARN No appenders could be found for logger (org.apache.hive.jdbc.Utils).
log4j:WARN Please initialize the log4j system properly.
log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more info.
Error: Could not open client transport with JDBC Uri: jdbc:hive2://localhost:10000/db_not_exist: Database 'db_not_exist' not found; (state=08S01,code=0)
Beeline version 2.3.7 by Apache Hive
beeline>
```
![image](https://user-images.githubusercontent.com/5399861/100560975-73ba5d80-32f2-11eb-8f92-b2509e7a121f.png)

After this pr:
```
[rootspark-3267648 spark]#  bin/beeline -u jdbc:hive2://localhost:10000/db_not_exist
NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly.
log4j:WARN No appenders could be found for logger (org.apache.hadoop.util.Shell).
log4j:WARN Please initialize the log4j system properly.
log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more info.
Connecting to jdbc:hive2://localhost:10000/db_not_exist
Error: Could not open client transport with JDBC Uri: jdbc:hive2://localhost:10000/db_not_exist: Failed to open new session: org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database 'db_not_exist' not found; (state=08S01,code=0)
Beeline version 2.3.7 by Apache Hive
beeline>
```
![image](https://user-images.githubusercontent.com/5399861/100560917-479edc80-32f2-11eb-986f-7a997f1163fc.png)

Closes #30536 from wangyum/SPARK-33589.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-30 11:21:02 +09:00
liucht 3d54774fb9 [SPARK-33517][SQL][DOCS] Fix the correct menu items and page links in PySpark Usage Guide for Pandas with Apache Arrow
### What changes were proposed in this pull request?

Change "Apache Arrow in Spark" to "Apache Arrow in PySpark"
and the link to “/sql-pyspark-pandas-with-arrow.html#apache-arrow-in-pyspark”

### Why are the changes needed?
When I click on the menu item it doesn't point to the correct page, and from the parent menu I can infer that the correct menu item name and link should be "Apache Arrow in PySpark".
like this:
 image
![image](https://user-images.githubusercontent.com/28332082/99954725-2b64e200-2dbe-11eb-9576-cf6a3d758980.png)

### Does this PR introduce any user-facing change?
Yes, clicking on the menu item will take you to the correct guide page

### How was this patch tested?
Manually build the doc. This can be verified as below:

cd docs
SKIP_API=1 jekyll build
open _site/sql-pyspark-pandas-with-arrow.html

Closes #30466 from liucht-inspur/master.

Authored-by: liucht <liucht@inspur.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-30 10:03:18 +09:00
Max Gekk a088a801ed
[SPARK-33585][SQL][DOCS] Fix the comment for SQLContext.tables() and mention the database column
### What changes were proposed in this pull request?
Change the comments for `SQLContext.tables()` to "The returned DataFrame has three columns, database, tableName and isTemporary".

### Why are the changes needed?
Currently, the comment mentions only 2 columns but `tables()` returns 3 columns actually:
```scala
scala> spark.range(10).createOrReplaceTempView("view1")
scala> val tables = spark.sqlContext.tables()
tables: org.apache.spark.sql.DataFrame = [database: string, tableName: string ... 1 more field]

scala> tables.printSchema
root
 |-- database: string (nullable = false)
 |-- tableName: string (nullable = false)
 |-- isTemporary: boolean (nullable = false)

scala> tables.show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
| default|       t1|      false|
| default|       t2|      false|
| default|      ymd|      false|
|        |    view1|       true|
+--------+---------+-----------+
```

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

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

Closes #30526 from MaxGekk/sqlcontext-tables-doc.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-29 12:18:07 -08:00
Max Gekk 0054fc937f
[SPARK-33588][SQL] Respect the spark.sql.caseSensitive config while resolving partition spec in v1 SHOW TABLE EXTENDED
### What changes were proposed in this pull request?
Perform partition spec normalization in `ShowTablesCommand` according to the table schema before getting partitions from the catalog. The normalization via `PartitioningUtils.normalizePartitionSpec()` adjusts the column names in partition specification, w.r.t. the real partition column names and case sensitivity.

### Why are the changes needed?
Even when `spark.sql.caseSensitive` is `false` which is the default value, v1 `SHOW TABLE EXTENDED` is case sensitive:
```sql
spark-sql> CREATE TABLE tbl1 (price int, qty int, year int, month int)
         > USING parquet
         > partitioned by (year, month);
spark-sql> INSERT INTO tbl1 PARTITION(year = 2015, month = 1) SELECT 1, 1;
spark-sql> SHOW TABLE EXTENDED LIKE 'tbl1' PARTITION(YEAR = 2015, Month = 1);
Error in query: Partition spec is invalid. The spec (YEAR, Month) must match the partition spec (year, month) defined in table '`default`.`tbl1`';
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the `SHOW TABLE EXTENDED` command respects the SQL config. And for example above, it returns correct result:
```sql
spark-sql> SHOW TABLE EXTENDED LIKE 'tbl1' PARTITION(YEAR = 2015, Month = 1);
default	tbl1	false	Partition Values: [year=2015, month=1]
Location: file:/Users/maximgekk/spark-warehouse/tbl1/year=2015/month=1
Serde Library: org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
Storage Properties: [serialization.format=1, path=file:/Users/maximgekk/spark-warehouse/tbl1]
Partition Parameters: {transient_lastDdlTime=1606595118, totalSize=623, numFiles=1}
Created Time: Sat Nov 28 23:25:18 MSK 2020
Last Access: UNKNOWN
Partition Statistics: 623 bytes
```

### How was this patch tested?
By running the modified test suite `v1/ShowTablesSuite`

Closes #30529 from MaxGekk/show-table-case-sensitive-spec.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-29 12:10:16 -08:00
Shixiong Zhu c8286ec416
[SPARK-33587][CORE] Kill the executor on nested fatal errors
### What changes were proposed in this pull request?

Currently we will kill the executor when hitting a fatal error. However, if the fatal error is wrapped by another exception, such as
- java.util.concurrent.ExecutionException, com.google.common.util.concurrent.UncheckedExecutionException, com.google.common.util.concurrent.ExecutionError when using Guava cache or Java thread pool.
- SparkException thrown from cf98a761de/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala (L231) or cf98a761de/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala (L296)

We will still keep the executor running. Fatal errors are usually unrecoverable (such as OutOfMemoryError), some components may be in a broken state when hitting a fatal error and it's hard to predicate the behaviors of a broken component. Hence, it's better to detect the nested fatal error as well and kill the executor. Then we can rely on Spark's fault tolerance to recover.

### Why are the changes needed?

Fatal errors are usually unrecoverable (such as OutOfMemoryError), some components may be in a broken state when hitting a fatal error and it's hard to predicate the behaviors of a broken component. Hence, it's better to detect the nested fatal error as well and kill the executor. Then we can rely on Spark's fault tolerance to recover.

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

Yep. There is a slight internal behavior change on when to kill an executor. We will kill the executor when detecting a nested fatal error in the exception chain. `spark.executor.killOnFatalError.depth` is added to allow users to turn off this change if the slight behavior change impacts them.

### How was this patch tested?

The new method `Executor.isFatalError` is tested by `spark.executor.killOnNestedFatalError`.

Closes #30528 from zsxwing/SPARK-33587.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-29 11:56:48 -08:00
Kazuaki Ishizaki b94ff1e870
[SPARK-33590][DOCS][SQL] Add missing sub-bullets in Spark SQL Guide
### What changes were proposed in this pull request?

Add the missing sub-bullets in the left side of `Spark SQL Guide`

### Why are the changes needed?

The three sub-bullets in the left side is not consistent with the contents (five bullets) in the right side.

![image](https://user-images.githubusercontent.com/1315079/100546388-7a21e880-32a4-11eb-922d-62a52f4f9f9b.png)

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

Yes, you can see more lines in the left menu.

### How was this patch tested?

Manually build the doc as follows. This can be verified as attached:

```
cd docs
SKIP_API=1 jekyll build
firefox _site/sql-pyspark-pandas-with-arrow.html
```

![image](https://user-images.githubusercontent.com/1315079/100546399-8ad25e80-32a4-11eb-80ac-44af0aebc717.png)

Closes #30537 from kiszk/SPARK-33590.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-29 11:24:58 -08:00
Yuming Wang ba178f852f [SPARK-33581][SQL][TEST] Refactor HivePartitionFilteringSuite
### What changes were proposed in this pull request?

This pr refactor HivePartitionFilteringSuite.

### Why are the changes needed?

To make it easy to maintain.

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

No.

### How was this patch tested?

N/A

Closes #30525 from wangyum/SPARK-33581.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-11-29 09:36:55 +08:00
Max Gekk bfe9380ba2
[MINOR][SQL] Remove getTables() from r.SQLUtils
### What changes were proposed in this pull request?
Remove the unused method `getTables()` from `r.SQLUtils`. The method was used before the changes https://github.com/apache/spark/pull/17483 but R's `tables.default` was rewritten using `listTables()`: https://github.com/apache/spark/pull/17483/files#diff-2c01472a7bcb1d318244afcd621d726e00d36cd15dffe7e44fa96c54fce4cd9aR220-R223

### Why are the changes needed?
To improve code maintenance, and remove the dead code.

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

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

Closes #30527 from MaxGekk/remove-getTables-in-r-SQLUtils.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-28 16:58:40 -08:00
Liang-Chi Hsieh 3650a6bd97
[SPARK-33580][CORE] resolveDependencyPaths should use classifier attribute of artifact
### What changes were proposed in this pull request?

This patch proposes to use classifier attribute to construct artifact path instead of type.

### Why are the changes needed?

`resolveDependencyPaths` now takes artifact type to decide to add "-tests" postfix. However, the path pattern of ivy in `resolveMavenCoordinates` is `[organization]_[artifact][revision](-[classifier]).[ext]`. We should use classifier instead of type to construct file path.

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

No

### How was this patch tested?

Unit test. Manual test.

Closes #30524 from viirya/SPARK-33580.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-28 12:47:47 -08:00
Kousuke Saruta cf98a761de [SPARK-33570][SQL][TESTS] Set the proper version of gssapi plugin automatically for MariaDBKrbIntegrationSuite
### What changes were proposed in this pull request?

This PR changes mariadb_docker_entrypoint.sh to set the proper version automatically for mariadb-plugin-gssapi-server.
The proper version is based on the one of mariadb-server.
Also, this PR enables to use arbitrary docker image by setting the environment variable `MARIADB_CONTAINER_IMAGE_NAME`.

### Why are the changes needed?

For `MariaDBKrbIntegrationSuite`, the version of `mariadb-plugin-gssapi-server` is currently set to `10.5.5` in `mariadb_docker_entrypoint.sh` but it's no longer available in the official apt repository and `MariaDBKrbIntegrationSuite` doesn't pass for now.
It seems that only the most recent three versions are available for each major version and they are `10.5.6`, `10.5.7` and `10.5.8` for now.
Further, the release cycle of MariaDB seems to be very rapid (1 ~ 2 months) so I don't think it's a good idea to set to an specific version for `mariadb-plugin-gssapi-server`.

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

No.

### How was this patch tested?

Confirmed that `MariaDBKrbIntegrationSuite` passes with the following commands.
```
$  build/sbt -Pdocker-integration-tests -Phive -Phive-thriftserver package "testOnly org.apache.spark.sql.jdbc.MariaDBKrbIntegrationSuite"
```
In this case, we can see what version of `mariadb-plugin-gssapi-server` is going to be installed in the following container log message.
```
Installing mariadb-plugin-gssapi-server=1:10.5.8+maria~focal
```

Or, we can set MARIADB_CONTAINER_IMAGE_NAME for a specific version of MariaDB.
```
$ MARIADB_DOCKER_IMAGE_NAME=mariadb:10.5.6 build/sbt -Pdocker-integration-tests -Phive -Phive-thriftserver package "testOnly org.apache.spark.sql.jdbc.MariaDBKrbIntegrationSuite"
```
```
Installing mariadb-plugin-gssapi-server=1:10.5.6+maria~focal
```

Closes #30515 from sarutak/fix-MariaDBKrbIntegrationSuite.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-11-28 23:38:11 +09:00
Josh Soref 13fd272cd3 Spelling r common dev mlib external project streaming resource managers python
### What changes were proposed in this pull request?

This PR intends to fix typos in the sub-modules:
* `R`
* `common`
* `dev`
* `mlib`
* `external`
* `project`
* `streaming`
* `resource-managers`
* `python`

Split per srowen https://github.com/apache/spark/pull/30323#issuecomment-728981618

NOTE: The misspellings have been reported at 706a726f87 (commitcomment-44064356)

### Why are the changes needed?

Misspelled words make it harder to read / understand content.

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

There are various fixes to documentation, etc...

### How was this patch tested?

No testing was performed

Closes #30402 from jsoref/spelling-R_common_dev_mlib_external_project_streaming_resource-managers_python.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-11-27 10:22:45 -06:00
luluorta 35ded12fc6 [SPARK-33141][SQL] Capture SQL configs when creating permanent views
### What changes were proposed in this pull request?
This PR makes CreateViewCommand/AlterViewAsCommand capturing runtime SQL configs and store them as view properties. These configs will be applied during the parsing and analysis phases of the view resolution. Users can set `spark.sql.legacy.useCurrentConfigsForView` to `true` to restore the behavior before.

### Why are the changes needed?
This PR is a sub-task of [SPARK-33138](https://issues.apache.org/jira/browse/SPARK-33138) that proposes to unify temp view and permanent view behaviors. This PR makes permanent views mimicking the temp view behavior that "fixes" view semantic by directly storing resolved LogicalPlan. For example, if a user uses spark 2.4 to create a view that contains null values from division-by-zero expressions, she may not want that other users' queries which reference her view throw exceptions when running on spark 3.x with ansi mode on.

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

### How was this patch tested?
added UT + existing UTs (improved)

Closes #30289 from luluorta/SPARK-33141.

Authored-by: luluorta <luluorta@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 13:32:25 +00:00
xuewei.linxuewei b9f2f78de5 [SPARK-33498][SQL] Datetime parsing should fail if the input string can't be parsed, or the pattern string is invalid
### What changes were proposed in this pull request?

Datetime parsing should fail if the input string can't be parsed, or the pattern string is invalid, when ANSI mode is enable. This patch should update GetTimeStamp, UnixTimeStamp, ToUnixTimeStamp and Cast.

### Why are the changes needed?

For ANSI mode.

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

No.

### How was this patch tested?

Added UT and Existing UT.

Closes #30442 from leanken/leanken-SPARK-33498.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 13:24:11 +00:00
gengjiaan e43255051c [SPARK-28645][SQL] ParseException is thrown when the window is redefined
### What changes were proposed in this pull request?
Currently in Spark one could redefine a window. For instance:

`select count(*) OVER w FROM tenk1 WINDOW w AS (ORDER BY unique1), w AS (ORDER BY unique1);`
The window `w` is defined two times. In PgSQL, on the other hand, a thrown will happen:

`ERROR:  window "w" is already defined`

### Why are the changes needed?
The current implement gives the following window definitions a higher priority. But it wasn't Spark's intention and users can't know from any document of Spark.
This PR fixes the bug.

### Does this PR introduce _any_ user-facing change?
Yes.
There is an example query output with/without this fix.
```
SELECT
    employee_name,
    salary,
    first_value(employee_name) OVER w highest_salary,
    nth_value(employee_name, 2) OVER w second_highest_salary
FROM
    basic_pays
WINDOW
    w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),
    w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING)
ORDER BY salary DESC
```
The output before this fix:
```
Larry Bott	11798	Larry Bott	Gerard Bondur
Gerard Bondur	11472	Larry Bott	Gerard Bondur
Pamela Castillo	11303	Larry Bott	Gerard Bondur
Barry Jones	10586	Larry Bott	Gerard Bondur
George Vanauf	10563	Larry Bott	Gerard Bondur
Loui Bondur	10449	Larry Bott	Gerard Bondur
Mary Patterson	9998	Larry Bott	Gerard Bondur
Steve Patterson	9441	Larry Bott	Gerard Bondur
Julie Firrelli	9181	Larry Bott	Gerard Bondur
Jeff Firrelli	8992	Larry Bott	Gerard Bondur
William Patterson	8870	Larry Bott	Gerard Bondur
Diane Murphy	8435	Larry Bott	Gerard Bondur
Leslie Jennings	8113	Larry Bott	Gerard Bondur
Gerard Hernandez	6949	Larry Bott	Gerard Bondur
Foon Yue Tseng	6660	Larry Bott	Gerard Bondur
Anthony Bow	6627	Larry Bott	Gerard Bondur
Leslie Thompson	5186	Larry Bott	Gerard Bondur
```
The output after this fix:
```
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException

The definition of window 'w' is repetitive(line 8, pos 0)
```

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

Closes #30512 from beliefer/SPARK-28645.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 10:27:08 +00:00
Terry Kim 2c41d9d8fa [SPARK-33522][SQL] Improve exception messages while handling UnresolvedTableOrView
### What changes were proposed in this pull request?

This PR proposes to improve the exception messages while `UnresolvedTableOrView` is handled based on this suggestion: https://github.com/apache/spark/pull/30321#discussion_r521127001.

Currently, when an identifier is resolved to a temp view when a table/permanent view is expected, the following exception message is displayed (e.g., for `SHOW CREATE TABLE`):
```
t is a temp view not table or permanent view.
```
After this PR, the message will be:
```
t is a temp view. 'SHOW CREATE TABLE' expects a table or permanent view.
```

Also, if an identifier is not resolved, the following exception message is currently used:
```
Table or view not found: t
```
After this PR, the message will be:
```
Table or permanent view not found for 'SHOW CREATE TABLE': t
```
or
```
Table or view not found for 'ANALYZE TABLE ... FOR COLUMNS ...': t
```

### Why are the changes needed?

To improve the exception message.

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

Yes, the exception message will be changed as described above.

### How was this patch tested?

Updated existing tests.

Closes #30475 from imback82/unresolved_table_or_view.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 10:16:56 +00:00
Terry Kim 8792280a73 [SPARK-33575][SQL] Fix misleading exception for "ANALYZE TABLE ... FOR COLUMNS" on temporary views
### What changes were proposed in this pull request?

This PR proposes to fix the exception message for `ANALYZE TABLE ... FOR COLUMNS` on temporary views.

The current behavior throws `NoSuchTableException` even if the temporary view exists:
```
sql("CREATE TEMP VIEW t AS SELECT 1 AS id")
sql("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS id")
org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 't' not found in database 'db';
  at org.apache.spark.sql.execution.command.AnalyzeColumnCommand.analyzeColumnInTempView(AnalyzeColumnCommand.scala:76)
  at org.apache.spark.sql.execution.command.AnalyzeColumnCommand.run(AnalyzeColumnCommand.scala:54)
```

After this PR, more reasonable exception is thrown:
```
org.apache.spark.sql.AnalysisException: Temporary view `testView` is not cached for analyzing columns.;
[info]   at org.apache.spark.sql.execution.command.AnalyzeColumnCommand.analyzeColumnInTempView(AnalyzeColumnCommand.scala:74)
[info]   at org.apache.spark.sql.execution.command.AnalyzeColumnCommand.run(AnalyzeColumnCommand.scala:54)
```

### Why are the changes needed?

To fix a misleading exception.

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

Yes, the exception thrown is changed as shown above.

### How was this patch tested?

Updated existing test.

Closes #30519 from imback82/analyze_table_message.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-27 07:08:24 +00:00
yangjie01 433ae9064f [SPARK-33566][CORE][SQL][SS][PYTHON] Make unescapedQuoteHandling option configurable when read CSV
### What changes were proposed in this pull request?
There are some differences between Spark CSV, opencsv and commons-csv, the typical case are described in SPARK-33566, When there are both unescaped quotes and unescaped qualifier in value,  the results of parsing are different.

The reason for the difference is Spark use `STOP_AT_DELIMITER` as default `UnescapedQuoteHandling` to build `CsvParser` and it not configurable.

On the other hand, opencsv and commons-csv use the parsing mechanism similar to `STOP_AT_CLOSING_QUOTE ` by default.

So this pr make `unescapedQuoteHandling` option configurable to get the same parsing result as opencsv and commons-csv.

### Why are the changes needed?
Make unescapedQuoteHandling option configurable when read CSV to make parsing more flexible。

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

### How was this patch tested?

- Pass the Jenkins or GitHub Action

- Add a new case similar to that described in SPARK-33566

Closes #30518 from LuciferYang/SPARK-33566.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-27 15:47:39 +09:00
zero323 d082ad0abf [SPARK-33563][PYTHON][R][SQL] Expose inverse hyperbolic trig functions in PySpark and SparkR
### What changes were proposed in this pull request?

This PR adds the following functions (introduced in Scala API with SPARK-33061):

- `acosh`
- `asinh`
- `atanh`

to Python and R.

### Why are the changes needed?

Feature parity.

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

New functions.

### How was this patch tested?

New unit tests.

Closes #30501 from zero323/SPARK-33563.

Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-27 11:00:09 +09:00
Maryann Xue dfa3978d91 [SPARK-33551][SQL] Do not use custom shuffle reader for repartition
### What changes were proposed in this pull request?

This PR fixes an AQE issue where local shuffle reader, partition coalescing, or skew join optimization can be mistakenly applied to a shuffle introduced by repartition or a regular shuffle that logically replaces a repartition shuffle.
The proposed solution checks for the presence of any repartition shuffle and filters out not applicable optimization rules for the final stage in an AQE plan.

### Why are the changes needed?

Without the change, the output of a repartition query may not be correct.

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

No.

### How was this patch tested?

Added UT.

Closes #30494 from maryannxue/csr-repartition.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-11-25 19:32:22 -08:00
HyukjinKwon ed9e6fc182 [SPARK-33565][INFRA][FOLLOW-UP] Keep the test coverage with Python 3.8 in GitHub Actions
### What changes were proposed in this pull request?

This PR proposes to keep the test coverage with Python 3.8 in GitHub Actions. It is not tested for now in Jenkins due to an env issue.

**Before this change in GitHub Actions:**

```
========================================================================
Running PySpark tests
========================================================================
Running PySpark tests. Output is in /__w/spark/spark/python/unit-tests.log
Will test against the following Python executables: ['python3.6', 'pypy3']
...
```

**After this change in GitHub Actions:**

```
========================================================================
Running PySpark tests
========================================================================
Running PySpark tests. Output is in /__w/spark/spark/python/unit-tests.log
Will test against the following Python executables: ['python3.6', 'python3.8', 'pypy3']
```

### Why are the changes needed?

To keep the test coverage with Python 3.8 in GitHub Actions.

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

No, dev-only.

### How was this patch tested?

GitHub Actions in this build will test.

Closes #30510 from HyukjinKwon/SPARK-33565.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-26 11:42:12 +09:00
Gengliang Wang 919ea45e89 [SPARK-33562][UI] Improve the style of the checkbox in executor page
### What changes were proposed in this pull request?

1. Remove the fixed width style of class `container-fluid-div`. So that the UI looks clean when the text is long.
2. Add one space between a checkbox and the text on the right side, which is consistent with the stage page.

### Why are the changes needed?

The width of class `container-fluid-div` is set as 200px after https://github.com/apache/spark/pull/21688 . This makes the checkbox in the executor page messy.
![image](https://user-images.githubusercontent.com/1097932/100242069-3bc5ab80-2ee9-11eb-8c7d-96c221398fee.png)

We should remove the width limit.

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

No
### How was this patch tested?

Manual test.
After the changes:
![image](https://user-images.githubusercontent.com/1097932/100257802-2f4a4e80-2efb-11eb-9eb0-92d6988ad14b.png)

Closes #30500 from gengliangwang/reviseStyle.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-26 10:19:38 +09:00
Liang-Chi Hsieh fb7b870214 [SPARK-33523][SQL][TEST][FOLLOWUP] Fix benchmark case name in SubExprEliminationBenchmark
### What changes were proposed in this pull request?

Fix the wrong benchmark case name.

### Why are the changes needed?

The last commit to refactor the benchmark code missed a change of case name.

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

No, dev only.

### How was this patch tested?

Unit test.

Closes #30505 from viirya/SPARK-33523-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-25 15:22:47 -08:00
shane knapp c529426d87 [SPARK-33565][BUILD][PYTHON] remove python3.8 and fix breakage
### What changes were proposed in this pull request?
remove python 3.8 from python/run-tests.py and stop build breaks

### Why are the changes needed?
the python tests are running against the bare-bones system install of python3, rather than an anaconda environment.

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

### How was this patch tested?
via jenkins

Closes #30506 from shaneknapp/remove-py38.

Authored-by: shane knapp <incomplete@gmail.com>
Signed-off-by: shane knapp <incomplete@gmail.com>
2020-11-25 15:15:50 -08:00
Yuming Wang 1de3fc4282 [SPARK-33525][SQL] Update hive-service-rpc to 3.1.2
### What changes were proposed in this pull request?

We supported Hive metastore are 0.12.0 through 3.1.2, but we supported hive-jdbc are 0.12.0 through 2.3.7. It will throw `TProtocolException` if we use hive-jdbc 3.x:

```
[rootspark-3267648 apache-hive-3.1.2-bin]# bin/beeline -u jdbc:hive2://localhost:10000/default
Connecting to jdbc:hive2://localhost:10000/default
Connected to: Spark SQL (version 3.1.0-SNAPSHOT)
Driver: Hive JDBC (version 3.1.2)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Beeline version 3.1.2 by Apache Hive
0: jdbc:hive2://localhost:10000/default> create table t1(id int) using parquet;
Unexpected end of file when reading from HS2 server. The root cause might be too many concurrent connections. Please ask the administrator to check the number of active connections, and adjust hive.server2.thrift.max.worker.threads if applicable.
Error: org.apache.thrift.transport.TTransportException (state=08S01,code=0)
```
```
org.apache.thrift.protocol.TProtocolException: Missing version in readMessageBegin, old client?
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:234)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:630)
	at java.base/java.lang.Thread.run(Thread.java:832)
```

This pr upgrade hive-service-rpc to 3.1.2 to fix this issue.

### Why are the changes needed?

To support hive-jdbc 3.x.

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

No.

### How was this patch tested?

Manual test:
```
[rootspark-3267648 apache-hive-3.1.2-bin]# bin/beeline -u jdbc:hive2://localhost:10000/default
Connecting to jdbc:hive2://localhost:10000/default
Connected to: Spark SQL (version 3.1.0-SNAPSHOT)
Driver: Hive JDBC (version 3.1.2)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Beeline version 3.1.2 by Apache Hive
0: jdbc:hive2://localhost:10000/default> create table t1(id int) using parquet;
+---------+
| Result  |
+---------+
+---------+
No rows selected (1.051 seconds)
0: jdbc:hive2://localhost:10000/default> insert into t1 values(1);
+---------+
| Result  |
+---------+
+---------+
No rows selected (2.08 seconds)
0: jdbc:hive2://localhost:10000/default> select * from t1;
+-----+
| id  |
+-----+
| 1   |
+-----+
1 row selected (0.605 seconds)
```

Closes #30478 from wangyum/SPARK-33525.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-25 12:37:59 -08:00
Dongjoon Hyun 7cf6a6f996 [SPARK-31257][SPARK-33561][SQL][FOLLOWUP] Fix Scala 2.13 compilation
### What changes were proposed in this pull request?

This PR is a follow-up to fix Scala 2.13 compilation.

### Why are the changes needed?

To support Scala 2.13 in Apache Spark 3.1.

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

No.

### How was this patch tested?

Pass the GitHub Action Scala 2.13 compilation job.

Closes #30502 from dongjoon-hyun/SPARK-31257.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-25 09:57:46 -08:00
Liang-Chi Hsieh 9643eab53e [SPARK-33540][SQL] Subexpression elimination for interpreted predicate
### What changes were proposed in this pull request?

This patch proposes to support subexpression elimination for interpreted predicate.

### Why are the changes needed?

Similar to interpreted projection, there are use cases when codegen predicate is not able to work, e.g. too complex schema, non-codegen expression, etc. When there are frequently occurring expressions (subexpressions) among predicate expression, the performance is quite bad as we need to re-compute same expressions. We should be able to support subexpression elimination for interpreted predicate like interpreted projection.

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

No, this doesn't change user behavior.

### How was this patch tested?

Unit test and benchmark.

Closes #30497 from viirya/SPARK-33540.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-11-25 08:55:39 -08:00
Gengliang Wang d691d85701 [SPARK-33496][SQL] Improve error message of ANSI explicit cast
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/30260, there are some type conversions disallowed under ANSI mode.
We should tell users what they can do if they have to use the disallowed casting.

### Why are the changes needed?

Make it more user-friendly.

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

Yes, the error message is improved on casting failure when ANSI mode is enabled
### How was this patch tested?

Unit tests.

Closes #30440 from gengliangwang/improveAnsiCastErrorMSG.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-11-25 23:15:52 +08:00
Ryan Blue 6f68ccf532 [SPARK-31257][SPARK-33561][SQL] Unify create table syntax
### What changes were proposed in this pull request?

* Unify the create table syntax in the parser by merging Hive and DataSource clauses
* Add `SerdeInfo` and `external` boolean to statement plans and update AstBuilder to produce them
* Add conversion from create statement plan to v1 create plans in ResolveSessionCatalog
* Support new statement clauses in ResolveCatalogs conversion to v2 create plans
* Remove SparkSqlParser rules for Hive syntax
* Add "option." namespace to distinguish SERDEPROPERTIES and OPTIONS in table properties

### Why are the changes needed?

* Current behavior is confusing.
* A way to pass the Hive create options to DSv2 is needed for a Hive source.

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

Not by default, but v2 sources will be able to handle STORED AS and other Hive clauses.

### How was this patch tested?

Existing tests validate there are no behavior changes.

Update unit tests for using a statement plan for Hive create syntax:
* Move create tests from spark-sql DDLParserSuite into PlanResolutionSuite
* Add parser tests to spark-catalyst DDLParserSuite

Closes #28026 from rdblue/unify-create-table.

Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 15:09:02 +00:00
duripeng 7c59aeeef4 [SPARK-27194][SPARK-29302][SQL] Fix commit collision in dynamic partition overwrite mode
### What changes were proposed in this pull request?

When using dynamic partition overwrite, each task has its working dir under staging dir like `stagingDir/.spark-staging-{jobId}`, each task commits to `outputPath/.spark-staging-{jobId}/{partitionId}/part-{taskId}-{jobId}{ext}`.
When speculation enable, multiple task attempts would be setup for one task, **they have same task id and they would commit to same file concurrently**. Due to host done or node preemption, the partly-committed files aren't cleaned up, a FileAlreadyExistsException would be raised in this situation, resulting in job failure.

I don't try to change task commit process for dynamic partition overwrite, like adding attempt id to task working dir for each attempts and committing to final output dir via a new outputCommitCoordinator, here is reason:

1. `FileOutputCommitter` already has commit coordinator for each task attempts, we can leverage it rather than build a new one.
2. To say the least, we implement a coordinator solving task attempts commit conflict, suppose a severe case, application master failover, tasks with same attempt id and same task id would commit to same files, the `FileAlreadyExistsException` risk still exists

In this pr, I leverage FileOutputCommitter to solve the problem:

1. when initing a write job description, set `outputPath/.spark-staging-{jobId}` as the output dir
2. each task attempt writes output to `outputPath/.spark-staging-{jobId}/_temporary/${appAttemptId}/_temporary/${taskAttemptId}/{partitionId}/part-{taskId}-{jobId}{ext}`
3. leverage `FileOutputCommitter` coordinator, write job firstly commits output to `outputPath/.spark-staging-{jobId}/{partitionId}`
4. for dynamic partition overwrite, write job finally move `outputPath/.spark-staging-{jobId}/{partitionId}` to `outputPath/{partitionId}`

### Why are the changes needed?

Without this pr, dynamic partition overwrite would fail

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

No.

### How was this patch tested?

added UT.

Closes #29000 from WinkerDu/master-fix-dynamic-partition-multi-commit.

Authored-by: duripeng <duripeng@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 12:50:21 +00:00
Max Gekk 2c5cc36e3f [SPARK-33509][SQL] List partition by names from a V2 table which supports partition management
### What changes were proposed in this pull request?
1. Add new method `listPartitionByNames` to the `SupportsPartitionManagement` interface. It allows to list partitions by partition names and their values.
2. Implement new method in `InMemoryPartitionTable` which is used in DSv2 tests.

### Why are the changes needed?
Currently, the `SupportsPartitionManagement` interface exposes only `listPartitionIdentifiers` which allows to list partitions by partition values. And it requires to specify all values for partition schema fields in the prefix. This restriction does not allow to list partitions by some of partition names (not all of them).

For example, the table `tableA` is partitioned by two column `year` and `month`
```
CREATE TABLE tableA (price int, year int, month int)
USING _
partitioned by (year, month)
```
and has the following partitions:
```
PARTITION(year = 2015, month = 1)
PARTITION(year = 2015, month = 2)
PARTITION(year = 2016, month = 2)
PARTITION(year = 2016, month = 3)
```
If we want to list all partitions with `month = 2`, we have to specify `year` for **listPartitionIdentifiers()** which not always possible as we don't know all `year` values in advance. New method **listPartitionByNames()** allows to specify partition values only for `month`, and get two partitions:
```
PARTITION(year = 2015, month = 2)
PARTITION(year = 2016, month = 2)
```

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

### How was this patch tested?
By running the affected test suite `SupportsPartitionManagementSuite`.

Closes #30452 from MaxGekk/column-names-listPartitionIdentifiers.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 12:41:53 +00:00
Gengliang Wang 19f3b89d62 [SPARK-33549][SQL] Remove configuration spark.sql.legacy.allowCastNumericToTimestamp
### What changes were proposed in this pull request?

Remove SQL configuration spark.sql.legacy.allowCastNumericToTimestamp

### Why are the changes needed?

In the current master branch, there is a new configuration `spark.sql.legacy.allowCastNumericToTimestamp` which controls whether to cast Numeric types to Timestamp or not. The default value is true.

After https://github.com/apache/spark/pull/30260, the type conversion between Timestamp type and Numeric type is disallowed in ANSI mode. So, we don't need to a separate configuration `spark.sql.legacy.allowCastNumericToTimestamp` for disallowing the conversion. Users just need to set `spark.sql.ansi.enabled` for the behavior.

As the configuration is not in any released yet, we should remove the configuration to make things simpler.

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

No, since the configuration is not released yet.

### How was this patch tested?

Existing test cases

Closes #30493 from gengliangwang/LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-25 08:59:31 +00:00