### What changes were proposed in this pull request?
Fix UT according to https://github.com/apache/spark/pull/29966#issuecomment-752830046
Change StructType construct from
```
def inputSchema: StructType = StructType(StructField("inputColumn", LongType) :: Nil)
```
to
```
def inputSchema: StructType = new StructType().add("inputColumn", LongType)
```
The whole udf class is :
```
package org.apache.spark.examples.sql
import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction}
import org.apache.spark.sql.types._
import org.apache.spark.sql.Row
class Spark33084 extends UserDefinedAggregateFunction {
// Data types of input arguments of this aggregate function
def inputSchema: StructType = new StructType().add("inputColumn", LongType)
// Data types of values in the aggregation buffer
def bufferSchema: StructType =
new StructType().add("sum", LongType).add("count", LongType)
// The data type of the returned value
def dataType: DataType = DoubleType
// Whether this function always returns the same output on the identical input
def deterministic: Boolean = true
// Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to
// standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides
// the opportunity to update its values. Note that arrays and maps inside the buffer are still
// immutable.
def initialize(buffer: MutableAggregationBuffer): Unit = {
buffer(0) = 0L
buffer(1) = 0L
}
// Updates the given aggregation buffer `buffer` with new input data from `input`
def update(buffer: MutableAggregationBuffer, input: Row): Unit = {
if (!input.isNullAt(0)) {
buffer(0) = buffer.getLong(0) + input.getLong(0)
buffer(1) = buffer.getLong(1) + 1
}
}
// Merges two aggregation buffers and stores the updated buffer values back to `buffer1`
def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = {
buffer1(0) = buffer1.getLong(0) + buffer2.getLong(0)
buffer1(1) = buffer1.getLong(1) + buffer2.getLong(1)
}
// Calculates the final result
def evaluate(buffer: Row): Double = buffer.getLong(0).toDouble / buffer.getLong(1)
}
```
### Why are the changes needed?
Fix UT for scala 2.13
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existed UT
Closes#30980 from AngersZhuuuu/spark-33084-followup.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
As a follow-up task to SPARK-32958, this patch takes safer approach to only prune columns from JsonToStructs if the parsing option is empty. It is to avoid unexpected behavior change regarding parsing.
This patch also adds a few e2e tests to make sure failfast parsing behavior is not changed.
### Why are the changes needed?
It is to avoid unexpected behavior change regarding parsing.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test.
Closes#30970 from viirya/SPARK-33907-3.2.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
In the `saveAsTable()` and `insertInto()` methods of `DataFrameWriter`, recognize `spark_catalog` as the default session catalog in table names.
### Why are the changes needed?
1. To simplify writing of unified v1 and v2 tests
2. To improve Spark SQL user experience. `insertInto()` should have feature parity with the `INSERT INTO` sql command. Currently, `insertInto()` fails on a table from a namespace in `spark_catalog`:
```scala
scala> sql("CREATE NAMESPACE spark_catalog.ns")
scala> Seq(0).toDF().write.saveAsTable("spark_catalog.ns.tbl")
org.apache.spark.sql.AnalysisException: Couldn't find a catalog to handle the identifier spark_catalog.ns.tbl.
at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:629)
... 47 elided
scala> Seq(0).toDF().write.insertInto("spark_catalog.ns.tbl")
org.apache.spark.sql.AnalysisException: Couldn't find a catalog to handle the identifier spark_catalog.ns.tbl.
at org.apache.spark.sql.DataFrameWriter.insertInto(DataFrameWriter.scala:498)
... 47 elided
```
but `INSERT INTO` succeed:
```sql
spark-sql> create table spark_catalog.ns.tbl (c int);
spark-sql> insert into spark_catalog.ns.tbl select 0;
spark-sql> select * from spark_catalog.ns.tbl;
0
```
### Does this PR introduce _any_ user-facing change?
Yes. After the changes for the example above:
```scala
scala> Seq(0).toDF().write.saveAsTable("spark_catalog.ns.tbl")
scala> Seq(1).toDF().write.insertInto("spark_catalog.ns.tbl")
scala> spark.table("spark_catalog.ns.tbl").show(false)
+-----+
|value|
+-----+
|0 |
|1 |
+-----+
```
### How was this patch tested?
By running the affected test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.ShowPartitionsSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.FileFormatWriterSuite"
```
Closes#30919 from MaxGekk/insert-into-spark_catalog.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Replace `SessionCatalogAndNamespace` by `DatabaseInSessionCatalog` in resolving database name from v1 session catalog.
2. Throw more precise errors from `DatabaseInSessionCatalog`
3. Fix expected error messages in `v1.ShowTablesSuiteBase`
Closes#30947
### Why are the changes needed?
Current error message "multi-part identifier cannot be empty" may confuse users. And this error message is just a consequence of "incorrectly" applied an implicit class. For example, `SHOW TABLES IN spark_catalog`:
1. Spark cuts off `spark_catalog` from namespaces in `SessionCatalogAndNamespace`, so, `ns == Seq.empty` here: 0617dfce7b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala (L365)
2. Then `ns.length != 1` is `true` and Spark tries to raise the exception at 0617dfce7b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala (L367)
3. ... but `ns.quoted` triggers implicit wrapping `Seq.empty` by `MultipartIdentifierHelper`, and hit to the second check `if (parts.isEmpty)` at 156704ba0d/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala (L120-L122)
So, Spark throws the exception at third step instead of `new AnalysisException(s"The database name is not valid: $quoted")` on the second step. And even on the second step, the exception doesn't show actual reason as it is pretty generic.
### Does this PR introduce _any_ user-facing change?
Yes in the case of v1 DDL commands when a database is not specified or nested databases is set.
### How was this patch tested?
By running the affected test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DDLSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowTablesSuite"
```
Closes#30963 from MaxGekk/database-in-session-catalog.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
For same SQL
```
SELECT TRANSFORM(a, b, c, null)
ROW FORMAT DELIMITED
USING 'cat'
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '&'
FROM (select 1 as a, 2 as b, 3 as c) t
```
In hive:
```
hive> SELECT TRANSFORM(a, b, c, null)
> ROW FORMAT DELIMITED
> USING 'cat'
> ROW FORMAT DELIMITED
> FIELDS TERMINATED BY '&'
> FROM (select 1 as a, 2 as b, 3 as c) t;
OK
123\N NULL
Time taken: 14.519 seconds, Fetched: 1 row(s)
hive> packet_write_wait: Connection to 10.191.58.100 port 32200: Broken pipe
```
In Spark
```
Spark master: local[*], Application Id: local-1609225830376
spark-sql> SELECT TRANSFORM(a, b, c, null)
> ROW FORMAT DELIMITED
> USING 'cat'
> ROW FORMAT DELIMITED
> FIELDS TERMINATED BY '&'
> FROM (select 1 as a, 2 as b, 3 as c) t;
1 2 3 null NULL
Time taken: 4.297 seconds, Fetched 1 row(s)
spark-sql>
```
We should keep same. Change default ROW FORMAT FIELD DELIMIT to `\u0001`
In hive default value is '1' to char is '\u0001'
```
bucket_count -1
column.name.delimiter ,
columns
columns.comments
columns.types
file.inputformat org.apache.hadoop.hive.ql.io.NullRowsInputFormat
```
### Why are the changes needed?
Keep same behavior with hive
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UT
Closes#30958 from AngersZhuuuu/SPARK-33930.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
1. Add `renamePartition()` to the `SupportsPartitionManagement`
2. Implement `renamePartition()` in `InMemoryPartitionTable`
3. Add v2 execution node `AlterTableRenamePartitionExec`
4. Resolve the logical node `AlterTableRenamePartition` to `AlterTableRenamePartitionExec` for v2 tables that support `SupportsPartitionManagement`
5. Move v1 tests to the base suite `org.apache.spark.sql.execution.command.AlterTableRenamePartitionSuiteBase` to run them for v2 table catalogs.
### Why are the changes needed?
To have feature parity with Datasource V1.
### Does this PR introduce _any_ user-facing change?
Yes
### How was this patch tested?
By running the unified tests:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite"
```
Closes#30935 from MaxGekk/alter-table-rename-partition-v2.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This patch proposes to do column pruning for CsvToStructs expression if we only require some fields from it.
### Why are the changes needed?
`CsvToStructs` takes a schema parameter used to tell CSV Parser what fields are needed to parse. If `CsvToStructs` is followed by GetStructField. We can prune the schema to only parse certain field.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test
Closes#30912 from viirya/SPARK-32968.
Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
For `InMemoryPartitionTable` used in tests, set empty partition metadata only when a partition doesn't exists.
### Why are the changes needed?
This bug fix is needed to use `INSERT INTO .. PARTITION` in other tests.
### Does this PR introduce _any_ user-facing change?
No. It affects only the v2 table catalog used in tests.
### How was this patch tested?
Added new UT to `DataSourceV2SQLSuite`, and run the affected test suite by:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly org.apache.spark.sql.connector.DataSourceV2SQLSuite"
```
Closes#30952 from MaxGekk/fix-insert-into-partition-v2.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Remove `assert(ns.nonEmpty)` in `ResolveSessionCatalog` for:
- `SHOW TABLES`
- `SHOW TABLE EXTENDED`
- `SHOW VIEWS`
### Why are the changes needed?
Spark SQL shouldn't fail with internal assert failures even for invalid user inputs. For instance:
```sql
spark-sql> show tables in spark_catalog;
20/12/24 11:19:46 ERROR SparkSQLDriver: Failed in [show tables in spark_catalog]
java.lang.AssertionError: assertion failed
at scala.Predef$.assert(Predef.scala:208)
at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:366)
at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:49)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUp$3(AnalysisHelper.scala:90)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:73)
```
### Does this PR introduce _any_ user-facing change?
Yes. After the changes, for the example above:
```sql
spark-sql> show tables in spark_catalog;
Error in query: multi-part identifier cannot be empty.
```
### How was this patch tested?
Added new UT to `v1/ShowTablesSuite`.
Closes#30915 from MaxGekk/remove-assert-ns-nonempty.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
* Remove the explicit AQE disable confs
* Use `AdaptiveSparkPlanHelper` to check plans
* No longer extending `DisableAdaptiveExecutionSuite` for `BucketedReadSuite` but only disable AQE for two certain tests there.
### Why are the changes needed?
Some tests that are fixed in https://github.com/apache/spark/pull/30655 doesn't really require AQE off. Instead, they could use `AdaptiveSparkPlanHelper` to pass when AQE on. It's better to run tests with AQE on since we've turned it on by default.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass all tests and the updated tests.
Closes#30941 from Ngone51/SPARK-33680-follow-up.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This patch proposes to unload inactive state store as soon as possible. The timing of unload inactive state stores, happens when we get to load active state store provider at executors. At the time, state store coordinator will return back the state store provider list including loaded stores that are already loaded by other executors in new batch. Each state store provider in the list will go to unload.
### Why are the changes needed?
Per the discussion at #30770, it makes sense to me we should unload inactive state store asap. Now we run a maintenance task periodically to unload inactive state stores. So there will be some delays between a state store becomes inactive and it is unloaded.
However, we can force Spark to always allocate a state store to same executor, by using task locality configuration. This can reduce the possibility to have inactive state store.
Normally, with locality configuration, we might not able to see inactive state store generally. There is still chance an executor can be failed and reallocated, but in this case, inactive state store is also lost too. So it is not an issue.
Making driver-executor bi-directional for unloading inactive state store looks non-trivial, and seems to me, it is not worth, after considering what we can do with locality.
This proposes a simpler but effective approach. We can check if loaded state store is already loaded at other executor during reporting active state store to the coordinator. If so, it means the loaded store is inactive now, and it is going to be unload by the next maintenance task. Then we unload that store immediately.
How do we make sure the loaded state store in previous batch is loaded at other executor in this batch before reporting in this executor? With task locality and preferred location, once an executor is ready to be scheduled, Spark should assign the state store provider previously loaded at the executor. So when this executor gets a new assignment other than previously loaded state store, it means the previously loaded one is already assigned to other executor.
There is still a delay between the state store is loaded at other executor, and unloading it when reporting active state store at this executor. But it should be minimized now. And there won't be multiple state store belonging to same operator are loaded at the same time at one single executor, because once the executor reports any active store, it will unload all inactive stores. This should not be an issue IMHO.
This is a minimal change to unload inactive state store asap without significant change.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test.
Closes#30827 from viirya/SPARK-33827.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
Add comments for the unified datasource tests, describe what kind of tests they contain, and put refs to other test suits.
### Why are the changes needed?
To improve code maintenance.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running `./dev/scalastyle`.
Closes#30929 from MaxGekk/doc-unified-tests.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
After CTAS / CREATE TABLE LIKE / CVAS/ alter table add columns, the target tables will display string instead of char/varchar
### Why are the changes needed?
bugfix
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
new tests
Closes#30918 from yaooqinn/SPARK-33901.
Lead-authored-by: Kent Yao <yao@apache.org>
Co-authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
There are total 15 compilation warnings about `Unicode escapes in triple quoted strings are deprecated` in Spark code now:
```
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2930: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2931: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2932: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2933: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2934: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2935: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2936: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/core/src/main/scala/org/apache/spark/util/Utils.scala:2937: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala:82: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala:32: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala:79: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala:97: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala:101: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala:76: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
[WARNING] /spark-source/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala:83: Unicode escapes in triple quoted strings are deprecated, use the literal character instead
```
This pr try to fix these warnnings.
### Why are the changes needed?
Cleanup compilation warnings about `Unicode escapes in triple quoted strings are deprecated`
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30926 from LuciferYang/SPARK-33801.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This pr mainly adds a comment for the 'rowgroupoffsets! = null' branch in `SpecificParquetRecordReaderBase.init(InputSplit, TaskAttemptContext)` to indicate that spark read parquet process will not enter this branch after SPARK-13883 and SPARK-13989. It is not deleted because PARQUET-131 wants to move `SpecificParquetRecordReaderBase` into the parquet-mr project.
### Why are the changes needed?
Add a useful comment.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30484 from LuciferYang/SPARK-33532.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Similar to SPARK-33441, this pr add `unused-import` check to Maven compilation process. After this pr `unused-import` will trigger Maven compilation error.
For Scala 2.13 profile, this pr also left TODO(SPARK-33499) similar to SPARK-33441 because `scala.language.higherKinds` no longer needs to be imported explicitly since Scala 2.13.1
### Why are the changes needed?
Let Maven build also check for unused imports as compilation error.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
- Pass the Jenkins or GitHub Action
- Local manual test:add an unused import intentionally to trigger maven compilation error.
Closes#30784 from LuciferYang/SPARK-33560.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
[The PySpark documentation](https://spark.apache.org/docs/3.0.1/api/python/pyspark.sql.html#pyspark.sql.DataFrame.join) says "Must be one of: inner, cross, outer, full, fullouter, full_outer, left, leftouter, left_outer, right, rightouter, right_outer, semi, leftsemi, left_semi, anti, leftanti and left_anti."
However, I get the following error when I set the cross option.
```
scala> val df1 = spark.createDataFrame(Seq((1,"a"),(2,"b")))
df1: org.apache.spark.sql.DataFrame = [_1: int, _2: string]
scala> val df2 = spark.createDataFrame(Seq((1,"A"),(2,"B"), (3, "C")))
df2: org.apache.spark.sql.DataFrame = [_1: int, _2: string]
scala> df1.join(right = df2, usingColumns = Seq("_1"), joinType = "cross").show()
java.lang.IllegalArgumentException: requirement failed: Unsupported using join type Cross
at scala.Predef$.require(Predef.scala:281)
at org.apache.spark.sql.catalyst.plans.UsingJoin.<init>(joinTypes.scala:106)
at org.apache.spark.sql.Dataset.join(Dataset.scala:1025)
... 53 elided
```
### Why are the changes needed?
The documentation says cross option can be set, but when I try to set it, I get an java.lang.IllegalArgumentException.
### Does this PR introduce _any_ user-facing change?
Accepting this PR fix will behave the same as the documentation.
### How was this patch tested?
There is already a test for [JoinTypes](1b9fd67904/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala), but I can't find a test for the join option itself.
Closes#30803 from kozakana/allow_cross_option.
Authored-by: kozakana <goki727@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Support add jar with ivy path
### Why are the changes needed?
Since submit app can support ivy, add jar we can also support ivy now.
### Does this PR introduce _any_ user-facing change?
User can add jar with sql like
```
add jar ivy:://group:artifict:version?exclude=xxx,xxx&transitive=true
add jar ivy:://group:artifict:version?exclude=xxx,xxx&transitive=false
```
core api
```
sparkContext.addJar("ivy:://group:artifict:version?exclude=xxx,xxx&transitive=true")
sparkContext.addJar("ivy:://group:artifict:version?exclude=xxx,xxx&transitive=false")
```
#### Doc Update snapshot
![image](https://user-images.githubusercontent.com/46485123/101227738-de451200-36d3-11eb-813d-78a8b879da4f.png)
### How was this patch tested?
Added UT
Closes#29966 from AngersZhuuuu/support-add-jar-ivy.
Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Display char/varchar in
- DESC table
- DESC column
- SHOW CREATE TABLE
### Why are the changes needed?
show the correct definition for users
### Does this PR introduce _any_ user-facing change?
yes, char/varchar column's will print char/varchar instead of string
### How was this patch tested?
new tests
Closes#30908 from yaooqinn/SPARK-33892.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add tests to check handling `null` and `''` (empty string) as partition values in commands `SHOW PARTITIONS`, `ALTER TABLE .. ADD PARTITION`, `ALTER TABLE .. DROP PARTITION`.
### Why are the changes needed?
To improve test coverage.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running the modified test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.ShowPartitionsSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *.AlterTableDropPartitionSuite"
```
Closes#30893 from MaxGekk/partition-value-empty-string.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This pr simplify conditional in predicate, after this change we can push down the filter to datasource:
Expression | After simplify
-- | --
IF(cond, trueVal, false) | AND(cond, trueVal)
IF(cond, trueVal, true) | OR(NOT(cond), trueVal)
IF(cond, false, falseVal) | AND(NOT(cond), elseVal)
IF(cond, true, falseVal) | OR(cond, elseVal)
CASE WHEN cond THEN trueVal ELSE false END | AND(cond, trueVal)
CASE WHEN cond THEN trueVal END | AND(cond, trueVal)
CASE WHEN cond THEN trueVal ELSE null END | AND(cond, trueVal)
CASE WHEN cond THEN trueVal ELSE true END | OR(NOT(cond), trueVal)
CASE WHEN cond THEN false ELSE elseVal END | AND(NOT(cond), elseVal)
CASE WHEN cond THEN false END | false
CASE WHEN cond THEN true ELSE elseVal END | OR(cond, elseVal)
CASE WHEN cond THEN true END | cond
### 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#30865 from wangyum/SPARK-33861.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Follow up work for #30521, document the following behaviors in the API doc:
- Figure out the effects when configurations are (provider/partitionBy) conflicting with the existing table.
- Document the lack of functionality on creating a v2 table, and guide that the users should ensure a table is created in prior to avoid the behavior unintended/insufficient table is being created.
### Why are the changes needed?
We didn't have full support for the V2 table created in the API now. (TODO SPARK-33638)
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Document only.
Closes#30885 from xuanyuanking/SPARK-33659.
Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This is a retry of #30177.
This is not a complete fix, but it would take long time to complete (#30242).
As discussed offline, at least using `ContextAwareIterator` should be helpful enough for many cases.
As the Python evaluation consumes the parent iterator in a separate thread, it could consume more data from the parent even after the task ends and the parent is closed. Thus, we should use `ContextAwareIterator` to stop consuming after the task ends.
### Why are the changes needed?
Python/Pandas UDF right after off-heap vectorized reader could cause executor crash.
E.g.,:
```py
spark.range(0, 100000, 1, 1).write.parquet(path)
spark.conf.set("spark.sql.columnVector.offheap.enabled", True)
def f(x):
return 0
fUdf = udf(f, LongType())
spark.read.parquet(path).select(fUdf('id')).head()
```
This is because, the Python evaluation consumes the parent iterator in a separate thread and it consumes more data from the parent even after the task ends and the parent is closed. If an off-heap column vector exists in the parent iterator, it could cause segmentation fault which crashes the executor.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added tests, and manually.
Closes#30899 from ueshin/issues/SPARK-33277/context_aware_iterator.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
1. Enhance `ReplaceNullWithFalseInPredicate` to replace None of elseValue inside `CaseWhen` with `FalseLiteral` if all branches are `FalseLiteral` . The use case is:
```sql
create table t1 using parquet as select id from range(10);
explain select id from t1 where (CASE WHEN id = 1 THEN 'a' WHEN id = 3 THEN 'b' end) = 'c';
```
Before this pr:
```
== Physical Plan ==
*(1) Filter CASE WHEN (id#1L = 1) THEN false WHEN (id#1L = 3) THEN false END
+- *(1) ColumnarToRow
+- FileScan parquet default.t1[id#1L] Batched: true, DataFilters: [CASE WHEN (id#1L = 1) THEN false WHEN (id#1L = 3) THEN false END], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/opensource/spark/spark-warehouse/org.apache.spark.sql.DataF..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:bigint>
```
After this pr:
```
== Physical Plan ==
LocalTableScan <empty>, [id#1L]
```
2. Enhance `SimplifyConditionals` if elseValue is None and all outputs are null.
### 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#30852 from wangyum/SPARK-33847.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
At `ShowPartitionsExec.run()`, check that a row returned by `listPartitionIdentifiers()` contains a `null` field, and convert it to `"null"`.
### Why are the changes needed?
Because `SHOW PARTITIONS` throws NPE on V2 table with `null` partition values.
### Does this PR introduce _any_ user-facing change?
Yes
### How was this patch tested?
Added new UT to `v2.ShowPartitionsSuite`.
Closes#30904 from MaxGekk/fix-npe-show-partitions.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Move the `ALTER TABLE .. RENAME PARTITION` parsing tests to `AlterTableRenamePartitionParserSuite`
2. Place the v1 tests for `ALTER TABLE .. RENAME PARTITION` from `DDLSuite` to `v1.AlterTableRenamePartitionSuite` and v2 tests from `AlterTablePartitionV2SQLSuite` to `v2.AlterTableRenamePartitionSuite`, so, the tests will run for V1, Hive V1 and V2 DS.
### Why are the changes needed?
- The unification will allow to run common `ALTER TABLE .. RENAME PARTITION` tests for both DSv1 and Hive DSv1, DSv2
- We can detect missing features and differences between DSv1 and DSv2 implementations.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionParserSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite"
```
Closes#30863 from MaxGekk/unify-rename-partition-tests.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR aims to override maxRows method in these follow `LogicalPlan`:
* `ReturnAnswer`
* `Join`
* `Range`
* `Sample`
* `RepartitionOperation`
* `Deduplicate`
* `LocalRelation`
* `Window`
### Why are the changes needed?
1. Logically, we know the max rows info with these `LogicalPlan`.
2. Before this PR, we already have some max rows with `LogicalPlan`, so we can eliminate limit with more case if we expand more.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Add test.
Closes#30443 from ulysses-you/SPARK-33497.
Lead-authored-by: ulysses-you <youxiduo@weidian.com>
Co-authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Add new methods `purgePartition()`/`purgePartitions()` to the interfaces `SupportsPartitionManagement`/`SupportsAtomicPartitionManagement`.
2. Default implementation of new methods throw the exception `UnsupportedOperationException`.
3. Add tests for new methods to `SupportsPartitionManagementSuite`/`SupportsAtomicPartitionManagementSuite`.
4. Add `ALTER TABLE .. DROP PARTITION` tests for DS v1 and v2.
Closes#30776Closes#30821
### Why are the changes needed?
Currently, the `PURGE` option that user can set in `ALTER TABLE .. DROP PARTITION` is completely ignored. We should pass this flag to the catalog implementation, so, the catalog should decide how to handle the flag.
### Does this PR introduce _any_ user-facing change?
The changes can impact on behavior of `ALTER TABLE .. DROP PARTITION` for v2 tables.
### How was this patch tested?
By running the affected test suites, for instance:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```
Closes#30886 from MaxGekk/purge-partition.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
```sql
spark-sql> select * from t10 where c0='abcd';
20/12/22 15:43:38 ERROR SparkSQLDriver: Failed in [select * from t10 where c0='abcd']
scala.MatchError: CharType(10) (of class org.apache.spark.sql.types.CharType)
at org.apache.spark.sql.catalyst.expressions.CastBase.cast(Cast.scala:815)
at org.apache.spark.sql.catalyst.expressions.CastBase.cast$lzycompute(Cast.scala:842)
at org.apache.spark.sql.catalyst.expressions.CastBase.cast(Cast.scala:842)
at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeEval(Cast.scala:844)
at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:476)
at org.apache.spark.sql.catalyst.catalog.CatalogTablePartition.$anonfun$toRow$2(interface.scala:164)
at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
at scala.collection.Iterator.foreach(Iterator.scala:941)
at scala.collection.Iterator.foreach$(Iterator.scala:941)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
at scala.collection.IterableLike.foreach(IterableLike.scala:74)
at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
at org.apache.spark.sql.types.StructType.foreach(StructType.scala:102)
at scala.collection.TraversableLike.map(TraversableLike.scala:238)
at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
at org.apache.spark.sql.types.StructType.map(StructType.scala:102)
at org.apache.spark.sql.catalyst.catalog.CatalogTablePartition.toRow(interface.scala:158)
at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.$anonfun$prunePartitionsByFilter$3(ExternalCatalogUtils.scala:157)
at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.$anonfun$prunePartitionsByFilter$3$adapted(ExternalCatalogUtils.scala:156)
```
c0 is a partition column, it fails in the partition pruning rule
In this PR, we relace char/varchar w/ string type before the CAST happends
### Why are the changes needed?
bugfix, see the case above
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
yes, new tests
Closes#30887 from yaooqinn/SPARK-33879.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR intends to fix flaky GitHub Actions (GA) tests below in `transform.sql` (this flakiness does not seem to happen in the Jenkins tests):
- https://github.com/apache/spark/runs/1592987501
- https://github.com/apache/spark/runs/1593196242
- https://github.com/apache/spark/runs/1595496305
- https://github.com/apache/spark/runs/1596309555
This is because the error message is different between test runs in GA (the error message seems to be truncated indeterministically) ,e.g.,
```
# https://github.com/apache/spark/runs/1592987501
Expected "...h status 127. Error:[ /bin/bash: some_non_existent_command: command not found]", but got "...h status 127. Error:[]" Result did not match for query #2
# https://github.com/apache/spark/runs/1593196242
Expected "...istent_command: comm[and not found]", but got "...istent_command: comm[]" Result did not match for query #2
```
The root cause of this indeterministic behaviour happening only in GA is not clear though, this test throws SparkException consistently even in GA. So, this PR proposes to make the test just check if it will be thrown when running it.
This PR comes from the dongjoon-hyun comment: https://github.com/apache/spark/pull/29414/files#r547414513
### Why are the changes needed?
Bugfix.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added tests.
Closes#30896 from maropu/SPARK-32106-FOLLOWUP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/30267
Inspired by https://github.com/apache/spark/pull/30886, it's better to have 2 methods `def dropTable` and `def purgeTable`, than `def dropTable(ident)` and `def dropTable(ident, purge)`.
### Why are the changes needed?
1. make the APIs orthogonal. Previously, `def dropTable(ident, purge)` calls `def dropTable(ident)` and is a superset.
2. simplifies the catalog implementation a little bit. Now the `if (purge) ... else ...` check is done at the Spark side.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
existing tests
Closes#30890 from cloud-fan/purgeTable.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Add support for Java Enums (`java.lang.Enum`) from the Scala typed Dataset APIs. This involves adding an implicit for `Encoder` creation in `SQLImplicits`, and updating `ScalaReflection` to handle Java Enums on the serialization and deserialization pathways.
Enums are mapped to a `StringType` which is just the name of the Enum value.
### Why are the changes needed?
In [SPARK-21255](https://issues.apache.org/jira/browse/SPARK-21255), support for (de)serialization of Java Enums was added, but only when called from Java code. It is common for Scala code to rely on Java libraries that are out of control of the Scala developer. Today, if there is a dependency on some Java code which defines an Enum, it would be necessary to define a corresponding Scala class. This change brings closer feature parity between Scala and Java APIs.
### Does this PR introduce _any_ user-facing change?
Yes, previously something like:
```
val ds = Seq(MyJavaEnum.VALUE1, MyJavaEnum.VALUE2).toDS
// or
val ds = Seq(CaseClass(MyJavaEnum.VALUE1), CaseClass(MyJavaEnum.VALUE2)).toDS
```
would fail. Now, it will succeed.
### How was this patch tested?
Additional unit tests are added in `DatasetSuite`. Tests include validating top-level enums, enums inside of case classes, enums inside of arrays, and validating that the Enum is stored as the expected string.
Closes#30877 from xkrogen/xkrogen-SPARK-23862-scalareflection-java-enums.
Lead-authored-by: Erik Krogen <xkrogen@apache.org>
Co-authored-by: Fangshi Li <fli@linkedin.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR adds the length check to the existing ApplyCharPadding rule. Tables will have external locations when users execute
SET LOCATION or CREATE TABLE ... LOCATION. If the location contains over length values we should FAIL ON READ.
### Why are the changes needed?
```sql
spark-sql> INSERT INTO t2 VALUES ('1', 'b12345');
Time taken: 0.141 seconds
spark-sql> alter table t set location '/tmp/hive_one/t2';
Time taken: 0.095 seconds
spark-sql> select * from t;
1 b1234
```
the above case should fail rather than implicitly applying truncation
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
new tests
Closes#30882 from yaooqinn/SPARK-33876.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Recognize `spark_catalog` as the default session catalog in the checks of `TestHiveQueryExecution`.
2. Move v2 and v1 in-memory catalog test `"SPARK-33305: DROP TABLE should also invalidate cache"` to the common trait `command/DropTableSuiteBase`, and run it with v1 Hive external catalog.
### Why are the changes needed?
To run In-memory catalog tests in Hive catalog.
### Does this PR introduce _any_ user-facing change?
No, the changes influence only on tests.
### How was this patch tested?
By running the affected test suites for `DROP TABLE`:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DropTableSuite"
```
Closes#30883 from MaxGekk/fix-spark_catalog-hive-tests.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR tries to rename `dataSourceRewriteRules` into something more generic.
### Why are the changes needed?
These changes are needed to address the post-review discussion [here](https://github.com/apache/spark/pull/30558#discussion_r533885837).
### Does this PR introduce _any_ user-facing change?
Yes but the changes haven't been released yet.
### How was this patch tested?
Existing tests.
Closes#30808 from aokolnychyi/spark-33784.
Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR adds logic to build logical writes introduced in SPARK-33779.
Note: This PR contains a subset of changes discussed in PR #29066.
### Why are the changes needed?
These changes are the next step as discussed in the [design doc](https://docs.google.com/document/d/1X0NsQSryvNmXBY9kcvfINeYyKC-AahZarUqg3nS1GQs/edit#) for SPARK-23889.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing tests.
Closes#30806 from aokolnychyi/spark-33808.
Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Orc support filter push down optimization, but this optimization will read file meta from external storage even if filters is empty.
This pr add a extra `filters.nonEmpty` when `spark.sql.orc.filterPushdown` is true
### Why are the changes needed?
Orc filters push down operation should only triggered when `filters.nonEmpty` is true
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30663 from LuciferYang/pushdownfilter-when-filter-nonempty.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Verify ALTER TABLE CHANGE COLUMN with Char and Varchar and avoid unexpected change
For v1 table, changing type is not allowed, we fix a regression that uses the replaced string instead of the original char/varchar type when altering char/varchar columns
For v2 table,
char/varchar to string,
char(x) to char(x),
char(x)/varchar(x) to varchar(y) if x <=y are valid cases,
other changes are invalid
### Why are the changes needed?
Verify ALTER TABLE CHANGE COLUMN with Char and Varchar and avoid unexpected change
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
new test
Closes#30833 from yaooqinn/SPARK-33834.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
* Implement `SparkScriptTransformationExec` based on `BaseScriptTransformationExec`
* Implement `SparkScriptTransformationWriterThread` based on `BaseScriptTransformationWriterThread` of writing data
* Add rule `SparkScripts` to support convert script LogicalPlan to SparkPlan in Spark SQL (without hive mode)
* Add `SparkScriptTransformationSuite` test spark spec case
* add test in `SQLQueryTestSuite`
And we will close#29085 .
### Why are the changes needed?
Support user use Script Transform without Hive
### Does this PR introduce _any_ user-facing change?
User can use Script Transformation without hive in no serde mode.
Such as :
**default no serde **
```
SELECT TRANSFORM(a, b, c)
USING 'cat' AS (a int, b string, c long)
FROM testData
```
**no serde with spec ROW FORMAT DELIMITED**
```
SELECT TRANSFORM(a, b, c)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '\t'
COLLECTION ITEMS TERMINATED BY '\u0002'
MAP KEYS TERMINATED BY '\u0003'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'null'
USING 'cat' AS (a, b, c)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '\t'
COLLECTION ITEMS TERMINATED BY '\u0004'
MAP KEYS TERMINATED BY '\u0005'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM testData
```
### How was this patch tested?
Added UT
Closes#29414 from AngersZhuuuu/SPARK-32106-MINOR.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
This pr push the `UnaryExpression` into (if / case) branches. The use case is:
```sql
create table t1 using parquet as select id from range(10);
explain select id from t1 where (CASE WHEN id = 1 THEN '1' WHEN id = 3 THEN '2' end) > 3;
```
Before this pr:
```
== Physical Plan ==
*(1) Filter (cast(CASE WHEN (id#1L = 1) THEN 1 WHEN (id#1L = 3) THEN 2 END as int) > 3)
+- *(1) ColumnarToRow
+- FileScan parquet default.t1[id#1L] Batched: true, DataFilters: [(cast(CASE WHEN (id#1L = 1) THEN 1 WHEN (id#1L = 3) THEN 2 END as int) > 3)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/opensource/spark/spark-warehouse/org.apache.spark.sql.DataF..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:bigint>
```
After this pr:
```
== Physical Plan ==
LocalTableScan <empty>, [id#1L]
```
This change can also improve this case:
a78d6ce376/sql/core/src/test/resources/tpcds/q62.sql (L5-L22)
### 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#30853 from wangyum/SPARK-33848.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR proposes to fill missing group tags and re-categorize all the group tags for built-in functions.
New groups below are added in this PR:
- binary_funcs
- bitwise_funcs
- collection_funcs
- predicate_funcs
- conditional_funcs
- conversion_funcs
- csv_funcs
- generator_funcs
- hash_funcs
- lambda_funcs
- math_funcs
- misc_funcs
- string_funcs
- struct_funcs
- xml_funcs
A basic policy to re-categorize functions is that functions in the same file are categorized into the same group. For example, all the functions in `hash.scala` are categorized into `hash_funcs`. But, there are some exceptional/ambiguous cases when categorizing them. Here are some special notes:
- All the aggregate functions are categorized into `agg_funcs`.
- `array_funcs` and `map_funcs` are sub-groups of `collection_funcs`. For example, `array_contains` is used only for arrays, so it is assigned to `array_funcs`. On the other hand, `reverse` is used for both arrays and strings, so it is assigned to `collection_funcs`.
- Some functions logically belong to multiple groups. In this case, these functions are categorized based on the file that they belong to. For example, `schema_of_csv` can be grouped into both `csv_funcs` and `struct_funcs` in terms of input types, but it is assigned to `csv_funcs` because it belongs to the `csvExpressions.scala` file that holds the other CSV-related functions.
- Functions in `nullExpressions.scala`, `complexTypeCreator.scala`, `randomExpressions.scala`, and `regexExpressions.scala` are categorized based on their functionalities. For example:
- `isnull` in `nullExpressions` is assigned to `predicate_funcs` because this is a predicate function.
- `array` in `complexTypeCreator.scala` is assigned to `array_funcs`based on its output type (The other functions in `array_funcs` are categorized based on their input types though).
A category list (after this PR) is as follows (the list below includes the exprs that already have a group tag in the current master):
|group|name|class|
|-----|----|-----|
|agg_funcs|any|org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr|
|agg_funcs|approx_count_distinct|org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus|
|agg_funcs|approx_percentile|org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile|
|agg_funcs|avg|org.apache.spark.sql.catalyst.expressions.aggregate.Average|
|agg_funcs|bit_and|org.apache.spark.sql.catalyst.expressions.aggregate.BitAndAgg|
|agg_funcs|bit_or|org.apache.spark.sql.catalyst.expressions.aggregate.BitOrAgg|
|agg_funcs|bit_xor|org.apache.spark.sql.catalyst.expressions.aggregate.BitXorAgg|
|agg_funcs|bool_and|org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd|
|agg_funcs|bool_or|org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr|
|agg_funcs|collect_list|org.apache.spark.sql.catalyst.expressions.aggregate.CollectList|
|agg_funcs|collect_set|org.apache.spark.sql.catalyst.expressions.aggregate.CollectSet|
|agg_funcs|corr|org.apache.spark.sql.catalyst.expressions.aggregate.Corr|
|agg_funcs|count_if|org.apache.spark.sql.catalyst.expressions.aggregate.CountIf|
|agg_funcs|count_min_sketch|org.apache.spark.sql.catalyst.expressions.aggregate.CountMinSketchAgg|
|agg_funcs|count|org.apache.spark.sql.catalyst.expressions.aggregate.Count|
|agg_funcs|covar_pop|org.apache.spark.sql.catalyst.expressions.aggregate.CovPopulation|
|agg_funcs|covar_samp|org.apache.spark.sql.catalyst.expressions.aggregate.CovSample|
|agg_funcs|cube|org.apache.spark.sql.catalyst.expressions.Cube|
|agg_funcs|every|org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd|
|agg_funcs|first_value|org.apache.spark.sql.catalyst.expressions.aggregate.First|
|agg_funcs|first|org.apache.spark.sql.catalyst.expressions.aggregate.First|
|agg_funcs|grouping_id|org.apache.spark.sql.catalyst.expressions.GroupingID|
|agg_funcs|grouping|org.apache.spark.sql.catalyst.expressions.Grouping|
|agg_funcs|kurtosis|org.apache.spark.sql.catalyst.expressions.aggregate.Kurtosis|
|agg_funcs|last_value|org.apache.spark.sql.catalyst.expressions.aggregate.Last|
|agg_funcs|last|org.apache.spark.sql.catalyst.expressions.aggregate.Last|
|agg_funcs|max_by|org.apache.spark.sql.catalyst.expressions.aggregate.MaxBy|
|agg_funcs|max|org.apache.spark.sql.catalyst.expressions.aggregate.Max|
|agg_funcs|mean|org.apache.spark.sql.catalyst.expressions.aggregate.Average|
|agg_funcs|min_by|org.apache.spark.sql.catalyst.expressions.aggregate.MinBy|
|agg_funcs|min|org.apache.spark.sql.catalyst.expressions.aggregate.Min|
|agg_funcs|percentile_approx|org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile|
|agg_funcs|percentile|org.apache.spark.sql.catalyst.expressions.aggregate.Percentile|
|agg_funcs|rollup|org.apache.spark.sql.catalyst.expressions.Rollup|
|agg_funcs|skewness|org.apache.spark.sql.catalyst.expressions.aggregate.Skewness|
|agg_funcs|some|org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr|
|agg_funcs|stddev_pop|org.apache.spark.sql.catalyst.expressions.aggregate.StddevPop|
|agg_funcs|stddev_samp|org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp|
|agg_funcs|stddev|org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp|
|agg_funcs|std|org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp|
|agg_funcs|sum|org.apache.spark.sql.catalyst.expressions.aggregate.Sum|
|agg_funcs|var_pop|org.apache.spark.sql.catalyst.expressions.aggregate.VariancePop|
|agg_funcs|var_samp|org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp|
|agg_funcs|variance|org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp|
|array_funcs|array_contains|org.apache.spark.sql.catalyst.expressions.ArrayContains|
|array_funcs|array_distinct|org.apache.spark.sql.catalyst.expressions.ArrayDistinct|
|array_funcs|array_except|org.apache.spark.sql.catalyst.expressions.ArrayExcept|
|array_funcs|array_intersect|org.apache.spark.sql.catalyst.expressions.ArrayIntersect|
|array_funcs|array_join|org.apache.spark.sql.catalyst.expressions.ArrayJoin|
|array_funcs|array_max|org.apache.spark.sql.catalyst.expressions.ArrayMax|
|array_funcs|array_min|org.apache.spark.sql.catalyst.expressions.ArrayMin|
|array_funcs|array_position|org.apache.spark.sql.catalyst.expressions.ArrayPosition|
|array_funcs|array_remove|org.apache.spark.sql.catalyst.expressions.ArrayRemove|
|array_funcs|array_repeat|org.apache.spark.sql.catalyst.expressions.ArrayRepeat|
|array_funcs|array_union|org.apache.spark.sql.catalyst.expressions.ArrayUnion|
|array_funcs|arrays_overlap|org.apache.spark.sql.catalyst.expressions.ArraysOverlap|
|array_funcs|arrays_zip|org.apache.spark.sql.catalyst.expressions.ArraysZip|
|array_funcs|array|org.apache.spark.sql.catalyst.expressions.CreateArray|
|array_funcs|flatten|org.apache.spark.sql.catalyst.expressions.Flatten|
|array_funcs|sequence|org.apache.spark.sql.catalyst.expressions.Sequence|
|array_funcs|shuffle|org.apache.spark.sql.catalyst.expressions.Shuffle|
|array_funcs|slice|org.apache.spark.sql.catalyst.expressions.Slice|
|array_funcs|sort_array|org.apache.spark.sql.catalyst.expressions.SortArray|
|bitwise_funcs|&|org.apache.spark.sql.catalyst.expressions.BitwiseAnd|
|bitwise_funcs|^|org.apache.spark.sql.catalyst.expressions.BitwiseXor|
|bitwise_funcs|bit_count|org.apache.spark.sql.catalyst.expressions.BitwiseCount|
|bitwise_funcs|shiftrightunsigned|org.apache.spark.sql.catalyst.expressions.ShiftRightUnsigned|
|bitwise_funcs|shiftright|org.apache.spark.sql.catalyst.expressions.ShiftRight|
|bitwise_funcs|~|org.apache.spark.sql.catalyst.expressions.BitwiseNot|
|collection_funcs|cardinality|org.apache.spark.sql.catalyst.expressions.Size|
|collection_funcs|concat|org.apache.spark.sql.catalyst.expressions.Concat|
|collection_funcs|reverse|org.apache.spark.sql.catalyst.expressions.Reverse|
|collection_funcs|size|org.apache.spark.sql.catalyst.expressions.Size|
|conditional_funcs|coalesce|org.apache.spark.sql.catalyst.expressions.Coalesce|
|conditional_funcs|ifnull|org.apache.spark.sql.catalyst.expressions.IfNull|
|conditional_funcs|if|org.apache.spark.sql.catalyst.expressions.If|
|conditional_funcs|nanvl|org.apache.spark.sql.catalyst.expressions.NaNvl|
|conditional_funcs|nullif|org.apache.spark.sql.catalyst.expressions.NullIf|
|conditional_funcs|nvl2|org.apache.spark.sql.catalyst.expressions.Nvl2|
|conditional_funcs|nvl|org.apache.spark.sql.catalyst.expressions.Nvl|
|conditional_funcs|when|org.apache.spark.sql.catalyst.expressions.CaseWhen|
|conversion_funcs|bigint|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|binary|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|boolean|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|cast|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|date|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|decimal|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|double|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|float|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|int|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|smallint|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|string|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|timestamp|org.apache.spark.sql.catalyst.expressions.Cast|
|conversion_funcs|tinyint|org.apache.spark.sql.catalyst.expressions.Cast|
|csv_funcs|from_csv|org.apache.spark.sql.catalyst.expressions.CsvToStructs|
|csv_funcs|schema_of_csv|org.apache.spark.sql.catalyst.expressions.SchemaOfCsv|
|csv_funcs|to_csv|org.apache.spark.sql.catalyst.expressions.StructsToCsv|
|datetime_funcs|add_months|org.apache.spark.sql.catalyst.expressions.AddMonths|
|datetime_funcs|current_date|org.apache.spark.sql.catalyst.expressions.CurrentDate|
|datetime_funcs|current_timestamp|org.apache.spark.sql.catalyst.expressions.CurrentTimestamp|
|datetime_funcs|current_timezone|org.apache.spark.sql.catalyst.expressions.CurrentTimeZone|
|datetime_funcs|date_add|org.apache.spark.sql.catalyst.expressions.DateAdd|
|datetime_funcs|date_format|org.apache.spark.sql.catalyst.expressions.DateFormatClass|
|datetime_funcs|date_from_unix_date|org.apache.spark.sql.catalyst.expressions.DateFromUnixDate|
|datetime_funcs|date_part|org.apache.spark.sql.catalyst.expressions.DatePart|
|datetime_funcs|date_sub|org.apache.spark.sql.catalyst.expressions.DateSub|
|datetime_funcs|date_trunc|org.apache.spark.sql.catalyst.expressions.TruncTimestamp|
|datetime_funcs|datediff|org.apache.spark.sql.catalyst.expressions.DateDiff|
|datetime_funcs|dayofmonth|org.apache.spark.sql.catalyst.expressions.DayOfMonth|
|datetime_funcs|dayofweek|org.apache.spark.sql.catalyst.expressions.DayOfWeek|
|datetime_funcs|dayofyear|org.apache.spark.sql.catalyst.expressions.DayOfYear|
|datetime_funcs|day|org.apache.spark.sql.catalyst.expressions.DayOfMonth|
|datetime_funcs|extract|org.apache.spark.sql.catalyst.expressions.Extract|
|datetime_funcs|from_unixtime|org.apache.spark.sql.catalyst.expressions.FromUnixTime|
|datetime_funcs|from_utc_timestamp|org.apache.spark.sql.catalyst.expressions.FromUTCTimestamp|
|datetime_funcs|hour|org.apache.spark.sql.catalyst.expressions.Hour|
|datetime_funcs|last_day|org.apache.spark.sql.catalyst.expressions.LastDay|
|datetime_funcs|make_date|org.apache.spark.sql.catalyst.expressions.MakeDate|
|datetime_funcs|make_interval|org.apache.spark.sql.catalyst.expressions.MakeInterval|
|datetime_funcs|make_timestamp|org.apache.spark.sql.catalyst.expressions.MakeTimestamp|
|datetime_funcs|minute|org.apache.spark.sql.catalyst.expressions.Minute|
|datetime_funcs|months_between|org.apache.spark.sql.catalyst.expressions.MonthsBetween|
|datetime_funcs|month|org.apache.spark.sql.catalyst.expressions.Month|
|datetime_funcs|next_day|org.apache.spark.sql.catalyst.expressions.NextDay|
|datetime_funcs|now|org.apache.spark.sql.catalyst.expressions.Now|
|datetime_funcs|quarter|org.apache.spark.sql.catalyst.expressions.Quarter|
|datetime_funcs|second|org.apache.spark.sql.catalyst.expressions.Second|
|datetime_funcs|timestamp_micros|org.apache.spark.sql.catalyst.expressions.MicrosToTimestamp|
|datetime_funcs|timestamp_millis|org.apache.spark.sql.catalyst.expressions.MillisToTimestamp|
|datetime_funcs|timestamp_seconds|org.apache.spark.sql.catalyst.expressions.SecondsToTimestamp|
|datetime_funcs|to_date|org.apache.spark.sql.catalyst.expressions.ParseToDate|
|datetime_funcs|to_timestamp|org.apache.spark.sql.catalyst.expressions.ParseToTimestamp|
|datetime_funcs|to_unix_timestamp|org.apache.spark.sql.catalyst.expressions.ToUnixTimestamp|
|datetime_funcs|to_utc_timestamp|org.apache.spark.sql.catalyst.expressions.ToUTCTimestamp|
|datetime_funcs|trunc|org.apache.spark.sql.catalyst.expressions.TruncDate|
|datetime_funcs|unix_date|org.apache.spark.sql.catalyst.expressions.UnixDate|
|datetime_funcs|unix_micros|org.apache.spark.sql.catalyst.expressions.UnixMicros|
|datetime_funcs|unix_millis|org.apache.spark.sql.catalyst.expressions.UnixMillis|
|datetime_funcs|unix_seconds|org.apache.spark.sql.catalyst.expressions.UnixSeconds|
|datetime_funcs|unix_timestamp|org.apache.spark.sql.catalyst.expressions.UnixTimestamp|
|datetime_funcs|weekday|org.apache.spark.sql.catalyst.expressions.WeekDay|
|datetime_funcs|weekofyear|org.apache.spark.sql.catalyst.expressions.WeekOfYear|
|datetime_funcs|year|org.apache.spark.sql.catalyst.expressions.Year|
|generator_funcs|explode_outer|org.apache.spark.sql.catalyst.expressions.Explode|
|generator_funcs|explode|org.apache.spark.sql.catalyst.expressions.Explode|
|generator_funcs|inline_outer|org.apache.spark.sql.catalyst.expressions.Inline|
|generator_funcs|inline|org.apache.spark.sql.catalyst.expressions.Inline|
|generator_funcs|posexplode_outer|org.apache.spark.sql.catalyst.expressions.PosExplode|
|generator_funcs|posexplode|org.apache.spark.sql.catalyst.expressions.PosExplode|
|generator_funcs|stack|org.apache.spark.sql.catalyst.expressions.Stack|
|hash_funcs|crc32|org.apache.spark.sql.catalyst.expressions.Crc32|
|hash_funcs|hash|org.apache.spark.sql.catalyst.expressions.Murmur3Hash|
|hash_funcs|md5|org.apache.spark.sql.catalyst.expressions.Md5|
|hash_funcs|sha1|org.apache.spark.sql.catalyst.expressions.Sha1|
|hash_funcs|sha2|org.apache.spark.sql.catalyst.expressions.Sha2|
|hash_funcs|sha|org.apache.spark.sql.catalyst.expressions.Sha1|
|hash_funcs|xxhash64|org.apache.spark.sql.catalyst.expressions.XxHash64|
|json_funcs|from_json|org.apache.spark.sql.catalyst.expressions.JsonToStructs|
|json_funcs|get_json_object|org.apache.spark.sql.catalyst.expressions.GetJsonObject|
|json_funcs|json_array_length|org.apache.spark.sql.catalyst.expressions.LengthOfJsonArray|
|json_funcs|json_object_keys|org.apache.spark.sql.catalyst.expressions.JsonObjectKeys|
|json_funcs|json_tuple|org.apache.spark.sql.catalyst.expressions.JsonTuple|
|json_funcs|schema_of_json|org.apache.spark.sql.catalyst.expressions.SchemaOfJson|
|json_funcs|to_json|org.apache.spark.sql.catalyst.expressions.StructsToJson|
|lambda_funcs|aggregate|org.apache.spark.sql.catalyst.expressions.ArrayAggregate|
|lambda_funcs|array_sort|org.apache.spark.sql.catalyst.expressions.ArraySort|
|lambda_funcs|exists|org.apache.spark.sql.catalyst.expressions.ArrayExists|
|lambda_funcs|filter|org.apache.spark.sql.catalyst.expressions.ArrayFilter|
|lambda_funcs|forall|org.apache.spark.sql.catalyst.expressions.ArrayForAll|
|lambda_funcs|map_filter|org.apache.spark.sql.catalyst.expressions.MapFilter|
|lambda_funcs|map_zip_with|org.apache.spark.sql.catalyst.expressions.MapZipWith|
|lambda_funcs|transform_keys|org.apache.spark.sql.catalyst.expressions.TransformKeys|
|lambda_funcs|transform_values|org.apache.spark.sql.catalyst.expressions.TransformValues|
|lambda_funcs|transform|org.apache.spark.sql.catalyst.expressions.ArrayTransform|
|lambda_funcs|zip_with|org.apache.spark.sql.catalyst.expressions.ZipWith|
|map_funcs|element_at|org.apache.spark.sql.catalyst.expressions.ElementAt|
|map_funcs|map_concat|org.apache.spark.sql.catalyst.expressions.MapConcat|
|map_funcs|map_entries|org.apache.spark.sql.catalyst.expressions.MapEntries|
|map_funcs|map_from_arrays|org.apache.spark.sql.catalyst.expressions.MapFromArrays|
|map_funcs|map_from_entries|org.apache.spark.sql.catalyst.expressions.MapFromEntries|
|map_funcs|map_keys|org.apache.spark.sql.catalyst.expressions.MapKeys|
|map_funcs|map_values|org.apache.spark.sql.catalyst.expressions.MapValues|
|map_funcs|map|org.apache.spark.sql.catalyst.expressions.CreateMap|
|map_funcs|str_to_map|org.apache.spark.sql.catalyst.expressions.StringToMap|
|math_funcs|%|org.apache.spark.sql.catalyst.expressions.Remainder|
|math_funcs|*|org.apache.spark.sql.catalyst.expressions.Multiply|
|math_funcs|+|org.apache.spark.sql.catalyst.expressions.Add|
|math_funcs|-|org.apache.spark.sql.catalyst.expressions.Subtract|
|math_funcs|/|org.apache.spark.sql.catalyst.expressions.Divide|
|math_funcs|abs|org.apache.spark.sql.catalyst.expressions.Abs|
|math_funcs|acosh|org.apache.spark.sql.catalyst.expressions.Acosh|
|math_funcs|acos|org.apache.spark.sql.catalyst.expressions.Acos|
|math_funcs|asinh|org.apache.spark.sql.catalyst.expressions.Asinh|
|math_funcs|asin|org.apache.spark.sql.catalyst.expressions.Asin|
|math_funcs|atan2|org.apache.spark.sql.catalyst.expressions.Atan2|
|math_funcs|atanh|org.apache.spark.sql.catalyst.expressions.Atanh|
|math_funcs|atan|org.apache.spark.sql.catalyst.expressions.Atan|
|math_funcs|bin|org.apache.spark.sql.catalyst.expressions.Bin|
|math_funcs|bround|org.apache.spark.sql.catalyst.expressions.BRound|
|math_funcs|cbrt|org.apache.spark.sql.catalyst.expressions.Cbrt|
|math_funcs|ceiling|org.apache.spark.sql.catalyst.expressions.Ceil|
|math_funcs|ceil|org.apache.spark.sql.catalyst.expressions.Ceil|
|math_funcs|conv|org.apache.spark.sql.catalyst.expressions.Conv|
|math_funcs|cosh|org.apache.spark.sql.catalyst.expressions.Cosh|
|math_funcs|cos|org.apache.spark.sql.catalyst.expressions.Cos|
|math_funcs|cot|org.apache.spark.sql.catalyst.expressions.Cot|
|math_funcs|degrees|org.apache.spark.sql.catalyst.expressions.ToDegrees|
|math_funcs|div|org.apache.spark.sql.catalyst.expressions.IntegralDivide|
|math_funcs|expm1|org.apache.spark.sql.catalyst.expressions.Expm1|
|math_funcs|exp|org.apache.spark.sql.catalyst.expressions.Exp|
|math_funcs|e|org.apache.spark.sql.catalyst.expressions.EulerNumber|
|math_funcs|factorial|org.apache.spark.sql.catalyst.expressions.Factorial|
|math_funcs|floor|org.apache.spark.sql.catalyst.expressions.Floor|
|math_funcs|greatest|org.apache.spark.sql.catalyst.expressions.Greatest|
|math_funcs|hex|org.apache.spark.sql.catalyst.expressions.Hex|
|math_funcs|hypot|org.apache.spark.sql.catalyst.expressions.Hypot|
|math_funcs|least|org.apache.spark.sql.catalyst.expressions.Least|
|math_funcs|ln|org.apache.spark.sql.catalyst.expressions.Log|
|math_funcs|log10|org.apache.spark.sql.catalyst.expressions.Log10|
|math_funcs|log1p|org.apache.spark.sql.catalyst.expressions.Log1p|
|math_funcs|log2|org.apache.spark.sql.catalyst.expressions.Log2|
|math_funcs|log|org.apache.spark.sql.catalyst.expressions.Logarithm|
|math_funcs|mod|org.apache.spark.sql.catalyst.expressions.Remainder|
|math_funcs|negative|org.apache.spark.sql.catalyst.expressions.UnaryMinus|
|math_funcs|pi|org.apache.spark.sql.catalyst.expressions.Pi|
|math_funcs|pmod|org.apache.spark.sql.catalyst.expressions.Pmod|
|math_funcs|positive|org.apache.spark.sql.catalyst.expressions.UnaryPositive|
|math_funcs|power|org.apache.spark.sql.catalyst.expressions.Pow|
|math_funcs|pow|org.apache.spark.sql.catalyst.expressions.Pow|
|math_funcs|radians|org.apache.spark.sql.catalyst.expressions.ToRadians|
|math_funcs|randn|org.apache.spark.sql.catalyst.expressions.Randn|
|math_funcs|random|org.apache.spark.sql.catalyst.expressions.Rand|
|math_funcs|rand|org.apache.spark.sql.catalyst.expressions.Rand|
|math_funcs|rint|org.apache.spark.sql.catalyst.expressions.Rint|
|math_funcs|round|org.apache.spark.sql.catalyst.expressions.Round|
|math_funcs|shiftleft|org.apache.spark.sql.catalyst.expressions.ShiftLeft|
|math_funcs|signum|org.apache.spark.sql.catalyst.expressions.Signum|
|math_funcs|sign|org.apache.spark.sql.catalyst.expressions.Signum|
|math_funcs|sinh|org.apache.spark.sql.catalyst.expressions.Sinh|
|math_funcs|sin|org.apache.spark.sql.catalyst.expressions.Sin|
|math_funcs|sqrt|org.apache.spark.sql.catalyst.expressions.Sqrt|
|math_funcs|tanh|org.apache.spark.sql.catalyst.expressions.Tanh|
|math_funcs|tan|org.apache.spark.sql.catalyst.expressions.Tan|
|math_funcs|unhex|org.apache.spark.sql.catalyst.expressions.Unhex|
|math_funcs|width_bucket|org.apache.spark.sql.catalyst.expressions.WidthBucket|
|misc_funcs|assert_true|org.apache.spark.sql.catalyst.expressions.AssertTrue|
|misc_funcs|current_catalog|org.apache.spark.sql.catalyst.expressions.CurrentCatalog|
|misc_funcs|current_database|org.apache.spark.sql.catalyst.expressions.CurrentDatabase|
|misc_funcs|input_file_block_length|org.apache.spark.sql.catalyst.expressions.InputFileBlockLength|
|misc_funcs|input_file_block_start|org.apache.spark.sql.catalyst.expressions.InputFileBlockStart|
|misc_funcs|input_file_name|org.apache.spark.sql.catalyst.expressions.InputFileName|
|misc_funcs|java_method|org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection|
|misc_funcs|monotonically_increasing_id|org.apache.spark.sql.catalyst.expressions.MonotonicallyIncreasingID|
|misc_funcs|raise_error|org.apache.spark.sql.catalyst.expressions.RaiseError|
|misc_funcs|reflect|org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection|
|misc_funcs|spark_partition_id|org.apache.spark.sql.catalyst.expressions.SparkPartitionID|
|misc_funcs|typeof|org.apache.spark.sql.catalyst.expressions.TypeOf|
|misc_funcs|uuid|org.apache.spark.sql.catalyst.expressions.Uuid|
|misc_funcs|version|org.apache.spark.sql.catalyst.expressions.SparkVersion|
|predicate_funcs|!|org.apache.spark.sql.catalyst.expressions.Not|
|predicate_funcs|<=>|org.apache.spark.sql.catalyst.expressions.EqualNullSafe|
|predicate_funcs|<=|org.apache.spark.sql.catalyst.expressions.LessThanOrEqual|
|predicate_funcs|<|org.apache.spark.sql.catalyst.expressions.LessThan|
|predicate_funcs|==|org.apache.spark.sql.catalyst.expressions.EqualTo|
|predicate_funcs|=|org.apache.spark.sql.catalyst.expressions.EqualTo|
|predicate_funcs|>=|org.apache.spark.sql.catalyst.expressions.GreaterThanOrEqual|
|predicate_funcs|>|org.apache.spark.sql.catalyst.expressions.GreaterThan|
|predicate_funcs|and|org.apache.spark.sql.catalyst.expressions.And|
|predicate_funcs|in|org.apache.spark.sql.catalyst.expressions.In|
|predicate_funcs|isnan|org.apache.spark.sql.catalyst.expressions.IsNaN|
|predicate_funcs|isnotnull|org.apache.spark.sql.catalyst.expressions.IsNotNull|
|predicate_funcs|isnull|org.apache.spark.sql.catalyst.expressions.IsNull|
|predicate_funcs|like|org.apache.spark.sql.catalyst.expressions.Like|
|predicate_funcs|not|org.apache.spark.sql.catalyst.expressions.Not|
|predicate_funcs|or|org.apache.spark.sql.catalyst.expressions.Or|
|predicate_funcs|regexp_like|org.apache.spark.sql.catalyst.expressions.RLike|
|predicate_funcs|rlike|org.apache.spark.sql.catalyst.expressions.RLike|
|string_funcs|ascii|org.apache.spark.sql.catalyst.expressions.Ascii|
|string_funcs|base64|org.apache.spark.sql.catalyst.expressions.Base64|
|string_funcs|bit_length|org.apache.spark.sql.catalyst.expressions.BitLength|
|string_funcs|char_length|org.apache.spark.sql.catalyst.expressions.Length|
|string_funcs|character_length|org.apache.spark.sql.catalyst.expressions.Length|
|string_funcs|char|org.apache.spark.sql.catalyst.expressions.Chr|
|string_funcs|chr|org.apache.spark.sql.catalyst.expressions.Chr|
|string_funcs|concat_ws|org.apache.spark.sql.catalyst.expressions.ConcatWs|
|string_funcs|decode|org.apache.spark.sql.catalyst.expressions.Decode|
|string_funcs|elt|org.apache.spark.sql.catalyst.expressions.Elt|
|string_funcs|encode|org.apache.spark.sql.catalyst.expressions.Encode|
|string_funcs|find_in_set|org.apache.spark.sql.catalyst.expressions.FindInSet|
|string_funcs|format_number|org.apache.spark.sql.catalyst.expressions.FormatNumber|
|string_funcs|format_string|org.apache.spark.sql.catalyst.expressions.FormatString|
|string_funcs|initcap|org.apache.spark.sql.catalyst.expressions.InitCap|
|string_funcs|instr|org.apache.spark.sql.catalyst.expressions.StringInstr|
|string_funcs|lcase|org.apache.spark.sql.catalyst.expressions.Lower|
|string_funcs|left|org.apache.spark.sql.catalyst.expressions.Left|
|string_funcs|length|org.apache.spark.sql.catalyst.expressions.Length|
|string_funcs|levenshtein|org.apache.spark.sql.catalyst.expressions.Levenshtein|
|string_funcs|locate|org.apache.spark.sql.catalyst.expressions.StringLocate|
|string_funcs|lower|org.apache.spark.sql.catalyst.expressions.Lower|
|string_funcs|lpad|org.apache.spark.sql.catalyst.expressions.StringLPad|
|string_funcs|ltrim|org.apache.spark.sql.catalyst.expressions.StringTrimLeft|
|string_funcs|octet_length|org.apache.spark.sql.catalyst.expressions.OctetLength|
|string_funcs|overlay|org.apache.spark.sql.catalyst.expressions.Overlay|
|string_funcs|parse_url|org.apache.spark.sql.catalyst.expressions.ParseUrl|
|string_funcs|position|org.apache.spark.sql.catalyst.expressions.StringLocate|
|string_funcs|printf|org.apache.spark.sql.catalyst.expressions.FormatString|
|string_funcs|regexp_extract_all|org.apache.spark.sql.catalyst.expressions.RegExpExtractAll|
|string_funcs|regexp_extract|org.apache.spark.sql.catalyst.expressions.RegExpExtract|
|string_funcs|regexp_replace|org.apache.spark.sql.catalyst.expressions.RegExpReplace|
|string_funcs|repeat|org.apache.spark.sql.catalyst.expressions.StringRepeat|
|string_funcs|replace|org.apache.spark.sql.catalyst.expressions.StringReplace|
|string_funcs|right|org.apache.spark.sql.catalyst.expressions.Right|
|string_funcs|rpad|org.apache.spark.sql.catalyst.expressions.StringRPad|
|string_funcs|rtrim|org.apache.spark.sql.catalyst.expressions.StringTrimRight|
|string_funcs|sentences|org.apache.spark.sql.catalyst.expressions.Sentences|
|string_funcs|soundex|org.apache.spark.sql.catalyst.expressions.SoundEx|
|string_funcs|space|org.apache.spark.sql.catalyst.expressions.StringSpace|
|string_funcs|split|org.apache.spark.sql.catalyst.expressions.StringSplit|
|string_funcs|substring_index|org.apache.spark.sql.catalyst.expressions.SubstringIndex|
|string_funcs|substring|org.apache.spark.sql.catalyst.expressions.Substring|
|string_funcs|substr|org.apache.spark.sql.catalyst.expressions.Substring|
|string_funcs|translate|org.apache.spark.sql.catalyst.expressions.StringTranslate|
|string_funcs|trim|org.apache.spark.sql.catalyst.expressions.StringTrim|
|string_funcs|ucase|org.apache.spark.sql.catalyst.expressions.Upper|
|string_funcs|unbase64|org.apache.spark.sql.catalyst.expressions.UnBase64|
|string_funcs|upper|org.apache.spark.sql.catalyst.expressions.Upper|
|struct_funcs|named_struct|org.apache.spark.sql.catalyst.expressions.CreateNamedStruct|
|struct_funcs|struct|org.apache.spark.sql.catalyst.expressions.CreateNamedStruct|
|window_funcs|cume_dist|org.apache.spark.sql.catalyst.expressions.CumeDist|
|window_funcs|dense_rank|org.apache.spark.sql.catalyst.expressions.DenseRank|
|window_funcs|lag|org.apache.spark.sql.catalyst.expressions.Lag|
|window_funcs|lead|org.apache.spark.sql.catalyst.expressions.Lead|
|window_funcs|nth_value|org.apache.spark.sql.catalyst.expressions.NthValue|
|window_funcs|ntile|org.apache.spark.sql.catalyst.expressions.NTile|
|window_funcs|percent_rank|org.apache.spark.sql.catalyst.expressions.PercentRank|
|window_funcs|rank|org.apache.spark.sql.catalyst.expressions.Rank|
|window_funcs|row_number|org.apache.spark.sql.catalyst.expressions.RowNumber|
|xml_funcs|xpath_boolean|org.apache.spark.sql.catalyst.expressions.xml.XPathBoolean|
|xml_funcs|xpath_double|org.apache.spark.sql.catalyst.expressions.xml.XPathDouble|
|xml_funcs|xpath_float|org.apache.spark.sql.catalyst.expressions.xml.XPathFloat|
|xml_funcs|xpath_int|org.apache.spark.sql.catalyst.expressions.xml.XPathInt|
|xml_funcs|xpath_long|org.apache.spark.sql.catalyst.expressions.xml.XPathLong|
|xml_funcs|xpath_number|org.apache.spark.sql.catalyst.expressions.xml.XPathDouble|
|xml_funcs|xpath_short|org.apache.spark.sql.catalyst.expressions.xml.XPathShort|
|xml_funcs|xpath_string|org.apache.spark.sql.catalyst.expressions.xml.XPathString|
|xml_funcs|xpath|org.apache.spark.sql.catalyst.expressions.xml.XPathList|
Closes#30040
NOTE: An original author of this PR is tanelk, so the credit should be given to tanelk.
### Why are the changes needed?
For better documents.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Add a test to check if exprs have a group tag in `ExpressionInfoSuite`.
Closes#30867 from maropu/pr30040.
Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
It's a known issue that re-analyzing an optimized plan can lead to various issues. We made several attempts to avoid it from happening, but the current solution `AlreadyOptimized` is still not 100% safe, as people can inject catalyst rules to call analyzer directly.
This PR proposes a simpler and safer idea: we set the `analyzed` flag to true after optimization, and analyzer will skip processing plans whose `analyzed` flag is true.
### Why are the changes needed?
make the code simpler and safer
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
existing tests.
Closes#30777 from cloud-fan/ds.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Throw `PartitionAlreadyExistsException` from `ALTER TABLE .. RENAME TO PARTITION` for a table from Hive V1 External Catalog in the case when the target partition already exists.
### Why are the changes needed?
1. To have the same behavior of V1 In-Memory and Hive External Catalog.
2. To not propagate internal Hive's exceptions to users.
### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the partition renaming command throws `PartitionAlreadyExistsException` for tables from the Hive catalog.
### How was this patch tested?
Added new UT:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *HiveCatalogedDDLSuite"
```
Closes#30866 from MaxGekk/throw-PartitionAlreadyExistsException.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
1. Move the `DROP TABLE` parsing tests to `DropTableParserSuite`
2. Place the v1 tests for `DROP TABLE` from `DDLSuite` and v2 tests from `DataSourceV2SQLSuite` to the common trait `DropTableSuiteBase`, so, the tests will run for V1, Hive V1 and V2 DS.
### Why are the changes needed?
- The unification will allow to run common `DROP TABLE` tests for both DSv1 and Hive DSv1, DSv2
- We can detect missing features and differences between DSv1 and DSv2 implementations.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DropTableParserSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DropTableSuite"
```
Closes#30854 from MaxGekk/unify-drop-table-tests.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to migrate `ALTER TABLE ... RENAME TO PARTITION` to use `UnresolvedTable` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).
Note that `ALTER TABLE ... RENAME TO PARTITION` is not supported for v2 tables.
### Why are the changes needed?
The PR makes the resolution consistent behavior consistent. For example,
```
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint, val string) USING csv PARTITIONED BY (id)")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("ALTER TABLE t PARTITION (id=1) RENAME TO PARTITION (id=2)") // works fine assuming id=1 exists.
```
, but after this PR:
```
sql("ALTER TABLE t PARTITION (id=1) RENAME TO PARTITION (id=2)")
org.apache.spark.sql.AnalysisException: t is a temp view. 'ALTER TABLE ... RENAME TO PARTITION' expects a table; line 1 pos 0
```
, which is the consistent behavior with other commands.
### Does this PR introduce _any_ user-facing change?
After this PR, `ALTER TABLE` in the above example is resolved to a temp view `t` first instead of `spark_catalog.test.t`.
### How was this patch tested?
Updated existing tests.
Closes#30862 from imback82/alter_table_rename_partition_v2.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR mainly improves and cleans up the test code introduced in #30855 based on the comment.
The test code is actually taken from another test `explain formatted - check presence of subquery in case of DPP` so this PR cleans the code too ( removed unnecessary `withTable`).
### Why are the changes needed?
To keep the test code clean.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`ExplainSuite` passes.
Closes#30861 from sarutak/followup-SPARK-33850.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
This PR fixes an issue that when AQE is enabled, EXPLAIN FORMATTED doesn't show the plan for subqueries.
```scala
val df = spark.range(1, 100)
df.createTempView("df")
spark.sql("SELECT (SELECT min(id) AS v FROM df)").explain("FORMATTED")
== Physical Plan ==
AdaptiveSparkPlan (3)
+- Project (2)
+- Scan OneRowRelation (1)
(1) Scan OneRowRelation
Output: []
Arguments: ParallelCollectionRDD[0] at explain at <console>:24, OneRowRelation, UnknownPartitioning(0)
(2) Project
Output [1]: [Subquery subquery#3, [id=#20] AS scalarsubquery()#5L]
Input: []
(3) AdaptiveSparkPlan
Output [1]: [scalarsubquery()#5L]
Arguments: isFinalPlan=false
```
After this change, the plan for the subquerie is shown.
```scala
== Physical Plan ==
* Project (2)
+- * Scan OneRowRelation (1)
(1) Scan OneRowRelation [codegen id : 1]
Output: []
Arguments: ParallelCollectionRDD[0] at explain at <console>:24, OneRowRelation, UnknownPartitioning(0)
(2) Project [codegen id : 1]
Output [1]: [Subquery scalar-subquery#3, [id=#24] AS scalarsubquery()#5L]
Input: []
===== Subqueries =====
Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery scalar-subquery#3, [id=#24]
* HashAggregate (6)
+- Exchange (5)
+- * HashAggregate (4)
+- * Range (3)
(3) Range [codegen id : 1]
Output [1]: [id#0L]
Arguments: Range (1, 100, step=1, splits=Some(12))
(4) HashAggregate [codegen id : 1]
Input [1]: [id#0L]
Keys: []
Functions [1]: [partial_min(id#0L)]
Aggregate Attributes [1]: [min#7L]
Results [1]: [min#8L]
(5) Exchange
Input [1]: [min#8L]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#20]
(6) HashAggregate [codegen id : 2]
Input [1]: [min#8L]
Keys: []
Functions [1]: [min(id#0L)]
Aggregate Attributes [1]: [min(id#0L)#4L]
Results [1]: [min(id#0L)#4L AS v#2L]
```
### Why are the changes needed?
For better debuggability.
### Does this PR introduce _any_ user-facing change?
Yes. Users can see the formatted plan for subqueries.
### How was this patch tested?
New test.
Closes#30855 from sarutak/fix-aqe-explain.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
The title is pretty self-explanatory.
### What changes were proposed in this pull request?
Fixing typos in the docs for `foreachBatch` functions.
### Why are the changes needed?
To fix typos in JavaDoc/ScalaDoc.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Yes.
Closes#30782 from ammar1x/patch-1.
Lead-authored-by: Ammar Al-Batool <ammar.albatool@gmail.com>
Co-authored-by: Ammar Al-Batool <ammar.al-batool@disneystreaming.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Currently, renaming v2 tables does not invalidate/recreate the cache, leading to an incorrect behavior (cache not being used) when v2 tables are renamed. This PR fixes the behavior.
### Why are the changes needed?
Fixing a bug since the cache associated with the renamed table is not being cleaned up/recreated.
### Does this PR introduce _any_ user-facing change?
Yes, now when a v2 table is renamed, cache is correctly updated.
### How was this patch tested?
Added a new test
Closes#30825 from imback82/rename_recreate_cache_v2.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Hive metastore has a limitation for the table property length. To work around it, Spark split the schema json string into several parts when saving to hive metastore as table properties. We need to do the same for histogram column stats as it can go very big.
This PR refactors the table property splitting code, so that we can share it between the schema json string and histogram column stats.
### Why are the changes needed?
To be able to analyze table when histogram data is big.
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
existing test and new tests
Closes#30809 from cloud-fan/cbo.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR group exception messages in `/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis`.
### 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#30717 from beliefer/SPARK-33599.
Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Currently when enable parquet vectorized reader, use binary type as partition col will return incorrect value as below UT
```scala
test("Parquet vector reader incorrect with binary partition value") {
Seq(false, true).foreach(tag => {
withSQLConf("spark.sql.parquet.enableVectorizedReader" -> tag.toString) {
withTable("t1") {
sql(
"""CREATE TABLE t1(name STRING, id BINARY, part BINARY)
| USING PARQUET PARTITIONED BY (part)""".stripMargin)
sql(s"INSERT INTO t1 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')")
if (tag) {
checkAnswer(sql("SELECT name, cast(id as string), cast(part as string) FROM t1"),
Row("a", "Spark SQL", ""))
} else {
checkAnswer(sql("SELECT name, cast(id as string), cast(part as string) FROM t1"),
Row("a", "Spark SQL", "Spark SQL"))
}
}
}
})
}
```
### Why are the changes needed?
Fix data incorrect issue
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UT
Closes#30824 from AngersZhuuuu/SPARK-33593.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR proposes to update `CACHE TABLE` to use a `LogicalPlan` when caching a query to avoid creating a `DataFrame` as suggested here: https://github.com/apache/spark/pull/30743#discussion_r543123190
For reference, `UNCACHE TABLE` also uses `LogicalPlan`: 0c12900120/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala (L91-L98)
### Why are the changes needed?
To avoid creating an unnecessary dataframe and make it consistent with `uncacheQuery` used in `UNCACHE TABLE`.
### Does this PR introduce _any_ user-facing change?
No, just internal changes.
### How was this patch tested?
Existing tests since this is an internal refactoring change.
Closes#30815 from imback82/cache_with_logical_plan.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR intends to fix the bug that throws a unsupported exception when running [the TPCDS q5](https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q5.sql) with AQE enabled ([this option is enabled by default now via SPARK-33679](031c5ef280)):
```
java.lang.UnsupportedOperationException: BroadcastExchange does not support the execute() code path.
at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecute(BroadcastExchangeExec.scala:189)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:180)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:218)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:215)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:176)
at org.apache.spark.sql.execution.exchange.ReusedExchangeExec.doExecute(Exchange.scala:60)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:180)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:218)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:215)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:176)
at org.apache.spark.sql.execution.adaptive.QueryStageExec.doExecute(QueryStageExec.scala:115)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:180)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:218)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:215)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:176)
at org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:321)
at org.apache.spark.sql.execution.SparkPlan.executeCollectIterator(SparkPlan.scala:397)
at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.$anonfun$relationFuture$1(BroadcastExchangeExec.scala:118)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
...
```
I've checked the AQE code and I found `EnsureRequirements` wrongly puts `BroadcastExchange` on a top of `BroadcastQueryStage` in the `reOptimize` phase as follows:
```
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#2183]
+- BroadcastQueryStage 2
+- ReusedExchange [d_date_sk#1086], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#1963]
```
A root cause is that a `Cast` class in a required child's distribution does not have a `timeZoneId` field (`timeZoneId=None`), and a `Cast` class in `child.outputPartitioning` has it. So, this difference can make the distribution requirement check fail in `EnsureRequirements`:
1e85707738/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala (L47-L50)
The `Cast` class that does not have a `timeZoneId` field is generated in the `HashJoin` object. To fix this issue, this PR proposes to use the `CastSupport.cast` method there.
### Why are the changes needed?
Bugfix.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually checked that q5 passed.
Closes#30818 from maropu/BugfixInAQE.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR changes the rule `RemoveRedundantProjects` from by default passing column ordering requirements from parent nodes to always require column orders regardless of the requirements from parent nodes unless otherwise specified. More specifically, instead of excluding a few nodes like GenerateExec, UnionExec that are known to require children columns to be ordered, the rule now includes a whitelist of nodes that allow passing through the ordering requirements from their parents.
### Why are the changes needed?
Currently, this rule passes through ordering requirements from parents directly to children except for a few excluded nodes. This incorrectly removes the necessary project nodes below a UnionExec since it is not excluded. An earlier PR also fixed a similar issue for GenerateExec (SPARK-32861). In order to prevent similar issues, the rule should be changed to always require column ordering except for a few specific nodes that we know for sure can pass through the requirements.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit tests
Closes#30659 from allisonwang-db/spark-33697-remove-project-union.
Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to migrate `ALTER TABLE ... SET [SERDE|SERDEPROPERTIES` to use `UnresolvedTable` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).
Note that `ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]` is not supported for v2 tables.
### Why are the changes needed?
The PR makes the resolution consistent behavior consistent. For example,
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint, val string) USING csv PARTITIONED BY (id)")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("ALTER TABLE t SET SERDE 'serdename'") // works fine
```
, but after this PR:
```
sql("ALTER TABLE t SET SERDE 'serdename'")
org.apache.spark.sql.AnalysisException: t is a temp view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES\' expects a table; line 1 pos 0
```
, which is the consistent behavior with other commands.
### Does this PR introduce _any_ user-facing change?
After this PR, `t` in the above example is resolved to a temp view first instead of `spark_catalog.test.t`.
### How was this patch tested?
Updated existing tests.
Closes#30813 from imback82/alter_table_serde_v2.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
It seems a very popular way that people use DISTRIBUTE BY clause with a literal to coalesce partition in the pure SQL data processing.
For example
```
insert into table src select * from values (1), (2), (3) t(a) distribute by 1
```
Users may want the final output to be one single data file, but if the reality is not always true. Spark will always create a file for partition 0 whether it contains data or not, so when the data all goes to a partition(IDX >0), there will be always 2 files there and the part-00000 is empty. On the other hand, a lot of empty tasks will be launched too, this is unnecessary.
When users repeat the insert statement daily, hourly, or minutely, it causes small file issues.
```
spark-sql> set spark.sql.shuffle.partitions=3;drop table if exists test2;create table test2 using parquet as select * from values (1), (2), (3) t(a) distribute by 1;
kentyaohulk ~/spark SPARK-33806 tree /Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20201202/spark-warehouse/test2/ -s
/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20201202/spark-warehouse/test2/
├── [ 0] _SUCCESS
├── [ 298] part-00000-5dc19733-9405-414b-9681-d25c4d3e9ee6-c000.snappy.parquet
└── [ 426] part-00001-5dc19733-9405-414b-9681-d25c4d3e9ee6-c000.snappy.parquet
```
To avoid this, there are some options you can take.
1. use `distribute by null`, let the data go to the partition 0
2. set spark.sql.adaptive.enabled to true for Spark to automatically coalesce
3. using hints instead of `distribute by`
4. set spark.sql.shuffle.partitions to 1
In this PR, we set the partition number to 1 in this particular case.
### Why are the changes needed?
1. avoid small file issues
2. avoid unnecessary empty tasks when no adaptive execution
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
new test
Closes#30800 from yaooqinn/SPARK-33806.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Based on the discussion https://github.com/apache/spark/pull/30743#discussion_r543124594, this PR proposes to remove the command name in AnalysisException message when a relation is not resolved.
For some of the commands that use `UnresolvedTable`, `UnresolvedView`, and `UnresolvedTableOrView` to resolve an identifier, when the identifier cannot be resolved, the exception will be something like `Table or view not found for 'SHOW TBLPROPERTIES': badtable`. The command name (`SHOW TBLPROPERTIES` in this case) should be dropped to be consistent with other existing commands.
### Why are the changes needed?
To make the exception message consistent.
### Does this PR introduce _any_ user-facing change?
Yes, the exception message will be changed from
```
Table or view not found for 'SHOW TBLPROPERTIES': badtable
```
to
```
Table or view not found: badtable
```
for commands that use `UnresolvedTable`, `UnresolvedView`, and `UnresolvedTableOrView` to resolve an identifier.
### How was this patch tested?
Updated existing tests.
Closes#30794 from imback82/remove_cmd_from_exception_msg.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
As a follow-up of https://github.com/apache/spark/pull/30045, we modify the RESET command here to respect the session initial configs per session first then fall back to the `SharedState` conf, which makes each session could maintain a different copy of initial configs for resetting.
### Why are the changes needed?
to make reset command saner.
### Does this PR introduce _any_ user-facing change?
yes, RESET will respect session initials first not always go to the system defaults
### How was this patch tested?
add new tests
Closes#30642 from yaooqinn/SPARK-32991-F.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Move common utility functions such as `test()`, `withNsTable()` and `checkPartitions()` to `DDLCommandTestUtils`.
2. Place common settings such as `version`, `catalog`, `defaultUsing`, `sparkConf` to `CommandSuiteBase`.
### Why are the changes needed?
To improve code maintenance of the unified tests.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running the affected test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowPartitionsSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowTablesSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```
Closes#30779 from MaxGekk/refactor-unified-tests.
Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to sort table properties in DESCRIBE TABLE command. This is consistent with DSv2 command as well:
e3058ba17c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala (L63)
This PR fixes the test case in Scala 2.13 build as well where the table properties have different order in the map.
### Why are the changes needed?
To keep the deterministic and pretty output, and fix the tests in Scala 2.13 build.
See https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-3.2-scala-2.13/49/testReport/junit/org.apache.spark.sql/SQLQueryTestSuite/describe_sql/
```
describe.sql
Expected "...spark_catalog, view.[query.out.col.2=c, view.referredTempFunctionsNames=[], view.catalogAndNamespace.part.1=default]]", but got "...spark_catalog, view.[catalogAndNamespace.part.1=default, view.query.out.col.2=c, view.referredTempFunctionsNames=[]]]" Result did not match for query #29
DESC FORMATTED v
```
### Does this PR introduce _any_ user-facing change?
Yes, it will change the text output from `DESCRIBE [EXTENDED|FORMATTED] table_name`.
Now the table properties are sorted by its key.
### How was this patch tested?
Related unittests were fixed accordingly.
Closes#30799 from HyukjinKwon/SPARK-33803.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to retain the cache's storage level when a table name is altered by `ALTER TABLE ... RENAME TO ...`.
### Why are the changes needed?
Currently, when a table name is altered, the table's cache is refreshed (if exists), but the storage level is not retained. For example:
```scala
def getStorageLevel(tableName: String): StorageLevel = {
val table = spark.table(tableName)
val cachedData = spark.sharedState.cacheManager.lookupCachedData(table).get
cachedData.cachedRepresentation.cacheBuilder.storageLevel
}
Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath)
sql(s"CREATE TABLE old USING parquet LOCATION '${path.toURI}'")
sql("CACHE TABLE old OPTIONS('storageLevel' 'MEMORY_ONLY')")
val oldStorageLevel = getStorageLevel("old")
sql("ALTER TABLE old RENAME TO new")
val newStorageLevel = getStorageLevel("new")
```
`oldStorageLevel` will be `StorageLevel(memory, deserialized, 1 replicas)` whereas `newStorageLevel` will be `StorageLevel(disk, memory, deserialized, 1 replicas)`, which is the default storage level.
### Does this PR introduce _any_ user-facing change?
Yes, now the storage level for the cache will be retained.
### How was this patch tested?
Added a unit test.
Closes#30774 from imback82/alter_table_rename_cache_fix.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to migrate `UNCACHE TABLE` to use `UnresolvedRelation` to resolve the table/view identifier in Analyzer as discussed https://github.com/apache/spark/pull/30403/files#r532360022.
### Why are the changes needed?
To resolve the table/view in the analyzer.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Updated existing tests
Closes#30743 from imback82/uncache_v2.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Throw `NoSuchPartitionsException` from `ALTER TABLE .. DROP TABLE` for not existing partitions of a table in V1 Hive external catalog.
### Why are the changes needed?
The behaviour of Hive external catalog deviates from V1/V2 in-memory catalogs that throw `NoSuchPartitionsException`. To improve user experience with Spark SQL, it would be better to throw the same exception.
### Does this PR introduce _any_ user-facing change?
Yes, the command throws `NoSuchPartitionsException` instead of the general exception `AnalysisException`.
### How was this patch tested?
By running tests for `ALTER TABLE .. DROP PARTITION`:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```
Closes#30778 from MaxGekk/hive-drop-partition-exception.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/30559 . The default parallelism config in Spark core is not good, as it's unclear where it applies. To not inherit this problem in Spark SQL, this PR refines the default parallelism SQL config, to make it clear that it only applies to leaf nodes.
### Why are the changes needed?
Make the config clearer.
### Does this PR introduce _any_ user-facing change?
It changes an unreleased config.
### How was this patch tested?
existing tests
Closes#30736 from cloud-fan/follow.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR tries to prune the unrequired output partitionings in cases when the columns are dropped from Project/Aggregates etc.
### Why are the changes needed?
Consider this query:
select t1.id from t1 JOIN t2 on t1.id = t2.id
This query will have top level Project node which will just project t1.id. But the outputPartitioning of this project node will be: PartitioningCollection(HashPartitioning(t1.id), HashPartitioning(t2.id)).
But since we are not propagating t2.id column, so we can drop HashPartitioning(t2.id) from the output partitioning of Project node.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UTs.
Closes#30762 from prakharjain09/SPARK-33758-prune-partitioning.
Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
The current `getSimpleMessage` of `AnalysisException` may adds semicolon repeatedly. There show an example below:
`select decode()`
The output will be:
```
org.apache.spark.sql.AnalysisException
Invalid number of arguments for function decode. Expected: 2; Found: 0;; line 1 pos 7
```
### Why are the changes needed?
Fix a bug, because it adds semicolon repeatedly.
### Does this PR introduce _any_ user-facing change?
Yes. the message of AnalysisException will be correct.
### How was this patch tested?
Jenkins test.
Closes#30724 from beliefer/SPARK-33752.
Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
The proposition of this pull request is described in this JIRA ticket: [https://issues.apache.org/jira/browse/SPARK-33769](url)
It proposes to improve the next-day function of the sql component to deal with Column type for the parameter dayOfWeek.
### Why are the changes needed?
It makes this functionality easier to use.
Actually the signature of this function is:
> def next_day(date: Column, dayOfWeek: String): Column.
It accepts the dayOfWeek parameter as a String. However in some cases, the dayOfWeek is in a Column, so a different value for each row of the dataframe.
A current workaround is to use the NextDay function like this:
> NextDay(dateCol.expr, dayOfWeekCol.expr).
The proposition is to add another signature for this function:
> def next_day(date: Column, dayOfWeek: Column): Column
In fact it is already the case for some other functions in this scala object, exemple:
> def date_sub(start: Column, days: Int): Column = date_sub(start, lit(days))
> def date_sub(start: Column, days: Column): Column = withExpr \{ DateSub(start.expr, days.expr) }
or
> def add_months(startDate: Column, numMonths: Int): Column = add_months(startDate, lit(numMonths))
> def add_months(startDate: Column, numMonths: Column): Column = withExpr {
> AddMonths(startDate.expr, numMonths.expr)
> }
This pull request is the same idea for the function next_day.
### Does this PR introduce _any_ user-facing change?
Yes
With this pull request, users of spark will have a new signature of the function:
> def next_day(date: Column, dayOfWeek: Column): Column
But the existing function signature should still work:
> def next_day(date: Column, dayOfWeek: String): Column
So this change should be retrocompatible.
### How was this patch tested?
The unit tests of the next_day function has been enhanced.
It tests the dayOfWeek parameter both as String and Column.
I also added a test case for the existing signature where the dayOfWeek is a non valid String. This should return null.
Closes#30761 from chongguang/SPARK-33769.
Authored-by: Chongguang LIU <chongguang.liu@laposte.fr>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
If we call `SubqueryExec.executeTake`, it will call `SubqueryExec.execute` which will trigger the codegen of the query plan and create an RDD. However, `SubqueryExec` already has a thread (`SubqueryExec.relationFuture`) to execute the query plan, which means we have 2 threads triggering codegen of the same query plan at the same time.
Spark codegen is not thread-safe, as we have places like `HashAggregateExec.bufferVars` that is a shared variable. The bug in `SubqueryExec` may lead to correctness bugs.
Since https://issues.apache.org/jira/browse/SPARK-33119, `ScalarSubquery` will call `SubqueryExec.executeTake`, so flaky tests start to appear.
This PR fixes the bug by reimplementing https://github.com/apache/spark/pull/30016 . We should pass the number of rows we want to collect to `SubqueryExec` at planning time, so that we can use `executeTake` inside `SubqueryExec.relationFuture`, and the caller side should always call `SubqueryExec.executeCollect`. This PR also adds checks so that we can make sure only `SubqueryExec.executeCollect` is called.
### Why are the changes needed?
fix correctness bug.
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
run `build/sbt "sql/testOnly *SQLQueryTestSuite -- -z scalar-subquery-select"` more than 10 times. Previously it fails, now it passes.
Closes#30765 from cloud-fan/bug.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
1. Move the `ALTER TABLE .. DROP PARTITION` parsing tests to `AlterTableDropPartitionParserSuite`
2. Place v1 tests for `ALTER TABLE .. DROP PARTITION` from `DDLSuite` and v2 tests from `AlterTablePartitionV2SQLSuite` to the common trait `AlterTableDropPartitionSuiteBase`, so, the tests will run for V1, Hive V1 and V2 DS.
### Why are the changes needed?
- The unification will allow to run common `ALTER TABLE .. DROP PARTITION` tests for both DSv1 and Hive DSv1, DSv2
- We can detect missing features and differences between DSv1 and DSv2 implementations.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *AlterTableDropPartitionParserSuite"
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite"
```
Closes#30747 from MaxGekk/unify-alter-table-drop-partition-tests.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to migrate `ALTER TABLE ... RECOVER PARTITIONS` to use `UnresolvedTable` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).
Note that `ALTER TABLE ... RECOVER PARTITIONS` is not supported for v2 tables.
### Why are the changes needed?
The PR makes the resolution consistent behavior consistent. For example,
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint, val string) USING csv PARTITIONED BY (id)")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("ALTER TABLE t RECOVER PARTITIONS") // works fine
```
, but after this PR:
```
sql("ALTER TABLE t RECOVER PARTITIONS")
org.apache.spark.sql.AnalysisException: t is a temp view. 'ALTER TABLE ... RECOVER PARTITIONS' expects a table; line 1 pos 0
```
, which is the consistent behavior with other commands.
### Does this PR introduce _any_ user-facing change?
After this PR, `ALTER TABLE t RECOVER PARTITIONS` in the above example is resolved to a temp view `t` first instead of `spark_catalog.test.t`.
### How was this patch tested?
Updated existing tests.
Closes#30773 from imback82/alter_table_recover_part_v2.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This changes DSv2 refresh table semantics to also recache the target table itself.
### Why are the changes needed?
Currently "REFRESH TABLE" in DSv2 only invalidate all caches referencing the table. With #30403 merged which adds support for caching a DSv2 table, we should also recache the target table itself to make the behavior consistent with DSv1.
### Does this PR introduce _any_ user-facing change?
Yes, now refreshing table in DSv2 also recache the target table itself.
### How was this patch tested?
Added coverage of this new behavior in the existing UT for v2 refresh table command
Closes#30742 from sunchao/SPARK-33653.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
List partitions returned by the V2 `SHOW PARTITIONS` command in alphabetical order.
### Why are the changes needed?
To have the same behavior as:
1. V1 in-memory catalog, see a28ed86a38/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala (L546)
2. V1 Hive catalogs, see fab2995972/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L715)
### Does this PR introduce _any_ user-facing change?
Yes, after the changes, V2 SHOW PARTITIONS sorts its output.
### How was this patch tested?
Added new UT to the base trait `ShowPartitionsSuiteBase` which contains tests for V1 and V2.
Closes#30764 from MaxGekk/sort-show-partitions.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Use Long value store encode value will overflow and return unexpected result, use BigInt to replace Long value and make logical more simple.
### Why are the changes needed?
Fix value overflow issue
### Does this PR introduce _any_ user-facing change?
People can sue `conf` function to convert value big then LONG.MAX_VALUE
### How was this patch tested?
Added UT
#### BenchMark
```
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.execution.benchmark
import scala.util.Random
import org.apache.spark.benchmark.Benchmark
import org.apache.spark.sql.functions._
object ConvFuncBenchMark extends SqlBasedBenchmark {
val charset =
Array[String]("0", "1", "2", "3", "4", "5", "6", "7", "8", "9",
"A", "B", "C", "D", "E", "F", "G",
"H", "I", "J", "K", "L", "M", "N",
"O", "P", "Q", "R", "S", "T",
"U", "V", "W", "X", "Y", "Z")
def constructString(from: Int, length: Int): String = {
val chars = charset.slice(0, from)
(0 to length).map(x => {
val v = Random.nextInt(from)
chars(v)
}).mkString("")
}
private def doBenchmark(cardinality: Long, length: Int, from: Int, toBase: Int): Unit = {
spark.range(cardinality)
.withColumn("str", lit(constructString(from, length)))
.select(conv(col("str"), from, toBase))
.noop()
}
/**
* Main process of the whole benchmark.
* Implementations of this method are supposed to use the wrapper method `runBenchmark`
* for each benchmark scenario.
*/
override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
val N = 1000000L
val benchmark = new Benchmark("conv", N, output = output)
benchmark.addCase("length 10 from 2 to 16") { _ =>
doBenchmark(N, 10, 2, 16)
}
benchmark.addCase("length 10 from 2 to 10") { _ =>
doBenchmark(N, 10, 2, 10)
}
benchmark.addCase("length 10 from 10 to 16") { _ =>
doBenchmark(N, 10, 10, 16)
}
benchmark.addCase("length 10 from 10 to 36") { _ =>
doBenchmark(N, 10, 10, 36)
}
benchmark.addCase("length 10 from 16 to 10") { _ =>
doBenchmark(N, 10, 10, 10)
}
benchmark.addCase("length 10 from 16 to 36") { _ =>
doBenchmark(N, 10, 16, 36)
}
benchmark.addCase("length 10 from 36 to 10") { _ =>
doBenchmark(N, 10, 36, 10)
}
benchmark.addCase("length 10 from 36 to 16") { _ =>
doBenchmark(N, 10, 36, 16)
}
//
benchmark.addCase("length 20 from 10 to 16") { _ =>
doBenchmark(N, 20, 10, 16)
}
benchmark.addCase("length 20 from 10 to 36") { _ =>
doBenchmark(N, 20, 10, 36)
}
benchmark.addCase("length 30 from 10 to 16") { _ =>
doBenchmark(N, 30, 10, 16)
}
benchmark.addCase("length 30 from 10 to 36") { _ =>
doBenchmark(N, 30, 10, 36)
}
//
benchmark.addCase("length 20 from 16 to 10") { _ =>
doBenchmark(N, 20, 16, 10)
}
benchmark.addCase("length 20 from 16 to 36") { _ =>
doBenchmark(N, 20, 16, 36)
}
benchmark.addCase("length 30 from 16 to 10") { _ =>
doBenchmark(N, 30, 16, 10)
}
benchmark.addCase("length 30 from 16 to 36") { _ =>
doBenchmark(N, 30, 16, 36)
}
benchmark.run()
}
}
```
Result with patch :
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.14.6
Intel(R) Core(TM) i5-8259U CPU 2.30GHz
conv: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
length 10 from 2 to 16 54 73 18 18.7 53.6 1.0X
length 10 from 2 to 10 43 47 5 23.5 42.5 1.3X
length 10 from 10 to 16 39 47 12 25.5 39.2 1.4X
length 10 from 10 to 36 38 42 3 26.5 37.7 1.4X
length 10 from 16 to 10 39 41 3 25.7 38.9 1.4X
length 10 from 16 to 36 36 41 4 27.6 36.3 1.5X
length 10 from 36 to 10 38 40 2 26.3 38.0 1.4X
length 10 from 36 to 16 37 39 2 26.8 37.2 1.4X
length 20 from 10 to 16 36 39 2 27.4 36.5 1.5X
length 20 from 10 to 36 37 39 2 27.2 36.8 1.5X
length 30 from 10 to 16 37 39 2 27.0 37.0 1.4X
length 30 from 10 to 36 36 38 2 27.5 36.3 1.5X
length 20 from 16 to 10 35 38 2 28.3 35.4 1.5X
length 20 from 16 to 36 34 38 3 29.2 34.3 1.6X
length 30 from 16 to 10 38 40 2 26.3 38.1 1.4X
length 30 from 16 to 36 37 38 1 27.2 36.8 1.5X
```
Result without patch:
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.14.6
Intel(R) Core(TM) i5-8259U CPU 2.30GHz
conv: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
length 10 from 2 to 16 66 101 29 15.1 66.1 1.0X
length 10 from 2 to 10 50 55 5 20.2 49.5 1.3X
length 10 from 10 to 16 46 51 5 21.8 45.9 1.4X
length 10 from 10 to 36 43 48 4 23.4 42.7 1.5X
length 10 from 16 to 10 44 47 4 22.9 43.7 1.5X
length 10 from 16 to 36 40 44 2 24.7 40.5 1.6X
length 10 from 36 to 10 40 44 4 25.0 40.1 1.6X
length 10 from 36 to 16 41 43 2 24.3 41.2 1.6X
length 20 from 10 to 16 39 41 2 25.7 38.9 1.7X
length 20 from 10 to 36 40 42 2 24.9 40.2 1.6X
length 30 from 10 to 16 39 40 1 25.9 38.6 1.7X
length 30 from 10 to 36 40 41 1 25.0 40.0 1.7X
length 20 from 16 to 10 40 41 1 25.1 39.8 1.7X
length 20 from 16 to 36 40 42 2 25.2 39.7 1.7X
length 30 from 16 to 10 39 42 2 25.6 39.0 1.7X
length 30 from 16 to 36 39 40 2 25.7 38.8 1.7X
```
Closes#30350 from AngersZhuuuu/SPARK-33428.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
As described in SPARK-33673, some test suites in `ParquetV2SchemaPruningSuite` will failed when set `parquet.version` to 1.11.1 because Parquet will return empty results for non-existent column since PARQUET-1765.
This pr change to use `readDataSchema()` instead of `schema` to build `pushedParquetFilters` in `ParquetScanBuilder` to avoid push down partition filters to `ParquetScan` for `DataSourceV2`
### Why are the changes needed?
Prepare for upgrade using Parquet 1.11.1.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- Pass the Jenkins or GitHub Action
- Manual test as follows:
```
mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.execution.datasources.parquet.ParquetV2SchemaPruningSuite -Dparquet.version=1.11.1 test -pl sql/core -am
```
**Before**
```
Run completed in 3 minutes, 13 seconds.
Total number of tests run: 134
Suites: completed 2, aborted 0
Tests: succeeded 120, failed 14, canceled 0, ignored 0, pending 0
*** 14 TESTS FAILED ***
```
**After**
```
Run completed in 3 minutes, 46 seconds.
Total number of tests run: 134
Suites: completed 2, aborted 0
Tests: succeeded 134, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```
Closes#30652 from LuciferYang/SPARK-33673.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
### What changes were proposed in this pull request?
This PR migrates `ALTER VIEW ... AS` to use `UnresolvedView` to resolve the view identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).
The `TempViewOrV1Table` extractor in `ResolveSessionCatalog.scala` can now be removed as well.
### Why are the changes needed?
To use `UnresolvedView` for view resolution.
### Does this PR introduce _any_ user-facing change?
The exception message changes if a table is found instead of view:
```
// OLD
`tab1` is not a view"
```
```
// NEW
"tab1 is a table. 'ALTER VIEW ... AS' expects a view."
```
### How was this patch tested?
Updated existing tests.
Closes#30723 from imback82/alter_view_as_statement.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Addressed comments in PR #30567, including:
1. add test case for SPARK-33647 and SPARK-33142
2. add migration guide
3. add `getRawTempView` and `getRawGlobalTempView` to return the raw view info (i.e. TemporaryViewRelation)
4. other minor code clean
### Why are the changes needed?
Code clean and more test cases
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existing and newly added test cases
Closes#30666 from linhongliu-db/SPARK-33142-followup.
Lead-authored-by: Linhong Liu <linhong.liu@databricks.com>
Co-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
[SPARK-33546] stated the there are three inconsistency behaviors for CREATE TABLE LIKE.
1. CREATE TABLE LIKE does not validate the user-specified hive serde. e.g., STORED AS PARQUET can't be used with ROW FORMAT SERDE.
2. CREATE TABLE LIKE requires STORED AS and ROW FORMAT SERDE to be specified together, which is not necessary.
3. CREATE TABLE LIKE does not respect the default hive serde.
This PR fix No.1, and after investigate, No.2 and No.3 turn out not to be issue.
Within Hive.
CREATE TABLE abc ... ROW FORMAT SERDE 'xxx.xxx.SerdeClass' (Without Stored as) will have
following result. Using the user specific SerdeClass and fetch default input/output format from default textfile format.
```
SerDe Library: xxx.xxx.SerdeClass
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
```
But for
CREATE TABLE dst LIKE src ROW FORMAT SERDE 'xxx.xxx.SerdeClass' (Without Stored as) will just ignore user specific SerdeClass and using (input, output, serdeClass) from src table.
It's better to just throw an exception on such ambiguous behavior, so No.2 is not an issue, but in the PR, we add some comments.
For No.3, in fact, CreateTableLikeCommand is using following logical to try to follow src table's storageFormat if current fileFormat.inputFormat is empty
```
val newStorage = if (fileFormat.inputFormat.isDefined) {
fileFormat
} else {
sourceTableDesc.storage.copy(locationUri = fileFormat.locationUri)
}
```
If we try to fill the new target table with HiveSerDe.getDefaultStorage if file format and row format is not explicity spefified, it will break the CREATE TABLE LIKE semantic.
### Why are the changes needed?
Bug Fix.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Added UT and Existing UT.
Closes#30705 from leanken/leanken-SPARK-33546.
Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Remove the `retainData` parameter from the logical node `AlterTableDropPartition`.
### Why are the changes needed?
The `AlterTableDropPartition` command reflects the sql statement (see SqlBase.g4):
```
| ALTER (TABLE | VIEW) multipartIdentifier
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions
```
but Spark doesn't allow to specify data retention. So, the parameter can be removed to improve code maintenance.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running the test suite `DDLParserSuite`.
Closes#30748 from MaxGekk/remove-retainData.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Modify the tests that add partitions with `LOCATION`, and where the number of nested folders in `LOCATION` doesn't match to the number of partitioned columns. In that case, `ALTER TABLE .. DROP PARTITION` tries to access (delete) folder out of the "base" path in `LOCATION`.
The problem belongs to Hive's MetaStore method `drop_partition_common`:
8696c82d07/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (L4876)
which tries to delete empty partition sub-folders recursively starting from the most deeper partition sub-folder up to the base folder. In the case when the number of sub-folder is not equal to the number of partitioned columns `part_vals.size()`, the method will try to list and delete folders out of the base path.
### Why are the changes needed?
To fix test failures like https://github.com/apache/spark/pull/30643#issuecomment-743774733:
```
org.apache.spark.sql.hive.execution.command.AlterTableAddPartitionSuite.ALTER TABLE .. ADD PARTITION Hive V1: SPARK-33521: universal type conversions of partition values
sbt.ForkMain$ForkError: org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: File file:/home/jenkins/workspace/SparkPullRequestBuilder/target/tmp/spark-832cb19c-65fd-41f3-ae0b-937d76c07897 does not exist;
at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:112)
at org.apache.spark.sql.hive.HiveExternalCatalog.dropPartitions(HiveExternalCatalog.scala:1014)
...
Caused by: sbt.ForkMain$ForkError: org.apache.hadoop.hive.metastore.api.MetaException: File file:/home/jenkins/workspace/SparkPullRequestBuilder/target/tmp/spark-832cb19c-65fd-41f3-ae0b-937d76c07897 does not exist
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.drop_partition_with_environment_context(HiveMetaStore.java:3381)
at sun.reflect.GeneratedMethodAccessor304.invoke(Unknown Source)
```
The issue can be reproduced by the following steps:
1. Create a base folder, for example: `/Users/maximgekk/tmp/part-location`
2. Create a sub-folder in the base folder and drop permissions for it:
```
$ mkdir /Users/maximgekk/tmp/part-location/aaa
$ chmod a-rwx chmod a-rwx /Users/maximgekk/tmp/part-location/aaa
$ ls -al /Users/maximgekk/tmp/part-location
total 0
drwxr-xr-x 3 maximgekk staff 96 Dec 13 18:42 .
drwxr-xr-x 33 maximgekk staff 1056 Dec 13 18:32 ..
d--------- 2 maximgekk staff 64 Dec 13 18:42 aaa
```
3. Create a table with a partition folder in the base folder:
```sql
spark-sql> create table tbl (id int) partitioned by (part0 int, part1 int);
spark-sql> alter table tbl add partition (part0=1,part1=2) location '/Users/maximgekk/tmp/part-location/tbl';
```
4. Try to drop this partition:
```
spark-sql> alter table tbl drop partition (part0=1,part1=2);
20/12/13 18:46:07 ERROR HiveClientImpl:
======================
Attempt to drop the partition specs in table 'tbl' database 'default':
Map(part0 -> 1, part1 -> 2)
In this attempt, the following partitions have been dropped successfully:
The remaining partitions have not been dropped:
[1, 2]
======================
Error in query: org.apache.hadoop.hive.ql.metadata.HiveException: Error accessing file:/Users/maximgekk/tmp/part-location/aaa;
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: Error accessing file:/Users/maximgekk/tmp/part-location/aaa;
```
The command fails because it tries to access to the sub-folder `aaa` that is out of the partition path `/Users/maximgekk/tmp/part-location/tbl`.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running the affected tests from local IDEA which does not have access to folders out of partition paths.
Closes#30752 from MaxGekk/fix-drop-partition-location.
Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Currently, when casting a string as timestamp type in ANSI mode, Spark throws a runtime exception on parsing error.
However, the result for casting a string to date is always null. We should throw an exception on parsing error as well.
### Why are the changes needed?
Add missing feature for ANSI mode
### Does this PR introduce _any_ user-facing change?
Yes for ANSI mode, Casting string to date will throw an exception on parsing error
### How was this patch tested?
Unit test
Closes#30687 from gengliangwang/castDate.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR intends to escape meta-characters (e.g., \n and \t) in `Dataset.showString`.
Before this PR:
```
scala> Seq("aaa\nbbb\t\tccccc").toDF("value").show()
+--------------+
| value|
+--------------+
|aaa
bbb ccccc|
+--------------+
```
After this PR:
```
+-----------------+
| value|
+-----------------+
|aaa\nbbb\t\tccccc|
+-----------------+
```
### Why are the changes needed?
For better output.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added a unit test.
Closes#30647 from maropu/EscapeMetaInShow.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Currently the maintenance interval is hard-coded in `StateStore`. This patch proposes to make it as SQL config.
### Why are the changes needed?
Currently the maintenance interval is hard-coded in `StateStore`. For consistency reason, it should be placed together with other SS configs together. SQLConf also has a better way to have doc and default value setting.
### Does this PR introduce _any_ user-facing change?
Yes. Previously users use Spark config to set the maintenance interval. Now they could use SQL config to set it.
### How was this patch tested?
Unit test.
Closes#30741 from viirya/maintenance-interval-sqlconfig.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This fixes `CatalogImpl.refreshTable` by using a new logical plan when recache the target table.
### Why are the changes needed?
In `CatalogImpl.refreshTable`, we currently recache the target table via:
```scala
sparkSession.sharedState.cacheManager.cacheQuery(table, cacheName, cacheLevel)
```
However, here `table` is generated before the `tableRelationCache` in `SessionCatalog` is invalidated, and therefore it still refers to old and staled logical plan, which is incorrect.
### Does this PR introduce _any_ user-facing change?
Yes, this fix behavior when a table is refreshed.
### How was this patch tested?
Added a unit test.
Closes#30699 from sunchao/SPARK-33729.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR proposes to migrate `CACHE TABLE` to use `UnresolvedRelation` to resolve the table/view identifier in Analyzer as discussed https://github.com/apache/spark/pull/30403/files#r532360022.
### Why are the changes needed?
To resolve the table in the analyzer.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existing tests
Closes#30598 from imback82/cache_v2.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Throw `PartitionsAlreadyExistException` from `createPartitions()` in Hive external catalog when a partition exists. Currently, `HiveExternalCatalog.createPartitions()` throws `AlreadyExistsException` wrapped by `AnalysisException`.
In the PR, I propose to catch `AlreadyExistsException` in `HiveClientImpl` and replace it by `PartitionsAlreadyExistException`.
### Why are the changes needed?
The behaviour of Hive external catalog deviates from V1/V2 in-memory catalogs that throw `PartitionsAlreadyExistException`. To improve user experience with Spark SQL, it would be better to throw the same exception.
### Does this PR introduce _any_ user-facing change?
Yes
### How was this patch tested?
By running existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
```
Closes#30711 from MaxGekk/hive-partition-exception.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
org.apache.hadoop.conf.Configuration#setIfUnset will ignore those with defaults too
### 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#30709 from yaooqinn/SPARK-33740.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Using the view captured catalog and namespace to lookup function, so the view
referred functions won't be overridden by newly created function with the same name,
but different database or function type (i.e. temporary function)
### Why are the changes needed?
bug fix, without this PR, changing database or create a temporary function with
the same name may cause failure when querying a view.
### Does this PR introduce _any_ user-facing change?
Yes, bug fix.
### How was this patch tested?
newly added and existing test cases.
Closes#30662 from linhongliu-db/SPARK-33692.
Lead-authored-by: Linhong Liu <linhong.liu@databricks.com>
Co-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR adds `allowTemp` flag to `UnresolvedView` so that `Analyzer` can check whether to resolve temp views or not.
This PR also migrates `ALTER VIEW ... SET/UNSET TBLPROPERTIES` to use `UnresolvedView` to resolve the table/view identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).
### Why are the changes needed?
To use `UnresolvedView` for view resolution.
One benefit is that the exception message is better for `ALTER VIEW ... SET/UNSET TBLPROPERTIES`. Before, if a temp view is passed, you will just get `NoSuchTableException` with `Table or view 'tmpView' not found in database 'default'`. But with this PR, you will get more description exception message: `tmpView is a temp view. ALTER VIEW ... SET TBLPROPERTIES expects a permanent view`.
### Does this PR introduce _any_ user-facing change?
The exception message changes as describe above.
### How was this patch tested?
Updated existing tests.
Closes#30676 from imback82/alter_view_set_unset_properties.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Move the `ALTER TABLE .. ADD PARTITION` parsing tests to `AlterTableAddPartitionParserSuite`
2. Place v1 tests for `ALTER TABLE .. ADD PARTITION` from `DDLSuite` and v2 tests from `AlterTablePartitionV2SQLSuite` to the common trait `AlterTableAddPartitionSuiteBase`, so, the tests will run for V1, Hive V1 and V2 DS.
### Why are the changes needed?
- The unification will allow to run common `ALTER TABLE .. ADD PARTITION` tests for both DSv1 and Hive DSv1, DSv2
- We can detect missing features and differences between DSv1 and DSv2 implementations.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite"
```
Closes#30685 from MaxGekk/unify-alter-table-add-partition-tests.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR is a followup of https://github.com/apache/spark/pull/30488. This PR proposes to rename `Alias.deniedMetadataKeys` to `Alias.nonInheritableMetadataKeys` to make it less confusing.
### Why are the changes needed?
To make it easier to maintain and read.
### Does this PR introduce _any_ user-facing change?
No. This is rather a code cleanup.
### How was this patch tested?
Ran the unittests written in the previous PR manually. Jenkins and GitHub Actions in this PR should also test them.
Closes#30682 from HyukjinKwon/SPARK-33071-SPARK-33536.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR proposes to migrate `MSCK REPAIR TABLE` to use `UnresolvedTable` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).
Note that `MSCK REPAIR TABLE` is not supported for v2 tables.
### Why are the changes needed?
The PR makes the resolution consistent behavior consistent. For example,
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint, val string) USING csv PARTITIONED BY (id)")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("MSCK REPAIR TABLE t") // works fine
```
, but after this PR:
```
sql("MSCK REPAIR TABLE t")
org.apache.spark.sql.AnalysisException: t is a temp view. 'MSCK REPAIR TABLE' expects a table; line 1 pos 0
```
, which is the consistent behavior with other commands.
### Does this PR introduce _any_ user-facing change?
After this PR, `MSCK REPAIR TABLE t` in the above example is resolved to a temp view `t` first instead of `spark_catalog.test.t`.
### How was this patch tested?
Updated existing tests.
Closes#30664 from imback82/repair_table_V2.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR intends to fix typos in the sub-modules:
* `sql/core`
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#30531 from jsoref/spelling-sql-core.
Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Sean Owen <srowen@gmail.com>