Commit graph

2619 commits

Author SHA1 Message Date
Kent Yao f918c123a0 [SPARK-36552][SQL] Fix different behavior for writing char/varchar to hive and datasource table
### What changes were proposed in this pull request?

For the hive table, the actual write path and the schema handling are inconsistent when `spark.sql.legacy.charVarcharAsString` is true.

This causes problems like SPARK-36552 described.

In this PR we respect `spark.sql.legacy.charVarcharAsString` when generates hive table schema from spark data types.

### Why are the changes needed?

bugfix

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

yes, when `spark.sql.legacy.charVarcharAsString` is true, hive table with char/varchar will respect string behavior.

### How was this patch tested?

newly added test

Closes #33798 from yaooqinn/SPARK-36552.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-22 09:38:39 +09:00
Max Gekk 82a31508af [SPARK-36524][SQL] Common class for ANSI interval types
### What changes were proposed in this pull request?
Add new type `AnsiIntervalType` to `AbstractDataType.scala`, and extend it by `YearMonthIntervalType` and by `DayTimeIntervalType`

### Why are the changes needed?
To improve code maintenance. The change will allow to replace checking of both `YearMonthIntervalType` and `DayTimeIntervalType` by a check of `AnsiIntervalType`, for instance:
```scala
    case _: YearMonthIntervalType | _: DayTimeIntervalType => false
```
by
```scala
    case _: AnsiIntervalType => false
```

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

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

Closes #33753 from MaxGekk/ansi-interval-type-trait.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-08-17 12:27:56 +03:00
Kent Yao c7fa3c9090 [SPARK-36421][SQL][DOCS] Use ConfigEntry.key to fix docs and set command results
### What changes were proposed in this pull request?

This PR fixes the issue that `ConfigEntry` to be introduced to the doc field directly without calling `.key`, which causes malformed documents on the web site and in the result of `SET -v`

1. https://spark.apache.org/docs/3.1.2/configuration.html#static-sql-configuration - spark.sql.hive.metastore.jars

2. set -v
![image](https://user-images.githubusercontent.com/8326978/128292412-85100f95-24fd-4b40-a14f-d31a256dab7d.png)

### Why are the changes needed?

bugfix

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

no, but contains doc fix
### How was this patch tested?

new tests

Closes #33647 from yaooqinn/SPARK-36421.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-06 11:01:47 +09:00
Max Gekk 67cbc93263 [SPARK-36349][SQL] Disallow ANSI intervals in file-based datasources
### What changes were proposed in this pull request?
In the PR, I propose to ban `YearMonthIntervalType` and `DayTimeIntervalType` at the analysis phase while creating a table using a built-in filed-based datasource or writing a dataset to such datasource. In particular, add the following case:
```scala
case _: DayTimeIntervalType | _: YearMonthIntervalType => false
```
to all methods that override either:
- V2 `FileTable.supportsDataType()`
- V1 `FileFormat.supportDataType()`

### Why are the changes needed?
To improve user experience with Spark SQL, and output a proper error message at the analysis phase.

### Does this PR introduce _any_ user-facing change?
Yes but ANSI interval types haven't released yet. So, for users this is new behavior.

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt -Phive-2.3 "test:testOnly *HiveOrcSourceSuite"
```

Closes #33580 from MaxGekk/interval-ban-in-ds.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-08-03 20:30:20 +03:00
Chao Sun 7a27f8a07f [SPARK-36137][SQL] HiveShim should fallback to getAllPartitionsOf even if directSQL is enabled in remote HMS
### What changes were proposed in this pull request?

Change `HiveShim.getPartitionsByFilter` to always fallback to use `getAllPartitionsMethod` even if `hive.metastore.try.direct.sql` is set to true in the remote HMS.

### Why are the changes needed?

At the moment `getPartitionsByFilter` in `HiveShim` only fallback to use `getAllPartitionsMethod` when `hive.metastore.try.direct.sql` is disabled in the remote HMS, and will fail the query otherwise. However, in certain cases the remote HMS will fallback to use ORM (which only support string type for partition columns) to query the underlying RDBMS **even if this config is set to true**. In this scenario, currently Spark will not be able to recover from the exception and will just fail the query.

For instance, we encountered this bug [HIVE-21497](https://issues.apache.org/jira/browse/HIVE-21497) in HMS running Hive 3.1.2, and Spark was not able to pushdown filter for date column.

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

Yes, now if Spark is querying partitions from a remote HMS which throws exception even if `hive.metastore.try.direct.sql` is set to true, Spark will fallback to list all partitions and do the pruning on client side, instead of failing the query.

### How was this patch tested?

Tested locally with a HMS instance running 3.1.2. It's pretty hard to add a unit test for this since we don't have a mock HMS.

Closes #33382 from sunchao/SPARK-36137-direct-sql.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-08-02 16:48:43 -07:00
Linhong Liu 2f700773c2 [SPARK-36224][SQL] Use Void as the type name of NullType
### What changes were proposed in this pull request?
Change the `NullType.simpleString` to "void" to set "void" as the formal type name of `NullType`

### Why are the changes needed?
This PR is intended to address the type name discussion in PR #28833. Here are the reasons:
1. The type name of NullType is displayed everywhere, e.g. schema string, error message, document. Hence it's not possible to hide it from users, we have to choose a proper name
2. The "void" is widely used as the type name of "NULL", e.g. Hive, pgSQL
3. Changing to "void" can enable the round trip of `toDDL`/`fromDDL` for NullType. (i.e. make `from_json(col, schema.toDDL)`) work

### Does this PR introduce _any_ user-facing change?
Yes, the type name of "NULL" is changed from "null" to "void". for example:
```
scala> sql("select null as a, 1 as b").schema.catalogString
res5: String = struct<a:void,b:int>
```

### How was this patch tested?
existing test cases

Closes #33437 from linhongliu-db/SPARK-36224-void-type-name.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-02 23:19:54 +08:00
Chao Sun 0ece865ea4 [SPARK-36136][SQL][TESTS] Refactor PruneFileSourcePartitionsSuite etc to a different package
### What changes were proposed in this pull request?

Move both `PruneFileSourcePartitionsSuite` and `PrunePartitionSuiteBase` to the package `org.apache.spark.sql.execution.datasources`. Did a few refactoring to enable this.

### Why are the changes needed?

Currently both `PruneFileSourcePartitionsSuite` and `PrunePartitionSuiteBase` are in package `org.apache.spark.sql.hive.execution` which doesn't look correct as these tests are not specific to Hive. Therefore, it's better to move them into `org.apache.spark.sql.execution.datasources`, the same place where the rule `PruneFileSourcePartitions` is at.

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

No, it's just test refactoring.

### How was this patch tested?

Using existing tests:
```
build/sbt "sql/testOnly *PruneFileSourcePartitionsSuite"
```
and
```
build/sbt "hive/testOnly *PruneHiveTablePartitionsSuite"
```

Closes #33564 from sunchao/SPARK-36136-partitions-suite.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-29 17:18:23 -07:00
Angerszhuuuu 59e0c25376 [SPARK-36312][SQL] ParquetWriterSupport.setSchema should check inner field
### What changes were proposed in this pull request?
Last pr only support add inner field check for hive ddl, this pr add check for parquet data source write API.

### Why are the changes needed?
Failed earlier

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

### How was this patch tested?
Added Ut

Without this UI it failed as
```
[info] - SPARK-36312: ParquetWriteSupport should check inner field *** FAILED *** (8 seconds, 29 milliseconds)
[info]   Expected exception org.apache.spark.sql.AnalysisException to be thrown, but org.apache.spark.SparkException was thrown (HiveDDLSuite.scala:3035)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
[info]   at org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1563)
[info]   at org.scalatest.Assertions.intercept(Assertions.scala:756)
[info]   at org.scalatest.Assertions.intercept$(Assertions.scala:746)
[info]   at org.scalatest.funsuite.AnyFunSuite.intercept(AnyFunSuite.scala:1563)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$396(HiveDDLSuite.scala:3035)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$396$adapted(HiveDDLSuite.scala:3034)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withTempPath(SQLHelper.scala:69)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withTempPath$(SQLHelper.scala:66)
[info]   at org.apache.spark.sql.QueryTest.withTempPath(QueryTest.scala:34)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$395(HiveDDLSuite.scala:3034)
[info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
[info]   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1468)
[info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withView(SQLTestUtils.scala:316)
[info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withView$(SQLTestUtils.scala:314)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.withView(HiveDDLSuite.scala:396)
[info]   at org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$394(HiveDDLSuite.scala:3032)
[info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
[info]   at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
[info]   at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
[info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
[info]   at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
[info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
[info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
[info]   at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
[info]   at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
[info]   at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
[info]   at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
[info]   at scala.collection.immutable.List.foreach(List.scala:431)
[info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
[info]   at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
[info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:269)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:268)
[info]   at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1563)
[info]   at org.scalatest.Suite.run(Suite.scala:1112)
[info]   at org.scalatest.Suite.run$(Suite.scala:1094)
[info]   at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1563)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:273)
[info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:273)
[info]   at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:272)
[info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:62)
[info]   at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
[info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
[info]   at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
[info]   at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:62)
[info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:318)
[info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:513)
[info]   at sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:413)
[info]   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
[info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[info]   at java.lang.Thread.run(Thread.java:748)
[info]   Cause: org.apache.spark.SparkException: Job aborted.
[info]   at org.apache.spark.sql.errors.QueryExecutionErrors$.jobAbortedError(QueryExecutionErrors.scala:496)
[info]   at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:251)
[info]   at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:186)
[info]   at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:113)
[info]   at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:111)
[info]   at org.apache.spark.sql.execution.command.DataWritingCommandExec.executeCollect(commands.scala:125)
[info]   at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:97)
[info]   at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
[info]   at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
[info]   at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
[info]   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
[info]   at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
[info]   at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:97)
[info]   at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:93)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:481)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:481)
[info]   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
[info]   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
[info]   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
[info]   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
[info]   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:457)
[info]   at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:93)
[info]   at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:80)
[info]   at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:78)
[info]   at org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:115)
[info]   at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:848)
[info]   at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:382)
[info]   at org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:355)
[info]   at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:239)
[info]   at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:781)
[in
```

Closes #33531 from AngersZhuuuu/SPARK-36312.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-28 13:52:27 +08:00
Liang-Chi Hsieh 22ac98dcbf Revert "[SPARK-36136][SQL][TESTS] Refactor PruneFileSourcePartitionsSuite etc to a different package"
This reverts commit 634f96dde4.

Closes #33533 from viirya/revert-SPARK-36136.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-27 19:11:42 +09:00
Linhong Liu 8e7e14dc0d [SPARK-36241][SQL] Support creating tables with null column
### What changes were proposed in this pull request?
Previously we blocked creating tables with the null column to follow the hive behavior in PR #28833
In this PR, I propose the restore the previous behavior to support the null column in a table.

### Why are the changes needed?
For a complex query, it's possible to generate a column with null type. If this happens to the input query of
CTAS, the query will fail due to Spark doesn't allow creating a table with null type. From the user's perspective,
it’s hard to figure out why the null type column is produced in the complicated query and how to fix it. So removing
this constraint is more friendly to users.

### Does this PR introduce _any_ user-facing change?
Yes, this reverts the previous behavior change in #28833, for example, below command will success after this PR
```sql
CREATE TABLE t (col_1 void, col_2 int)
```

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

Closes #33488 from linhongliu-db/SPARK-36241-support-void-column.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-27 17:31:52 +08:00
Chao Sun 634f96dde4 [SPARK-36136][SQL][TESTS] Refactor PruneFileSourcePartitionsSuite etc to a different package
### What changes were proposed in this pull request?

Move both `PruneFileSourcePartitionsSuite` and `PrunePartitionSuiteBase` to the package `org.apache.spark.sql.execution.datasources`. Did a few refactoring to enable this.

### Why are the changes needed?

Currently both `PruneFileSourcePartitionsSuite` and `PrunePartitionSuiteBase` are in package `org.apache.spark.sql.hive.execution` which doesn't look correct as these tests are not specific to Hive. Therefore, it's better to move them into `org.apache.spark.sql.execution.datasources`, the same place where the rule `PruneFileSourcePartitions` is at.

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

No, it's just test refactoring.

### How was this patch tested?

Using existing tests:
```
build/sbt "sql/testOnly *PruneFileSourcePartitionsSuite"
```
and
```
build/sbt "hive/testOnly *PruneHiveTablePartitionsSuite"
```

Closes #33350 from sunchao/SPARK-36136-partitions-suite.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-26 13:03:50 -07:00
Angerszhuuuu a63802f2c6 [SPARK-34402][SQL] Group exception about data format schema
### What changes were proposed in this pull request?
Group exception about data format schema of different format, orc/parquet

### Why are the changes needed?
group exception

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

### How was this patch tested?
Not need

Closes #33296 from AngersZhuuuu/SPARK-34402.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-26 19:18:43 +08:00
Cheng Su e5616e32ee [SPARK-36269][SQL] Fix only set data columns to Hive column names config
### What changes were proposed in this pull request?

When reading Hive table, we set the Hive column id and column name configs (`hive.io.file.readcolumn.ids` and `hive.io.file.readcolumn.names`). We should set non-partition columns (data columns) for both configs, as Spark always [appends partition columns in its own Hive reader](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala#L240). The column id config has only non-partition columns, but column name config has both partition and non-partition columns. We should keep them to be consistent with only non-partition columns. This does not cause issue for public OSS Hive file format for now. But for customized internal Hive file format, it causes the issue as we are expecting these two configs to be same.

### Why are the changes needed?

Fix the code logic to be more consistent.

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

No.

### How was this patch tested?

Existing Hive tests.

Closes #33489 from c21/hive-col.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-26 18:48:06 +08:00
Shardul Mahadik 685c3fd05b [SPARK-28266][SQL] convertToLogicalRelation should not interpret path property when reading Hive tables
### What changes were proposed in this pull request?

For non-datasource Hive tables, e.g. tables written outside of Spark (through Hive or Trino), we have certain optimzations in Spark where we use Spark ORC and Parquet datasources to read these tables ([Ref](fbf53dee37/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala (L128))) rather than using the Hive serde.
If such a table contains a `path` property, Spark will try to list this path property in addition to the table location when creating an `InMemoryFileIndex`. ([Ref](fbf53dee37/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L575))) This can lead to wrong data if `path` property points to a directory location or an error if `path` is not a location. A concrete example is provided in [SPARK-28266 (comment)](https://issues.apache.org/jira/browse/SPARK-28266?focusedCommentId=17380170&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17380170).

Since these tables were not written through Spark, Spark should not interpret this `path` property as it can be set by an external system with a different meaning.

### Why are the changes needed?

For better compatibility with Hive tables generated by other platforms (non-Spark)

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

No

### How was this patch tested?

Added unit test

Closes #33328 from shardulm94/spark-28266.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-21 22:40:39 +08:00
Angerszhuuuu 251885772d [SPARK-36201][SQL][FOLLOWUP] Schema check should check inner field too
### What changes were proposed in this pull request?
When inner field have wrong schema filed name should check field name too.
![image](https://user-images.githubusercontent.com/46485123/126101009-c192d87f-1e18-4355-ad53-1419dacdeb76.png)

### Why are the changes needed?
Early check early faield

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

### How was this patch tested?
Added UT

Closes #33409 from AngersZhuuuu/SPARK-36201.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-20 21:08:03 +08:00
Kent Yao ef80356614 [SPARK-36197][SQL] Use PartitionDesc instead of TableDesc for reading hive partitioned tables
### What changes were proposed in this pull request?

A hive partition can have different `PartitionDesc`s from `TableDesc` for describing Serde/InputFormatClass/OutputFormatClass, for a hive partitioned table, we shall respect those in `PartitionDesc`.

### Why are the changes needed?

in many cases, that Spark reads hive tables could result in surprise because of this issue.

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

yes, hive partition table that contains different serde/input/output could be recognized by Spark

### How was this patch tested?

new test added

Closes #33406 from yaooqinn/SPARK-36197.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-07-19 15:59:36 +08:00
Chao Sun 37dc3f9ea7 [SPARK-36128][SQL] Apply spark.sql.hive.metastorePartitionPruning for non-Hive tables that uses Hive metastore for partition management
### What changes were proposed in this pull request?

In `CatalogFileIndex.filterPartitions`, check the config `spark.sql.hive.metastorePartitionPruning` and don't pushdown predicates to remote HMS if it is false. Instead, fallback to the `listPartitions` API and do the filtering on the client side.

### Why are the changes needed?

Currently the config `spark.sql.hive.metastorePartitionPruning` is only effective for Hive tables, and for non-Hive tables we'd always use the `listPartitionsByFilter` API from HMS client. On the other hand, by default all data source tables also manage their partitions through HMS, when the config `spark.sql.hive.manageFilesourcePartitions` is turned on. Therefore, it seems reasonable to extend the above config for non-Hive tables as well.

In certain cases the remote HMS service could throw exceptions when using the `listPartitionsByFilter` API, which, on the Spark side, is unrecoverable at the current state. Therefore it would be better to allow users to disable the API by using the above config.

For instance, HMS only allow pushdown date column when direct SQL is used instead of JDO for interacting with the underlying RDBMS, and will throw exception otherwise. Even though the Spark Hive client will attempt to recover itself when the exception happens, it only does so when the config `hive.metastore.try.direct.sql` from remote HMS is `false`. There could be cases where the value of `hive.metastore.try.direct.sql` is true but remote HMS still throws exception.

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

Yes now the config `spark.sql.hive.metastorePartitionPruning` is extended for non-Hive tables which use HMS to manage their partition metadata.

### How was this patch tested?

Added a new unit test:
```
build/sbt "hive/testOnly *PruneFileSourcePartitionsSuite -- -z SPARK-36128"
```

Closes #33348 from sunchao/SPARK-36128-by-filter.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-16 13:32:25 -07:00
Steven Aerts f06aa4a3f3 [SPARK-35985][SQL] push partitionFilters for empty readDataSchema
this commit makes sure that for File Source V2 partition filters are
also taken into account when the readDataSchema is empty.
This is the case for queries like:

    SELECT count(*) FROM tbl WHERE partition=foo
    SELECT input_file_name() FROM tbl WHERE partition=foo

### What changes were proposed in this pull request?

As described in SPARK-35985 there is bug in the File Datasource V2 which prevents it to push down to the FileScanner for queries like the ones listed above.

### Why are the changes needed?

If partitions filters are not pushed down, the whole dataset will be scanned while only one partition is interesting.

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

### How was this patch tested?

An extra test was added which relies on the output of explain, as is done in other places.

Closes #33191 from steven-aerts/SPARK-35985.

Authored-by: Steven Aerts <steven.aerts@airties.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-16 04:52:46 +00:00
PengLei e071721a51 [SPARK-36012][SQL] Add null flag in SHOW CREATE TABLE
### What changes were proposed in this pull request?
When exec the command `SHOW CREATE TABLE`, we should not lost the info null flag if the table column that
is specified `NOT NULL`

### Why are the changes needed?
[SPARK-36012](https://issues.apache.org/jira/browse/SPARK-36012)

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

### How was this patch tested?
Add UT test for V1 and existed UT for V2

Closes #33219 from Peng-Lei/SPARK-36012.

Authored-by: PengLei <peng.8lei@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-09 01:21:38 +08:00
Dongjoon Hyun f9f95686cb [SPARK-35996][BUILD] Setting version to 3.3.0-SNAPSHOT
### What changes were proposed in this pull request?

This PR aims to update `master` branch version to 3.3.0-SNAPSHOT.

### Why are the changes needed?

Start to prepare Apache Spark 3.3.0 and the published snapshot version should not conflict with `branch-3.2`.

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

N/A.

### How was this patch tested?

Pass the CIs.

Closes #33196 from dongjoon-hyun/SPARK-35996.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 13:47:36 -07:00
Linhong Liu 3c683434fa [SPARK-35686][SQL] Not allow using auto-generated alias when creating view
### What changes were proposed in this pull request?
As described in  #32831, Spark has compatible issues when querying a view created by an
older version. The root cause is that Spark changed the auto-generated alias name. To avoid
this in the future, we could ask the user to specify explicit column names when creating
a view.

### Why are the changes needed?
Avoid compatible issue when querying a view

### Does this PR introduce _any_ user-facing change?
Yes. User will get error when running query below after this change
```
CREATE OR REPLACE VIEW v AS SELECT CAST(t.a AS INT), to_date(t.b, 'yyyyMMdd') FROM t
```

### How was this patch tested?
not yet

Closes #32832 from linhongliu-db/SPARK-35686-no-auto-alias.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 12:47:38 +00:00
Vinod KC 4dabba8f76 [SPARK-35747][CORE] Avoid printing full Exception stack trace, if Hbase/Kafka/Hive services are not running in a secure cluster
### What changes were proposed in this pull request?
In a secure Yarn cluster, even though HBase or Kafka, or Hive services are not used in the user application, yarn client unnecessarily trying to generate  Delegations token from these services. This will add additional delays while submitting spark application in a yarn cluster

 Also during HBase delegation token generation step in the application submit stage,  HBaseDelegationTokenProvider prints a full Exception Stack trace and it causes a noisy warning.
 Apart from printing exception stack trace, Application submission taking more time as it retries connection to HBase master multiple times before it gives up. So, if HBase is not used in the user Applications, it is better to suggest User disable HBase Delegation Token generation.

 This PR aims to avoid printing full Exception Stack by just printing just Exception name and also add a suggestion message to disable `Delegation Token generation` if service is not used in the Spark Application.

 eg: `If HBase is not used, set spark.security.credentials.hbase.enabled to false`

### Why are the changes needed?

To avoid printing full Exception stack trace in WARN log
#### Before the fix
----------------
```
spark-shell --master yarn
.......
.......
21/06/12 14:29:41 WARN security.HBaseDelegationTokenProvider: Failed to get token from service hbase
java.lang.reflect.InvocationTargetException
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at org.apache.spark.deploy.security.HBaseDelegationTokenProvider.obtainDelegationTokensWithHBaseConn(HBaseDelegationT
okenProvider.scala:93)
        at org.apache.spark.deploy.security.HBaseDelegationTokenProvider.obtainDelegationTokens(HBaseDelegationTokenProvider.
scala:60)
        at org.apache.spark.deploy.security.HadoopDelegationTokenManager$$anonfun$6.apply(HadoopDelegationTokenManager.scala:
166)
        at org.apache.spark.deploy.security.HadoopDelegationTokenManager$$anonfun$6.apply(HadoopDelegationTokenManager.scala:
164)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.Iterator$class.foreach(Iterator.scala:891)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
        at scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
        at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
        at scala.collection.AbstractTraversable.flatMap(Traversable.scala:104)
        at org.apache.spark.deploy.security.HadoopDelegationTokenManager.obtainDelegationTokens(HadoopDelegationTokenManager.
scala:164)
```

#### After  the fix
------------
```
 spark-shell --master yarn

Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
21/06/13 02:10:02 WARN security.HBaseDelegationTokenProvider: Failed to get token from service hbase due to  java.lang.reflect.InvocationTargetException Retrying to fetch HBase security token with hbase connection parameter.
21/06/13 02:10:40 WARN security.HBaseDelegationTokenProvider: Failed to get token from service hbase java.lang.reflect.InvocationTargetException. If HBase is not used, set spark.security.credentials.hbase.enabled to false
21/06/13 02:10:47 WARN cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!
```
### Does this PR introduce _any_ user-facing change?

Yes, in the log, it avoids printing full Exception stack trace.
Instread prints this.
**WARN security.HBaseDelegationTokenProvider: Failed to get token from service hbase java.lang.reflect.InvocationTargetException. If HBase is not used, set spark.security.credentials.hbase.enabled to false**

### How was this patch tested?

Tested manually as it can be verified only in a secure cluster

Closes #32894 from vinodkc/br_fix_Hbase_DT_Exception_stack_printing.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-23 23:12:02 -07:00
Angerszhuuuu 077cf2acdb [SPARK-35733][SQL][TESTS] Check all day-time interval types in HiveInspectors tests
### What changes were proposed in this pull request?
Check all day-time interval types in HiveInspectors tests.

### Why are the changes needed?
New tests should improve test coverage for day-time interval types.

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

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

Closes #33036 from AngersZhuuuu/SPARK-35733.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 19:20:51 +03:00
Gengliang Wang 6f51e37eb5 [SPARK-35857][SQL] The ANSI flag of Cast should be kept after being copied
### What changes were proposed in this pull request?

Make the ANSI flag part of expression `Cast`'s  parameter list, instead of fetching it from the sessional SQLConf.

### Why are the changes needed?

For Views, it is important to show consistent results even the ANSI configuration is different in the running session. This is why many expressions like 'Add'/'Divide' making the ANSI flag part of its case class parameter list.

We should make it consistent for the expression `Cast`

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

Yes, the `Cast` inside a View always behaves the same, independent of the ANSI model SQL configuration in the current session.

### How was this patch tested?

Existing UT

Closes #33027 from gengliangwang/ansiFlagInCast.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-23 16:52:33 +08:00
Angerszhuuuu df55945804 [SPARK-35772][SQL][TESTS] Check all year-month interval types in HiveInspectors tests
### What changes were proposed in this pull request?
Check all year-month interval types in HiveInspectors tests.

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

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

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

Closes #32970 from AngersZhuuuu/SPARK-35772.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 08:54:07 +03:00
Wenchen Fan a2c1a55b1f [SPARK-35700][SQL][FOLLOWUP] Read schema from ORC files should strip CHAR/VARCHAR types
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/33001 , to provide a more direct fix.

The regression in 3.1 was caused by the fact that we changed the parser and allow the parser to return CHAR/VARCHAR type. We should have replaced CHAR/VARCHAR with STRING before the data type flows into the query engine, however, `OrcUtils` is missed.

When reading ORC files, at the task side we will read the real schema from ORC file metadata, then apply filter pushdown. For some reason, the implementation turns ORC schema to Spark schema before filter pushdown, and this step does not strip CHAR/VARCHAR. Note, for Parquet we use the Parquet schema directly in filter pushdown, and do not this have problem.

This PR proposes to replace the CHAR/VARCHAR with STRING when turning ORC schema to Spark schema.

### Why are the changes needed?

a more directly bug fix

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

no

### How was this patch tested?

existing tests

Closes #33030 from cloud-fan/help.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-22 13:50:49 -07:00
Kent Yao 9f734978d9 [SPARK-35700][SQL] Read char/varchar orc table with created and written by external systems
### What changes were proposed in this pull request?

The char/varchar type should be mapped to orc's string type too, see https://orc.apache.org/docs/types.html

### Why are the changes needed?

fix a regression

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

no

### How was this patch tested?

new tests

Closes #33001 from yaooqinn/SPARK-35700.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-21 19:20:55 -07:00
Peter Toth abf9675a75 [SPARK-35798][SQL] Fix SparkPlan.sqlContext usage
### What changes were proposed in this pull request?
There might be `SparkPlan` nodes where canonicalization on executor side can cause issues. This is a follow-up fix to conversation https://github.com/apache/spark/pull/32885/files#r651019687.

### Why are the changes needed?
To avoid potential NPEs when canonicalization happens on executors.

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

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

Closes #32947 from peter-toth/SPARK-35798-fix-sparkplan.sqlcontext-usage.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 13:49:38 +00:00
Linhong Liu b86a69f026 [SPARK-35792][SQL] View should not capture configs used in RelationConversions
### What changes were proposed in this pull request?
`RelationConversions` is actually an optimization rule while it's executed in the analysis phase.
For view, it's designed to only capture semantic configs, so we should ignore the optimization
configs that will be used in the analysis phase.

This PR also fixes the issue that view resolution will always use the default value for uncaptured config

### Why are the changes needed?
Bugfix

### Does this PR introduce _any_ user-facing change?
Yes, after this PR view resolution will respect the values set in the current session for the below configs
```
"spark.sql.hive.convertMetastoreParquet"
"spark.sql.hive.convertMetastoreOrc"
"spark.sql.hive.convertInsertingPartitionedTable"
"spark.sql.hive.convertMetastoreCtas"
```

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

Closes #32941 from linhongliu-db/SPARK-35792-ignore-convert-configs.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 21:40:53 +08:00
Chao Sun 506ef9aad7 [SPARK-29250][BUILD] Upgrade to Hadoop 3.3.1
### What changes were proposed in this pull request?

This upgrade default Hadoop version from 3.2.1 to 3.3.1. The changes here are simply update the version number and dependency file.

### Why are the changes needed?

Hadoop 3.3.1 just came out, which comes with many client-side improvements such as for S3A/ABFS (20% faster when accessing S3). These are important for users who want to use Spark in a cloud environment.

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

No

### How was this patch tested?

- Existing unit tests in Spark
- Manually tested using my S3 bucket for event log dir:
```
bin/spark-shell \
  -c spark.hadoop.fs.s3a.access.key=$AWS_ACCESS_KEY_ID \
  -c spark.hadoop.fs.s3a.secret.key=$AWS_SECRET_ACCESS_KEY \
  -c spark.eventLog.enabled=true
  -c spark.eventLog.dir=s3a://<my-bucket>
```
- Manually tested against docker-based YARN dev cluster, by running `SparkPi`.

Closes #30135 from sunchao/SPARK-29250.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-16 13:28:07 -07:00
YangJie 87bf6b0ea4 [SPARK-35556][SQL] Remove close HiveClient's SessionState
### What changes were proposed in this pull request?

It will not generate `tmpOutputFile`, `tmpErrOutputFile` and `sessionDirs` since [SPARK-35286](https://issues.apache.org/jira/browse/SPARK-35286). So we can remove `HiveClientImpl.closeState` to avoid these exceptions:
```
java.lang.NoSuchMethodError: org.apache.hadoop.hive.ql.session.SessionState.getTmpErrOutputFile()Ljava/io/File
```

### Why are the changes needed?

1. Avoid incompatible exceptions.
2. Remove useless code.

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

### How was this patch tested?

- Pass the GitHub Action
- Manual test:

Execute

```
mvn clean install -DskipTests -pl sql/hive -am
mvn test -pl sql/hive -DwildcardSuites=org.apache.spark.sql.hive.client.VersionsSuite -Dtest=none
```

**Before**

```
Run completed in 17 minutes, 18 seconds.
Total number of tests run: 867
Suites: completed 2, aborted 0
Tests: succeeded 867, failed 0, canceled 0, ignored 1, pending 0
All tests passed.
15:04:02.407 WARN org.apache.hadoop.hive.metastore.ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
15:04:02.408 WARN org.apache.hadoop.hive.metastore.ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore yangjie010.2.30.21
15:04:02.441 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database default, returning NoSuchObjectException
15:04:03.140 ERROR org.apache.spark.util.Utils: Uncaught exception in thread shutdown-hook-0
java.lang.NoSuchMethodError: org.apache.hadoop.hive.ql.session.SessionState.getTmpErrOutputFile()Ljava/io/File;
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$closeState$1(HiveClientImpl.scala:168)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:312)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:243)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:242)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:292)
	at org.apache.spark.sql.hive.client.HiveClientImpl.closeState(HiveClientImpl.scala:158)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$new$1(HiveClientImpl.scala:175)
	at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1994)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$1(ShutdownHookManager.scala:188)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at scala.util.Try$.apply(Try.scala:213)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(ShutdownHookManager.scala:188)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$2.run(ShutdownHookManager.scala:178)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
15:04:03.141 WARN org.apache.hadoop.util.ShutdownHookManager: ShutdownHook '$anon$2' failed, java.util.concurrent.ExecutionException: java.lang.NoSuchMethodError: org.apache.hadoop.hive.ql.session.SessionState.getTmpErrOutputFile()Ljava/io/File;
java.util.concurrent.ExecutionException: java.lang.NoSuchMethodError: org.apache.hadoop.hive.ql.session.SessionState.getTmpErrOutputFile()Ljava/io/File;
	at java.util.concurrent.FutureTask.report(FutureTask.java:122)
	at java.util.concurrent.FutureTask.get(FutureTask.java:206)
	at org.apache.hadoop.util.ShutdownHookManager.executeShutdown(ShutdownHookManager.java:124)
	at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:95)
Caused by: java.lang.NoSuchMethodError: org.apache.hadoop.hive.ql.session.SessionState.getTmpErrOutputFile()Ljava/io/File;
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$closeState$1(HiveClientImpl.scala:168)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:312)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:243)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:242)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:292)
	at org.apache.spark.sql.hive.client.HiveClientImpl.closeState(HiveClientImpl.scala:158)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$new$1(HiveClientImpl.scala:175)
	at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1994)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$1(ShutdownHookManager.scala:188)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at scala.util.Try$.apply(Try.scala:213)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(ShutdownHookManager.scala:188)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$2.run(ShutdownHookManager.scala:178)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

**After**

```
Run completed in 11 minutes, 41 seconds.
Total number of tests run: 867
Suites: completed 2, aborted 0
Tests: succeeded 867, failed 0, canceled 0, ignored 1, pending 0
All tests passed.
```

Closes #32693 from LuciferYang/SPARK-35556.

Lead-authored-by: YangJie <yangjie01@baidu.com>
Co-authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-06-16 23:30:30 +08:00
Sumeet Gajjar 864ff67746 [SPARK-35429][CORE] Remove commons-httpclient from Hadoop-3.2 profile due to EOL and CVEs
### What changes were proposed in this pull request?

Remove commons-httpclient as a direct dependency for Hadoop-3.2 profile.
Hadoop-2.7 profile distribution still has it, hadoop-client has a compile dependency on commons-httpclient, thus we cannot remove it for Hadoop-2.7 profile.
```
[INFO] +- org.apache.hadoop:hadoop-client:jar:2.7.4:compile
[INFO] |  +- org.apache.hadoop:hadoop-common:jar:2.7.4:compile
[INFO] |  |  +- commons-cli:commons-cli:jar:1.2:compile
[INFO] |  |  +- xmlenc:xmlenc:jar:0.52:compile
[INFO] |  |  +- commons-httpclient:commons-httpclient:jar:3.1:compile
```

### Why are the changes needed?

Spark is pulling in commons-httpclient as a dependency directly. commons-httpclient went EOL years ago and there are most likely CVEs not being reported against it, thus we should remove it.

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

No

### How was this patch tested?

- Existing unittests
- Checked the dependency tree before and after introducing the changes

Before:
```
./build/mvn dependency:tree -Phadoop-3.2 | grep -i "commons-httpclient"
Using `mvn` from path: /usr/bin/mvn
[INFO] +- commons-httpclient:commons-httpclient:jar:3.1:compile
[INFO] |  +- commons-httpclient:commons-httpclient:jar:3.1:provided
```

After
```
./build/mvn dependency:tree | grep -i "commons-httpclient"
Using `mvn` from path: /Users/sumeet.gajjar/cloudera/upstream-spark/build/apache-maven-3.6.3/bin/mvn
```

P.S. Reopening this since [spark upgraded](463daabd5a) its `hive.version` to `2.3.9` which does not have a dependency on `commons-httpclient`.

Closes #32912 from sumeetgajjar/SPARK-35429.

Authored-by: Sumeet Gajjar <sumeetgajjar93@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-15 14:43:30 -07:00
Max Gekk 61ce8f7649 [SPARK-35680][SQL] Add fields to YearMonthIntervalType
### What changes were proposed in this pull request?
Extend `YearMonthIntervalType` to support interval fields. Valid interval field values:
- 0 (YEAR)
- 1 (MONTH)

After the changes, the following year-month interval types are supported:
1. `YearMonthIntervalType(0, 0)` or `YearMonthIntervalType(YEAR, YEAR)`
2. `YearMonthIntervalType(0, 1)` or `YearMonthIntervalType(YEAR, MONTH)`. **It is the default one**.
3. `YearMonthIntervalType(1, 1)` or `YearMonthIntervalType(MONTH, MONTH)`

Closes #32825

### Why are the changes needed?
In the current implementation, Spark supports only `interval year to month` but the SQL standard allows to specify the start and end fields. The changes will allow to follow ANSI SQL standard more precisely.

### Does this PR introduce _any_ user-facing change?
Yes but `YearMonthIntervalType` has not been released yet.

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

Closes #32909 from MaxGekk/add-fields-to-YearMonthIntervalType.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-15 23:08:12 +03:00
Chao Sun 9c7250fa73 [SPARK-35321][SQL] Don't register Hive permanent functions when creating Hive client
### What changes were proposed in this pull request?

Instantiate a new Hive client through `Hive.getWithoutRegisterFns(conf, false)` instead of `Hive.get(conf)`, if `Hive` version is >= '2.3.9' (the built-in version).

### Why are the changes needed?

[HIVE-10319](https://issues.apache.org/jira/browse/HIVE-10319) introduced a new API `get_all_functions` which is only supported in Hive 1.3.0/2.0.0 and up. As result, when Spark 3.x talks to a HMS service of version 1.2 or lower, the following error will occur:
```
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.thrift.TApplicationException: Invalid method name: 'get_all_functions'
        at org.apache.hadoop.hive.ql.metadata.Hive.getAllFunctions(Hive.java:3897)
        at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:248)
        at org.apache.hadoop.hive.ql.metadata.Hive.registerAllFunctionsOnce(Hive.java:231)
        ... 96 more
Caused by: org.apache.thrift.TApplicationException: Invalid method name: 'get_all_functions'
        at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_get_all_functions(ThriftHiveMetastore.java:3845)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_all_functions(ThriftHiveMetastore.java:3833)
```

The `get_all_functions` is called only when `doRegisterAllFns` is set to true:
```java
  private Hive(HiveConf c, boolean doRegisterAllFns) throws HiveException {
    conf = c;
    if (doRegisterAllFns) {
      registerAllFunctionsOnce();
    }
  }
```

what this does is to register all Hive permanent functions defined in HMS in Hive's `FunctionRegistry` class, via iterating through results from `get_all_functions`. To Spark, this seems unnecessary as it loads Hive permanent (not built-in) UDF via directly calling the HMS API, i.e., `get_function`. The `FunctionRegistry` is only used in loading Hive's built-in function that is not supported by Spark. At this time, it only applies to `histogram_numeric`.

[HIVE-21563](https://issues.apache.org/jira/browse/HIVE-21563) introduced a new API `getWithoutRegisterFns` which skips the above registration and is available in Hive 2.3.9. Therefore, Spark should adopt it to avoid the cost.

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

Yes with this fix Spark now should be able to talk to HMS server with Hive 1.2.x and lower.

### How was this patch tested?

Manually started a HMS server of Hive version 1.2.2. Without the PR it failed with the above exception. With the PR the error disappeared and I can successfully perform common operations such as create table, create database, list tables, etc.

Closes #32887 from sunchao/SPARK-35321-new.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-06-12 10:32:30 +08:00
Max Gekk d53831ff5c [SPARK-35704][SQL] Add fields to DayTimeIntervalType
### What changes were proposed in this pull request?
Extend DayTimeIntervalType to support interval fields. Valid interval field values:
- 0 (DAY)
- 1 (HOUR)
- 2 (MINUTE)
- 3 (SECOND)

After the changes, the following day-time interval types are supported:
1. `DayTimeIntervalType(0, 0)` or `DayTimeIntervalType(DAY, DAY)`
2. `DayTimeIntervalType(0, 1)` or `DayTimeIntervalType(DAY, HOUR)`
3. `DayTimeIntervalType(0, 2)` or `DayTimeIntervalType(DAY, MINUTE)`
4. `DayTimeIntervalType(0, 3)` or `DayTimeIntervalType(DAY, SECOND)`. **It is the default one**. The second fraction precision is microseconds.
5. `DayTimeIntervalType(1, 1)` or `DayTimeIntervalType(HOUR, HOUR)`
6. `DayTimeIntervalType(1, 2)` or `DayTimeIntervalType(HOUR, MINUTE)`
7. `DayTimeIntervalType(1, 3)` or `DayTimeIntervalType(HOUR, SECOND)`
8. `DayTimeIntervalType(2, 2)` or `DayTimeIntervalType(MINUTE, MINUTE)`
9. `DayTimeIntervalType(2, 3)` or `DayTimeIntervalType(MINUTE, SECOND)`
10. `DayTimeIntervalType(3, 3)` or `DayTimeIntervalType(SECOND, SECOND)`

### Why are the changes needed?
In the current implementation, Spark supports only `interval day to second` but the SQL standard allows to specify the start and end fields. The changes will allow to follow ANSI SQL standard more precisely.

### Does this PR introduce _any_ user-facing change?
Yes but `DayTimeIntervalType` has not been released yet.

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

Closes #32849 from MaxGekk/day-time-interval-type-units.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-11 16:16:33 +03:00
Yuming Wang 463daabd5a [SPARK-34512][BUILD][SQL] Upgrade built-in Hive to 2.3.9
### What changes were proposed in this pull request?

This pr upgrades built-in Hive to 2.3.9. Hive 2.3.9 changes:
- [HIVE-17155] - findConfFile() in HiveConf.java has some issues with the conf path
- [HIVE-24797] - Disable validate default values when parsing Avro schemas
- [HIVE-24608] - Switch back to get_table in HMS client for Hive 2.3.x
- [HIVE-21200] - Vectorization: date column throwing java.lang.UnsupportedOperationException for parquet
- [HIVE-21563] - Improve Table#getEmptyTable performance by disabling registerAllFunctionsOnce
- [HIVE-19228] - Remove commons-httpclient 3.x usage

### Why are the changes needed?

Fix regression caused by AVRO-2035.

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

No.

### How was this patch tested?

Unit test.

Closes #32750 from wangyum/SPARK-34512.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-10 20:44:35 -07:00
beliefer ebb4858f71 [SPARK-35058][SQL] Group exception messages in hive/client
### What changes were proposed in this pull request?
This PR group exception messages in `sql/hive/src/main/scala/org/apache/spark/sql/hive/client`.

### 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 #32763 from beliefer/SPARK-35058.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-09 08:23:09 +00:00
gengjiaan 8013f985a4 [SPARK-35378][SQL] Eagerly execute commands in QueryExecution instead of caller sides
### What changes were proposed in this pull request?
Currently, Spark eagerly executes commands on the caller side of `QueryExecution`, which is a bit hacky as `QueryExecution` is not aware of it and leads to confusion.

For example, if you run `sql("show tables").collect()`, you will see two queries with identical query plans in the web UI.
![image](https://user-images.githubusercontent.com/3182036/121193729-a72d0480-c8a0-11eb-8b12-379019607ad5.png)
![image](https://user-images.githubusercontent.com/3182036/121193822-bc099800-c8a0-11eb-9d2a-34ab1329e2f7.png)
![image](https://user-images.githubusercontent.com/3182036/121193845-c0ce4c00-c8a0-11eb-96d0-ef604a4dfab0.png)

The first query is triggered at `Dataset.logicalPlan`, which eagerly executes the command.
The second query is triggered at `Dataset.collect`, which is the normal query execution.

From the web UI, it's hard to tell that these two queries are caused by eager command execution.

This PR proposes to move the eager command execution to `QueryExecution`, and turn the command plan to `CommandResult` to indicate that command has been executed already. Now `sql("show tables").collect()` still triggers two queries, but the quey plans are not identical. The second query becomes:
![image](https://user-images.githubusercontent.com/3182036/121194850-b3659180-c8a1-11eb-9abf-2980f84f089d.png)

In addition to the UI improvements, this PR also has other benefits:
1. Simplifies code as caller side no need to worry about eager command execution. `QueryExecution` takes care of it.
2. It helps https://github.com/apache/spark/pull/32442 , where there can be more plan nodes above commands, and we need to replace commands with something like local relation that produces unsafe rows.

### Why are the changes needed?
Explained above.

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

### How was this patch tested?
existing tests

Closes #32513 from beliefer/SPARK-35378.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-09 04:45:44 +00:00
fornaix 878527d9fa [SPARK-35612][SQL] Support LZ4 compression in ORC data source
### What changes were proposed in this pull request?

This PR aims to support LZ4 compression in the ORC data source.

### Why are the changes needed?

Apache ORC supports LZ4 compression, but we cannot set LZ4 compression in the ORC data source

**BEFORE**

```scala
scala> spark.range(10).write.option("compression", "lz4").orc("/tmp/lz4")
java.lang.IllegalArgumentException: Codec [lz4] is not available. Available codecs are uncompressed, lzo, snappy, zlib, none, zstd.
```

**AFTER**

```scala
scala> spark.range(10).write.option("compression", "lz4").orc("/tmp/lz4")
```
```bash
$ orc-tools meta /tmp/lz4
Processing data file file:/tmp/lz4/part-00000-6a244eee-b092-4c79-a977-fb8a69dde2eb-c000.lz4.orc [length: 222]
Structure for file:/tmp/lz4/part-00000-6a244eee-b092-4c79-a977-fb8a69dde2eb-c000.lz4.orc
File Version: 0.12 with ORC_517
Rows: 10
Compression: LZ4
Compression size: 262144
Type: struct<id:bigint>

Stripe Statistics:
  Stripe 1:
    Column 0: count: 10 hasNull: false
    Column 1: count: 10 hasNull: false bytesOnDisk: 7 min: 0 max: 9 sum: 45

File Statistics:
  Column 0: count: 10 hasNull: false
  Column 1: count: 10 hasNull: false bytesOnDisk: 7 min: 0 max: 9 sum: 45

Stripes:
  Stripe: offset: 3 data: 7 rows: 10 tail: 35 index: 35
    Stream: column 0 section ROW_INDEX start: 3 length 11
    Stream: column 1 section ROW_INDEX start: 14 length 24
    Stream: column 1 section DATA start: 38 length 7
    Encoding column 0: DIRECT
    Encoding column 1: DIRECT_V2

File length: 222 bytes
Padding length: 0 bytes
Padding ratio: 0%

User Metadata:
  org.apache.spark.version=3.2.0
```

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

Yes.

### How was this patch tested?

Pass the newly added test case.

Closes #32751 from fornaix/spark-35612.

Authored-by: fornaix <foxnaix@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-03 14:07:26 -07:00
gengjiaan 9f7cdb89f7 [SPARK-35059][SQL] Group exception messages in hive/execution
### What changes were proposed in this pull request?
This PR group exception messages in `sql/hive/src/main/scala/org/apache/spark/sql/hive/execution`.

### 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 #32694 from beliefer/SPARK-35059.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-02 13:06:55 +00:00
Max Gekk a59063d544 [SPARK-35581][SQL] Support special datetime values in typed literals only
### What changes were proposed in this pull request?
In the PR, I propose to support special datetime values introduced by #25708 and by #25716 only in typed literals, and don't recognize them in parsing strings to dates/timestamps. The following string values are supported only in typed timestamp literals:
- `epoch [zoneId]` - `1970-01-01 00:00:00+00 (Unix system time zero)`
- `today [zoneId]` - midnight today.
- `yesterday [zoneId]` - midnight yesterday
- `tomorrow [zoneId]` - midnight tomorrow
- `now` - current query start time.

For example:
```sql
spark-sql> SELECT timestamp 'tomorrow';
2019-09-07 00:00:00
```

Similarly, the following special date values are supported only in typed date literals:
- `epoch [zoneId]` - `1970-01-01`
- `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone`.
- `yesterday [zoneId]` - the current date -1
- `tomorrow [zoneId]` - the current date + 1
- `now` - the date of running the current query. It has the same notion as `today`.

For example:
```sql
spark-sql> SELECT date 'tomorrow' - date 'yesterday';
2
```

### Why are the changes needed?
In the current implementation, Spark supports the special date/timestamp value in any input strings casted to dates/timestamps that leads to the following problems:
- If executors have different system time, the result is inconsistent, and random. Column values depend on where the conversions were performed.
- The special values play the role of distributed non-deterministic functions though users might think of the values as constants.

### Does this PR introduce _any_ user-facing change?
Yes but the probability should be small.

### How was this patch tested?
By running existing test suites:
```
$ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite -- -z interval.sql"
$ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite -- -z date.sql"
$ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite -- -z timestamp.sql"
$ build/sbt "test:testOnly *DateTimeUtilsSuite"
```

Closes #32714 from MaxGekk/remove-datetime-special-values.

Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-01 15:29:05 +03:00
yangjie01 09d039da56 [SPARK-35526][CORE][SQL][ML][MLLIB] Re-Cleanup procedure syntax is deprecated compilation warning in Scala 2.13
### What changes were proposed in this pull request?
After SPARK-29291 and SPARK-33352, there are still some compilation warnings about `procedure syntax is deprecated` as follows:

```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/MapOutputTracker.scala:723: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `registerMergeResult`'s return type
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/MapOutputTracker.scala:748: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `unregisterMergeResult`'s return type
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala:223: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `testSimpleSpillingForAllCodecs`'s return type
[WARNING] [Warn] /spark/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASBenchmark.scala:53: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `runBLASBenchmark`'s return type
[WARNING] [Warn] /spark/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala:110: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `assertEmptyRootPath`'s return type
[WARNING] [Warn] /spark/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:602: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `executeCTASWithNonEmptyLocation`'s return type
```

So the main change of this pr is cleanup these compilation warnings.

### Why are the changes needed?
Eliminate compilation warnings in Scala 2.13 and this change should be compatible with Scala 2.12

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

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

Closes #32669 from LuciferYang/re-clean-procedure-syntax.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-30 16:49:47 -07:00
Kousuke Saruta 50fefc6447 [SPARK-35527][SQL][TESTS] Fix HiveExternalCatalogVersionsSuite to pass with Java 11
### What changes were proposed in this pull request?

This PR fixes `HiveExternalCatalogVersionsSuite`.
With this change, only <major>.<minor> version is set to `spark.sql.hive.metastore.version`.

### Why are the changes needed?

I'm personally checking whether all the tests pass with Java 11 for the current `master` and I found `HiveExternalCatalogVersionsSuite` fails.
The reason is that Spark 3.0.2 and 3.1.1 doesn't accept `2.3.8` as a hive metastore version.

`HiveExternalCatalogVersionsSuite` downloads Spark releases from https://dist.apache.org/repos/dist/release/spark/ and run test for each release. The Spark releases are `3.0.2` and `3.1.1` for the current `master` for now.
e47e615c0e/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala (L239-L259)

With Java 11, the suite run with a hive metastore version which corresponds to the builtin Hive version and it's `2.3.8` for the current `master`.
20750a3f9e/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala (L62-L66)

But `branch-3.0` and `branch-3.1` doesn't accept `2.3.8`, the suite with Java 11 fails.

Another solution would be backporting SPARK-34271 (#31371) but after [a discussion](https://github.com/apache/spark/pull/32668#issuecomment-848435170), we prefer to fix the test,

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

No.

### How was this patch tested?

Existing tests with CI.

Closes #32670 from sarutak/fix-version-suite-for-java11.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-26 17:20:51 +09:00
Linhong Liu af1dba7ca5 [SPARK-35440][SQL] Add function type to ExpressionInfo for UDF
### What changes were proposed in this pull request?
Add the function type, such as "scala_udf", "python_udf", "java_udf", "hive", "built-in" to the `ExpressionInfo` for UDF.

### Why are the changes needed?
Make the `ExpressionInfo` of UDF more meaningful

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

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

Closes #32587 from linhongliu-db/udf-language.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 04:40:53 +00:00
Wenchen Fan b624b7e93f [SPARK-28551][SQL][FOLLOWUP] Use the corrected hadoop conf
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/32411, to fix a mistake and use `sparkSession.sessionState.newHadoopConf` which includes SQL configs instead of `sparkSession.sparkContext.hadoopConfiguration` .

### Why are the changes needed?

fix mistake

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

no

### How was this patch tested?

existing tests

Closes #32618 from cloud-fan/follow1.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-22 10:33:57 +08:00
Vinod KC bdd8e1dbb1 [SPARK-28551][SQL] CTAS with LOCATION should not allow to a non-empty directory
### What changes were proposed in this pull request?

CTAS with location clause acts as an insert overwrite. This can cause problems when there are subdirectories within a location directory.
This causes some users to accidentally wipe out directories with very important data. We should not allow CTAS with location to a non-empty directory.

### Why are the changes needed?

Hive already handled this scenario: HIVE-11319

Steps to reproduce:

```scala
sql("""create external table  `demo_CTAS`( `comment` string) PARTITIONED BY (`col1` string, `col2` string) STORED AS parquet location '/tmp/u1/demo_CTAS'""")
sql("""INSERT OVERWRITE TABLE demo_CTAS partition (col1='1',col2='1') VALUES ('abc')""")
sql("select* from demo_CTAS").show
sql("""create table ctas1 location '/tmp/u2/ctas1' as select * from demo_CTAS""")
sql("select* from ctas1").show
sql("""create table ctas2 location '/tmp/u2' as select * from demo_CTAS""")
```

Before the fix: Both create table operations will succeed. But values in table ctas1 will be replaced by ctas2 accidentally.

After the fix: `create table ctas2...` will throw `AnalysisException`:

```
org.apache.spark.sql.AnalysisException: CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory /tmp/u2 . To allow overwriting the existing non-empty directory, set 'spark.sql.legacy.allowNonEmptyLocationInCTAS' to true.
```

### Does this PR introduce _any_ user-facing change?
Yes, if the location directory is not empty, CTAS with location will throw AnalysisException

```
sql("""create table ctas2 location '/tmp/u2' as select * from demo_CTAS""")
```
```
org.apache.spark.sql.AnalysisException: CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory /tmp/u2 . To allow overwriting the existing non-empty directory, set 'spark.sql.legacy.allowNonEmptyLocationInCTAS' to true.
```

`CREATE TABLE AS SELECT` with non-empty `LOCATION` will throw `AnalysisException`. To restore the behavior before Spark 3.2, need to  set `spark.sql.legacy.allowNonEmptyLocationInCTAS` to `true`. , default value is `false`.
Updated SQL migration guide.

### How was this patch tested?
Test case added in SQLQuerySuite.scala

Closes #32411 from vinodkc/br_fixCTAS_nonempty_dir.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-20 06:13:18 +00:00
Yuzhou Sun a72d05c7e6 [SPARK-35106][CORE][SQL] Avoid failing rename caused by destination directory not exist
### What changes were proposed in this pull request?

1. In HadoopMapReduceCommitProtocol, create parent directory before renaming custom partition path staging files
2. In InMemoryCatalog and HiveExternalCatalog, create new partition directory before renaming old partition path
3. Check return value of FileSystem#rename, if false, throw exception to avoid silent data loss cause by rename failure
4. Change DebugFilesystem#rename behavior to make it match HDFS's behavior (return false without rename when dst parent directory not exist)

### Why are the changes needed?

Depends on FileSystem#rename implementation, when destination directory does not exist, file system may
1. return false without renaming file nor throwing exception (e.g. HDFS), or
2. create destination directory, rename files, and return true (e.g. LocalFileSystem)

In the first case above, renames in HadoopMapReduceCommitProtocol for custom partition path will fail silently if the destination partition path does not exist. Failed renames can happen when
1. dynamicPartitionOverwrite == true, the custom partition path directories are deleted by the job before the rename; or
2. the custom partition path directories do not exist before the job; or
3. something else is wrong when file system handle `rename`

The renames in MemoryCatalog and HiveExternalCatalog for partition renaming also have similar issue.

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

No

### How was this patch tested?

Modified DebugFilesystem#rename, and added new unit tests.

Without the fix in src code, five InsertSuite tests and one AlterTableRenamePartitionSuite test failed:
InsertSuite.SPARK-20236: dynamic partition overwrite with custom partition path (existing test with modified FS)
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
struct<>                   struct<>
![2,1,1]
```

InsertSuite.SPARK-35106: insert overwrite with custom partition path
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
struct<>                   struct<>
![2,1,1]
```

InsertSuite.SPARK-35106: dynamic partition overwrite with custom partition path
```
== Results ==
!== Correct Answer - 2 ==   == Spark Answer - 1 ==
!struct<>                   struct<i:int,part1:int,part2:int>
 [1,1,1]                    [1,1,1]
![1,1,2]
```

InsertSuite.SPARK-35106: Throw exception when rename custom partition paths returns false
```
Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown
```

InsertSuite.SPARK-35106: Throw exception when rename dynamic partition paths returns false
```
Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown
```

AlterTableRenamePartitionSuite.ALTER TABLE .. RENAME PARTITION V1: multi part partition (existing test with modified FS)
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
 struct<>                   struct<>
![3,123,3]
```

Closes #32530 from YuzhouSun/SPARK-35106.

Authored-by: Yuzhou Sun <yuzhosun@amazon.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-19 15:46:27 +08:00
Yuming Wang 520a355516 [SPARK-35286][SQL] Replace SessionState.start with SessionState.setCurrentSessionState
### What changes were proposed in this pull request?

This PR replaces `SessionState.start` with `shim.setCurrentSessionState/SessionState.setCurrentSessionState`.

### Why are the changes needed?

To avoid [SessionState.createSessionDirs](https://github.com/apache/hive/blob/rel/release-2.3.8/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java#L652-L696) creating too many directories and Spark SQL do not need it:
![image](https://user-images.githubusercontent.com/5399861/116766834-28ea7080-aa5f-11eb-85ff-07bcaee444e5.png)

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

No.

### How was this patch tested?

Existing test.

Closes #32410 from wangyum/setCurrentSessionState.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-05-16 18:39:15 +08:00
Dongjoon Hyun e31bef1ed4 Revert "[SPARK-35321][SQL] Don't register Hive permanent functions when creating Hive client"
This reverts commit b4ec9e2304.
2021-05-08 13:01:17 -07:00
Chao Sun b4ec9e2304 [SPARK-35321][SQL] Don't register Hive permanent functions when creating Hive client
### What changes were proposed in this pull request?

Instantiate a new Hive client through `Hive.getWithFastCheck(conf, false)` instead of `Hive.get(conf)`.

### Why are the changes needed?

[HIVE-10319](https://issues.apache.org/jira/browse/HIVE-10319) introduced a new API `get_all_functions` which is only supported in Hive 1.3.0/2.0.0 and up. As result, when Spark 3.x talks to a HMS service of version 1.2 or lower, the following error will occur:
```
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.thrift.TApplicationException: Invalid method name: 'get_all_functions'
        at org.apache.hadoop.hive.ql.metadata.Hive.getAllFunctions(Hive.java:3897)
        at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:248)
        at org.apache.hadoop.hive.ql.metadata.Hive.registerAllFunctionsOnce(Hive.java:231)
        ... 96 more
Caused by: org.apache.thrift.TApplicationException: Invalid method name: 'get_all_functions'
        at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_get_all_functions(ThriftHiveMetastore.java:3845)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_all_functions(ThriftHiveMetastore.java:3833)
```

The `get_all_functions` is called only when `doRegisterAllFns` is set to true:
```java
  private Hive(HiveConf c, boolean doRegisterAllFns) throws HiveException {
    conf = c;
    if (doRegisterAllFns) {
      registerAllFunctionsOnce();
    }
  }
```

what this does is to register all Hive permanent functions defined in HMS in Hive's `FunctionRegistry` class, via iterating through results from `get_all_functions`. To Spark, this seems unnecessary as it loads Hive permanent (not built-in) UDF via directly calling the HMS API, i.e., `get_function`. The `FunctionRegistry` is only used in loading Hive's built-in function that is not supported by Spark. At this time, it only applies to `histogram_numeric`.

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

Yes with this fix Spark now should be able to talk to HMS server with Hive 1.2.x and lower (with HIVE-24608 too)

### How was this patch tested?

Manually started a HMS server of Hive version 1.2.2, with patched Hive 2.3.8 using HIVE-24608. Without the PR it failed with the above exception. With the PR the error disappeared and I can successfully perform common operations such as create table, create database, list tables, etc.

Closes #32446 from sunchao/SPARK-35321.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-07 15:06:04 -07:00