### What changes were proposed in this pull request?
Today, child expressions may be resolved based on "real" or metadata output attributes. We should prefer the real attribute during resolution if one exists.
### Why are the changes needed?
Today, attempting to resolve an expression when there is a "real" output attribute and a metadata attribute with the same name results in resolution failure. This is likely unexpected, as the user may not know about the metadata attribute.
### Does this PR introduce _any_ user-facing change?
Yes. Previously, the user would see an error message when resolving a column with the same name as a "real" output attribute and a metadata attribute as below:
```
org.apache.spark.sql.AnalysisException: Reference 'index' is ambiguous, could be: testcat.ns1.ns2.tableTwo.index, testcat.ns1.ns2.tableOne.index.; line 1 pos 71
at org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:363)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveChildren(LogicalPlan.scala:107)
```
Now, resolution succeeds and provides the "real" output attribute.
### How was this patch tested?
Added a unit test.
Closes#31654 from karenfeng/fallback-resolve-metadata.
Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### Why is this change being proposed?
This patch adds support for a new "product" aggregation function in `sql.functions` which multiplies-together all values in an aggregation group.
This is likely to be useful in statistical applications which involve combining probabilities, or financial applications that involve combining cumulative interest rates, but is also a versatile mathematical operation of similar status to `sum` or `stddev`. Other users [have noted](https://stackoverflow.com/questions/52991640/cumulative-product-in-spark) the absence of such a function in current releases of Spark.
This function is both much more concise than an expression of the form `exp(sum(log(...)))`, and avoids awkward edge-cases associated with some values being zero or negative, as well as being less computationally costly.
### Does this PR introduce _any_ user-facing change?
No - only adds new function.
### How was this patch tested?
Built-in tests have been added for the new `catalyst.expressions.aggregate.Product` class and its invocation via the (scala) `sql.functions.product` function. The latter, and the PySpark wrapper have also been manually tested in spark-shell and pyspark sessions. The SparkR wrapper is currently untested, and may need separate validation (I'm not an "R" user myself).
An illustration of the new functionality, within PySpark is as follows:
```
import pyspark.sql.functions as pf, pyspark.sql.window as pw
df = sqlContext.range(1, 17).toDF("x")
win = pw.Window.partitionBy(pf.lit(1)).orderBy(pf.col("x"))
df.withColumn("factorial", pf.product("x").over(win)).show(20, False)
+---+---------------+
|x |factorial |
+---+---------------+
|1 |1.0 |
|2 |2.0 |
|3 |6.0 |
|4 |24.0 |
|5 |120.0 |
|6 |720.0 |
|7 |5040.0 |
|8 |40320.0 |
|9 |362880.0 |
|10 |3628800.0 |
|11 |3.99168E7 |
|12 |4.790016E8 |
|13 |6.2270208E9 |
|14 |8.71782912E10 |
|15 |1.307674368E12 |
|16 |2.0922789888E13|
+---+---------------+
```
Closes#30745 from rwpenney/feature/agg-product.
Lead-authored-by: Richard Penney <rwp@rwpenney.uk>
Co-authored-by: Richard Penney <rwpenney@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
In the `SQLConf` object, the `sqlConfEntries` map is globally synchronized (it is a Java `Collections.synchronizedMap`): any operation, including a get, will need to acquire the lock.
An example of this is calling the `DatatType.sameType` method. This will trigger a check on `SQLConf.get.caseSensitiveAnalysis`. So every time we compare two datatypes with sameType, we hit a lock.
To avoid having multiple tasks locking on this, a better approach would be to use a map that does not lock on read (like a `ConcurrentHashMap`). This map implementation does not lock on read, and on write it only locks the map partially. The only lock that happens is on write on the same map key.
### Why are the changes needed?
Multiple tasks performing any operation that directly or indirectly trigger a query to the `SQLConf.sqlConfEntries` map, will require acquiring a global lock on that map. Something as easy as calling `DataType.sameType(...)` would be locking on the global `sqlConfEntries` lock of the `Collections.synchronizedMap`.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
No functionality change. Existing unit tests run normally.
Closes#31689 from gabrielenizzoli/SPARK-34573.
Authored-by: Gabriele Nizzoli <1545350+gabrielenizzoli@users.noreply.github.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
In the PR, I propose to generate unique attributes in the logical nodes of the `SHOW TABLES` command.
Also, this PR fixes similar issues in other logical nodes:
- ShowTableExtended
- ShowViews
- ShowTableProperties
- ShowFunctions
- ShowColumns
- ShowPartitions
- ShowNamespaces
### Why are the changes needed?
This fixes the issue which is demonstrated by the example below:
```scala
scala> val show1 = sql("SHOW TABLES IN ns1")
show1: org.apache.spark.sql.DataFrame = [namespace: string, tableName: string ... 1 more field]
scala> val show2 = sql("SHOW TABLES IN ns2")
show2: org.apache.spark.sql.DataFrame = [namespace: string, tableName: string ... 1 more field]
scala> show1.show
+---------+---------+-----------+
|namespace|tableName|isTemporary|
+---------+---------+-----------+
| ns1| tbl1| false|
+---------+---------+-----------+
scala> show2.show
+---------+---------+-----------+
|namespace|tableName|isTemporary|
+---------+---------+-----------+
| ns2| tbl2| false|
+---------+---------+-----------+
scala> show1.join(show2).where(show1("tableName") =!= show2("tableName")).show
org.apache.spark.sql.AnalysisException: Column tableName#17 are ambiguous. It's probably because you joined several Datasets together, and some of these Datasets are the same. This column points to one of the Datasets but Spark is unable to figure out which one. Please alias the Datasets with different names via `Dataset.as` before joining them, and specify the column using qualified name, e.g. `df.as("a").join(df.as("b"), $"a.id" > $"b.id")`. You can also set spark.sql.analyzer.failAmbiguousSelfJoin to false to disable this check.
at org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin$.apply(DetectAmbiguousSelfJoin.scala:157)
```
### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above works as expected:
```scala
scala> show1.join(show2).where(show1("tableName") =!= show2("tableName")).show
+---------+---------+-----------+---------+---------+-----------+
|namespace|tableName|isTemporary|namespace|tableName|isTemporary|
+---------+---------+-----------+---------+---------+-----------+
| ns1| tbl1| false| ns2| tbl2| false|
+---------+---------+-----------+---------+---------+-----------+
```
### How was this patch tested?
By running the new test:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowTablesSuite"
```
Closes#31675 from MaxGekk/fix-output-attrs.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose to generate "stable" output attributes per the logical node of the `DESCRIBE TABLE` command.
### Why are the changes needed?
This fixes the issue demonstrated by the example:
```scala
val tbl = "testcat.ns1.ns2.tbl"
sql(s"CREATE TABLE $tbl (c0 INT) USING _")
val description = sql(s"DESCRIBE TABLE $tbl")
description.drop("comment")
```
The `drop()` method fails with the error:
```
org.apache.spark.sql.AnalysisException: Resolved attribute(s) col_name#102,data_type#103 missing from col_name#29,data_type#30,comment#31 in operator !Project [col_name#102, data_type#103]. Attribute(s) with the same name appear in the operation: col_name,data_type. Please check if the right attribute(s) are used.;
!Project [col_name#102, data_type#103]
+- LocalRelation [col_name#29, data_type#30, comment#31]
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:51)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:50)
```
### Does this PR introduce _any_ user-facing change?
Yes. After the changes, `drop()`/`add()` works as expected:
```scala
description.drop("comment").show()
+---------------+---------+
| col_name|data_type|
+---------------+---------+
| c0| int|
| | |
| # Partitioning| |
|Not partitioned| |
+---------------+---------+
```
### How was this patch tested?
1. Run new test:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2SQLSuite"
```
2. Run existing test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```
Closes#31676 from MaxGekk/describe-table-drop-column.
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 makes partition spec parsing respect case sensitive conf.
### Why are the changes needed?
When parsing the partition spec, Spark will call `org.apache.spark.sql.catalyst.parser.ParserUtils.checkDuplicateKeys` to check if there are duplicate partition column names in the list. But this method is always case sensitive and doesn't detect duplicate partition column names when using different cases.
### Does this PR introduce _any_ user-facing change?
Yep. This prevents users from writing incorrect queries such as `INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)` when they don't enable case sensitive conf.
### How was this patch tested?
The new added test will fail without this change.
Closes#31669 from zsxwing/SPARK-34556.
Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Avro add zstandard codec since AVRO-2195. This pr add zstandard codec to Avro compression codec list.
### Why are the changes needed?
To make Avro support zstandard codec.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit test.
Closes#31673 from wangyum/SPARK-34479.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
To support +8:00 in Spark3 when execute sql
`select to_utc_timestamp("2020-02-07 16:00:00", "GMT+8:00")`
### Why are the changes needed?
+8:00 this format is supported in PostgreSQL,hive, presto, but not supported in Spark3
https://issues.apache.org/jira/browse/SPARK-34392
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
unit test
Closes#31624 from Karl-WangSK/zone.
Lead-authored-by: ShiKai Wang <wskqing@gmail.com>
Co-authored-by: Karl-WangSK <shikai.wang@linkflowtech.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Add more aggregate expressions to `EliminateDistinct` rule.
### Why are the changes needed?
Distinct aggregation can add a significant overhead. It's better to remove distinct whenever possible.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UT
Closes#30999 from tanelk/SPARK-33971_eliminate_distinct.
Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Implement `ColumnarMap.copy()` by using the `copy()` method of `ColumnarArray`.
### Why are the changes needed?
To eliminate `java.lang.UnsupportedOperationException` while using `ColumnarMap`.
### Does this PR introduce _any_ user-facing change?
Yes
### How was this patch tested?
By running new tests in `ColumnarBatchSuite`.
Closes#31663 from MaxGekk/columnar-map-copy.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR simplifies the resolution of v2 partition commands:
1. Add a common trait for v2 partition commands, so that we don't need to match them one by one in the rules.
2. Make partition spec an expression, so that it's easier to resolve them via tree node transformation.
3. Add `TruncatePartition` so that `TruncateTable` doesn't need to be a v2 partition command.
4. Simplify `CheckAnalysis` to only check if the table is partitioned. For partitioned tables, partition spec is always resolved, so we don't need to check it. The `SupportsAtomicPartitionManagement` check is also done in the runtime. Since Spark eagerly executes commands, exception in runtime will also be thrown at analysis time.
### Why are the changes needed?
code cleanup
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
existing tests
Closes#31637 from cloud-fan/simplify.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This patch proposes to let optimizer to remove unnecessary `Union` under `Distinct`/`Deduplicate`.
### Why are the changes needed?
For an `Union` under `Distinct`/`Deduplicate`, if its children are all the same, we can just keep one among them and remove the `Union`.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit tests.
Closes#31595 from viirya/remove-union.
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?
I found out during code review of https://github.com/apache/spark/pull/31567#discussion_r577379572, where we can push down limit to the left side of LEFT SEMI and LEFT ANTI join, if the join condition is empty.
Why it's safe to push down limit:
The semantics of LEFT SEMI join without condition:
(1). if right side is non-empty, output all rows from left side.
(2). if right side is empty, output nothing.
The semantics of LEFT ANTI join without condition:
(1). if right side is non-empty, output nothing.
(2). if right side is empty, output all rows from left side.
With the semantics of output all rows from left side or nothing (all or nothing), it's safe to push down limit to left side.
NOTE: LEFT SEMI / LEFT ANTI join with non-empty condition is not safe for limit push down, because output can be a portion of left side rows.
Reference: physical operator implementation for LEFT SEMI / LEFT ANTI join without condition - https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala#L200-L204 .
### Why are the changes needed?
Better performance. Save CPU and IO for these joins, as limit being pushed down before join.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added unit test in `LimitPushdownSuite.scala` and `SQLQuerySuite.scala`.
Closes#31630 from c21/limit-pushdown.
Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR follows up https://github.com/apache/spark/pull/30717
Maybe some contributors don't know the job and added some exception by the old way.
### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.
### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.
### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.
Closes#31316 from beliefer/SPARK-33599-followup.
Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to make `CreateViewStatement.child` to be `LogicalPlan`'s `children` so that it's resolved in the analyze phase.
### Why are the changes needed?
Currently, the `CreateViewStatement.child` is resolved when the create view command runs, which is inconsistent with other plan resolutions. For example, you may see the following in the physical plan:
```
== Physical Plan ==
Execute CreateViewCommand (1)
+- CreateViewCommand (2)
+- Project (4)
+- UnresolvedRelation (3)
```
### Does this PR introduce _any_ user-facing change?
Yes. For the example, you will now see the resolved plan:
```
== Physical Plan ==
Execute CreateViewCommand (1)
+- CreateViewCommand (2)
+- Project (5)
+- SubqueryAlias (4)
+- LogicalRelation (3)
```
### How was this patch tested?
Updated existing tests.
Closes#31273 from imback82/spark-34152.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In Spark ANSI mode, the type coercion rules are based on the type precedence lists of the input data types.
As per the section "Type precedence list determination" of "ISO/IEC 9075-2:2011
Information technology — Database languages - SQL — Part 2: Foundation (SQL/Foundation)", the type precedence lists of primitive data types are as following:
- Byte: Byte, Short, Int, Long, Decimal, Float, Double
- Short: Short, Int, Long, Decimal, Float, Double
- Int: Int, Long, Decimal, Float, Double
- Long: Long, Decimal, Float, Double
- Decimal: Any wider Numeric type
- Float: Float, Double
- Double: Double
- String: String
- Date: Date, Timestamp
- Timestamp: Timestamp
- Binary: Binary
- Boolean: Boolean
- Interval: Interval
As for complex data types, Spark will determine the precedent list recursively based on their sub-types.
With the definition of type precedent list, the general type coercion rules are as following:
- Data type S is allowed to be implicitly cast as type T iff T is in the precedence list of S
- Comparison is allowed iff the data type precedence list of both sides has at least one common element. When evaluating the comparison, Spark casts both sides as the tightest common data type of their precedent lists.
- There should be at least one common data type among all the children's precedence lists for the following operators. The data type of the operator is the tightest common precedent data type.
```
In, Except(odd), Intersect, Greatest, Least, Union, If, CaseWhen, CreateArray, Array Concat,Sequence, MapConcat, CreateMap
```
- For complex types (struct, array, map), Spark recursively looks into the element type and applies the rules above. If the element nullability is converted from true to false, add runtime null check to the elements.
Note: this new type coercion system will allow implicit converting String type literals as other primitive types, in case of breaking too many existing Spark SQL queries. This is a special rule and it is not from the ANSI SQL standard.
### Why are the changes needed?
The current type coercion rules are complex. Also, they are very hard to describe and understand. For details please refer the attached documentation "Default Type coercion rules of Spark"
[Default Type coercion rules of Spark.pdf](https://github.com/apache/spark/files/5874362/Default.Type.coercion.rules.of.Spark.pdf)
This PR is to create a new and strict type coercion system under ANSI mode. The rules are simple and clean, so that users can follow them easily
### Does this PR introduce _any_ user-facing change?
Yes, new implicit cast syntax rules in ANSI mode. All the details are in the first section of this description.
### How was this patch tested?
Unit tests
Closes#31349 from gengliangwang/ansiImplicitConversion.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
Implement the v2 execution node for the `TRUNCATE TABLE` command.
### Why are the changes needed?
To have feature parity with DS v1, and support truncation of v2 tables.
### Does this PR introduce _any_ user-facing change?
Yes
### How was this patch tested?
By running the unified tests for v1 and v2 tables:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *TruncateTableSuite"
```
Closes#31605 from MaxGekk/truncate-table-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?
In the PR, I propose to extend the `MSCK REPAIR TABLE` command, and support new options `{ADD|DROP|SYNC} PARTITIONS`. In particular:
1. Extend the logical node `RepairTable`, and add two new flags `enableAddPartitions` and `enableDropPartitions`.
2. Add similar flags to the v1 execution node `AlterTableRecoverPartitionsCommand`
3. Add new method `dropPartitions()` to `AlterTableRecoverPartitionsCommand` which drops partitions from the catalog if their locations in the file system don't exist.
4. Updated public docs about the `MSCK REPAIR TABLE` command:
<img width="1037" alt="Screenshot 2021-02-16 at 13 46 39" src="https://user-images.githubusercontent.com/1580697/108052607-7446d280-705d-11eb-8e25-7398254787a4.png">
Closes#31097
### Why are the changes needed?
- The changes allow to recover tables with removed partitions. The example below portraits the problem:
```sql
spark-sql> create table tbl2 (col int, part int) partitioned by (part);
spark-sql> insert into tbl2 partition (part=1) select 1;
spark-sql> insert into tbl2 partition (part=0) select 0;
spark-sql> show table extended like 'tbl2' partition (part = 0);
default tbl2 false Partition Values: [part=0]
Location: file:/Users/maximgekk/proj/apache-spark/spark-warehouse/tbl2/part=0
...
```
Remove the partition (part = 0) from the filesystem:
```
$ rm -rf /Users/maximgekk/proj/apache-spark/spark-warehouse/tbl2/part=0
```
Even after recovering, we cannot query the table:
```sql
spark-sql> msck repair table tbl2;
spark-sql> select * from tbl2;
21/01/08 22:49:13 ERROR SparkSQLDriver: Failed in [select * from tbl2]
org.apache.hadoop.mapred.InvalidInputException: Input path does not exist: file:/Users/maximgekk/proj/apache-spark/spark-warehouse/tbl2/part=0
```
- To have feature parity with Hive: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-RecoverPartitions(MSCKREPAIRTABLE)
### Does this PR introduce _any_ user-facing change?
Yes. After the changes, we can query recovered table:
```sql
spark-sql> msck repair table tbl2 sync partitions;
spark-sql> select * from tbl2;
1 1
spark-sql> show partitions tbl2;
part=1
```
### How was this patch tested?
- By running the modified test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *MsckRepairTableParserSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *PlanResolutionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsParallelSuite"
```
- Added unified v1 and v2 tests for `MSCK REPAIR TABLE`:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *MsckRepairTableSuite"
```
Closes#31499 from MaxGekk/repair-table-drop-partitions.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
In the PR, I propose to rename logical nodes of v2 commands in the form: `<verb> + <object>` like:
- AlterTableAddPartition -> AddPartition
- AlterTableSetLocation -> SetTableLocation
### Why are the changes needed?
1. For simplicity and readability of logical plans
2. For consistency with other logical nodes. For example, the logical node `RenameTable` for `ALTER TABLE .. RENAME TO` was added before `AlterTableRenamePartition`.
### Does this PR introduce _any_ user-facing change?
Should not since this is non-public APIs.
### How was this patch tested?
1. Check scala style: `./dev/scalastyle`
2. Affected test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite"
```
Closes#31596 from MaxGekk/rename-alter-table-logic-nodes.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
When resolving a view, we use the captured view name in `AnalysisContext` to
distinguish whether a relation name is a view or a table. But if the resolution failed,
other rules (e.g. `ResolveTables`) will try to resolve the relation again but without
`AnalysisContext`. So, in this case, the resolution may be incorrect. For example,
if the view refers to a dropped table while a view with the same name exists, the
dropped table will be resolved as a view rather than an unresolved exception.
### Why are the changes needed?
bugfix
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
newly added test cases
Closes#31606 from linhongliu-db/fix-temp-view-master.
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?
In the PR, I propose to revert https://github.com/apache/spark/pull/26659 partially regarding to comparability of interval values. The comment became incorrect after https://github.com/apache/spark/pull/27262.
### Why are the changes needed?
The comment is incorrect, and it might confuse Spark's devs/users.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By checking scala coding style `./dev/scalastyle`.
Closes#31610 from MaxGekk/doc-interval-not-comparable.
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 parser tests from `DDLParserSuite` to `AlterTableRenameParserSuite`.
2. Port DS v1 tests from `DDLSuite` and other test suites to `v1.AlterTableRenameBase` and to `v1.AlterTableRenameSuite`.
3. Add a test for DSv2 `ALTER TABLE .. RENAME` to `v2.AlterTableRenameSuite`.
### Why are the changes needed?
To improve test coverage.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running new test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenameSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenameParserSuite"
```
Closes#31575 from MaxGekk/unify-rename-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?
1. Add new interface `TruncatableTable` which represents tables that allow atomic truncation.
2. Implement new method in `InMemoryTable` and in `InMemoryPartitionTable`.
### Why are the changes needed?
To support `TRUNCATE TABLE` for v2 tables.
### Does this PR introduce _any_ user-facing change?
Should not.
### How was this patch tested?
Added new tests to `TableCatalogSuite` that check truncation of non-partitioned and partitioned tables:
```
$ build/sbt "test:testOnly *TableCatalogSuite"
```
Closes#31475 from MaxGekk/dsv2-truncate-table.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
UserDefinedType and UDTRegistration become public Developer APIs, not package-private to Spark.
### Why are the changes needed?
This proposes to simply open up the UserDefinedType class as a developer API. It was public in 1.x, but closed in 2.x for some possible redesign that does not seem to have happened.
Other libraries have managed to define UDTs anyway by inserting shims into the Spark namespace, and this evidently has worked OK. But package isolation in Java 9+ breaks this.
The logic here is mostly: this is de facto a stable API, so can at least be open to developers with the usual caveats about developer APIs.
Open questions:
- Is there in fact some important redesign that's needed before opening it? The comment to this effect is from 2016
- Is this all that needs to be opened up? Like PythonUserDefinedType?
- Should any of this be kept package-private?
This was first proposed in https://github.com/apache/spark/pull/16478 though it was a larger change, but, the other API issues it was fixing seem to have been addressed already (e.g. no need to return internal Spark types). It was never really reviewed.
My hunch is that there isn't much downside, and some upside, to just opening this as-is now.
### Does this PR introduce _any_ user-facing change?
UserDefinedType becomes visible to developers to subclass.
### How was this patch tested?
Existing tests; there is no change to the existing logic.
Closes#31461 from srowen/SPARK-7768.
Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Handled 'Deduplicate(Keys, Union)' operation in rule 'CombineUnions' to combine adjacent 'Union' operators into a single 'Union' if necessary when using 'Dataset.union.distinct.union.distinct'.
Currently only handle distinct-like 'Deduplicate', where the keys == output, for example:
```
val df1 = Seq((1, 2, 3)).toDF("a", "b", "c")
val df2 = Seq((6, 2, 5)).toDF("a", "b", "c")
val df3 = Seq((2, 4, 3)).toDF("c", "a", "b")
val df4 = Seq((1, 4, 5)).toDF("b", "a", "c")
val unionDF1 = df1.unionByName(df2).dropDuplicates(Seq("b", "a", "c"))
.unionByName(df3).dropDuplicates().unionByName(df4)
.dropDuplicates("a")
```
In this case, **all Union operators will be combined**.
but,
```
val df1 = Seq((1, 2, 3)).toDF("a", "b", "c")
val df2 = Seq((6, 2, 5)).toDF("a", "b", "c")
val df3 = Seq((2, 4, 3)).toDF("c", "a", "b")
val df4 = Seq((1, 4, 5)).toDF("b", "a", "c")
val unionDF = df1.unionByName(df2).dropDuplicates(Seq("a"))
.unionByName(df3).dropDuplicates("c").unionByName(df4)
.dropDuplicates("b")
```
In this case, **all unions will not be combined, because the Deduplicate.keys doesn't equal to Union.output**.
### Why are the changes needed?
When using 'Dataset.union.distinct.union.distinct', the operator is 'Deduplicate(Keys, Union)', but AstBuilder transform sql-style 'Union' to operator 'Distinct(Union)', the rule 'CombineUnions' in Optimizer only handle 'Distinct(Union)' operator but not Deduplicate(Keys, Union).
Please see the detailed description in [SPARK-34283](https://issues.apache.org/jira/browse/SPARK-34283).
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit tests.
Closes#31404 from zzcclp/SPARK-34283.
Authored-by: Zhichao Zhang <441586683@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Make the following SQL configs as internal:
- spark.sql.legacy.allowHashOnMapType
- spark.sql.legacy.sessionInitWithConfigDefaults
2. Add a test to check that all SQL configs from the `legacy` namespace are marked as internal configs.
### Why are the changes needed?
Assuming that legacy SQL configs shouldn't be set by users in common cases. The purpose of such configs is to allow switching to old behavior in corner cases. So, the configs should be marked as internals.
### Does this PR introduce _any_ user-facing change?
Should not.
### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *SQLConfSuite"
```
Closes#31577 from MaxGekk/mark-legacy-configs-as-internal.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
The current implement of some DDL not unify the output and not pass the output properly to physical command.
Such as: The output attributes of `ShowFunctions` does't pass to `ShowFunctionsCommand` properly.
As the query plan, this PR pass the output attributes from `ShowFunctions` to `ShowFunctionsCommand`.
### Why are the changes needed?
This PR pass the output attributes could keep the expr ID unchanged, so that avoid bugs when we apply more operators above the command output dataframe.
### Does this PR introduce _any_ user-facing change?
'No'.
### How was this patch tested?
Jenkins test.
Closes#31519 from beliefer/SPARK-34394.
Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
The current implement of some DDL not unify the output and not pass the output properly to physical command.
Such as: The output attributes of `ShowViews` does't pass to `ShowViewsCommand` properly.
As the query plan, this PR pass the output attributes from `ShowViews` to `ShowViewsCommand`.
### Why are the changes needed?
This PR pass the output attributes could keep the expr ID unchanged, so that avoid bugs when we apply more operators above the command output dataframe.
### Does this PR introduce _any_ user-facing change?
'No'.
### How was this patch tested?
Jenkins test.
Closes#31508 from beliefer/SPARK-34393.
Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
1. Put the SQL config `spark.sql.legacy.replaceDatabricksSparkAvro.enabled` to the list of deprecated configs `deprecatedSQLConfigs`
2. Update docs for the Avro datasource
<img width="982" alt="Screenshot 2021-02-17 at 21 04 26" src="https://user-images.githubusercontent.com/1580697/108249890-abed7180-7166-11eb-8cb7-0c246d2a34fc.png">
### Why are the changes needed?
The config exists for enough time. We can deprecate it, and recommend users to use `.format("avro")` instead.
### Does this PR introduce _any_ user-facing change?
Should not except of the warning with the recommendation to use the `avro` format.
### How was this patch tested?
1. By generating docs via:
```
$ SKIP_API=1 SKIP_SCALADOC=1 SKIP_PYTHONDOC=1 SKIP_RDOC=1 jekyll serve --watch
```
2. Manually checking the warning:
```
scala> spark.conf.set("spark.sql.legacy.replaceDatabricksSparkAvro.enabled", false)
21/02/17 21:20:18 WARN SQLConf: The SQL config 'spark.sql.legacy.replaceDatabricksSparkAvro.enabled' has been deprecated in Spark v3.2 and may be removed in the future. Use `.format("avro")` in `DataFrameWriter` or `DataFrameReader` instead.
```
Closes#31578 from MaxGekk/deprecate-replaceDatabricksSparkAvro.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR handles merge operations in `ReplaceNullWithFalseInPredicate`.
### Why are the changes needed?
These changes are needed to match what we already do for delete and update operations.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
This PR extends existing tests to cover merge operations.
Closes#31579 from aokolnychyi/spark-33736.
Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Move the datetime rebase SQL configs from the `legacy` namespace by:
1. Renaming of the existing rebase configs like `spark.sql.legacy.parquet.datetimeRebaseModeInRead` -> `spark.sql.parquet.datetimeRebaseModeInRead`.
2. Add the legacy configs as alternatives
3. Deprecate the legacy rebase configs.
### Why are the changes needed?
The rebasing SQL configs like `spark.sql.legacy.parquet.datetimeRebaseModeInRead` can be used not only for migration from previous Spark versions but also to read/write datatime columns saved by other systems/frameworks/libs. So, the configs shouldn't be considered as legacy configs.
### Does this PR introduce _any_ user-facing change?
Should not. Users will see a warning if they still use one of the legacy configs.
### How was this patch tested?
1. Manually checking new configs:
```scala
scala> spark.conf.get("spark.sql.parquet.datetimeRebaseModeInRead")
res0: String = EXCEPTION
scala> spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "LEGACY")
21/02/17 14:57:10 WARN SQLConf: The SQL config 'spark.sql.legacy.parquet.datetimeRebaseModeInRead' has been deprecated in Spark v3.2 and may be removed in the future. Use 'spark.sql.parquet.datetimeRebaseModeInRead' instead.
scala> spark.conf.get("spark.sql.parquet.datetimeRebaseModeInRead")
res2: String = LEGACY
```
2. By running a datetime rebasing test suite:
```
$ build/sbt "test:testOnly *ParquetRebaseDatetimeV1Suite"
```
Closes#31576 from MaxGekk/rebase-confs-alternatives.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Fix descriptions of the SQL configs `spark.sql.legacy.parquet.int96RebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInWrite`, and mention `EXCEPTION` as the default value.
### Why are the changes needed?
This fixes incorrect descriptions that can mislead users.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running `./dev/scalastyle`.
Closes#31557 from MaxGekk/int96-exception-by-default-followup.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Modify `RandomDataGenerator.forType()` to allow generation of dates/timestamps that are valid in both Julian and Proleptic Gregorian calendars. Currently, the function can produce a date (for example `1582-10-06`) which is valid in the Proleptic Gregorian calendar. Though it cannot be saved to ORC files AS IS since ORC format (ORC libs in fact) assumes Julian calendar. So, Spark shifts `1582-10-06` to the next valid date `1582-10-15` while saving it to ORC files. And as a consequence of that, the test fails because it compares original date `1582-10-06` and the date `1582-10-15` loaded back from the ORC files.
In this PR, I propose to generate valid dates/timestamps in both calendars for ORC datasource till SPARK-34440 is resolved.
### Why are the changes needed?
The changes fix failures of `HiveOrcHadoopFsRelationSuite`. For instance, the test "test all data types" fails with the seed **610710213676**:
```
== Results ==
!== Correct Answer - 20 == == Spark Answer - 20 ==
struct<index:int,col:date> struct<index:int,col:date>
...
![9,1582-10-06] [9,1582-10-15]
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running the modified test suite:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *HiveOrcHadoopFsRelationSuite"
```
Closes#31552 from MaxGekk/fix-HiveOrcHadoopFsRelationSuite.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR proposes to support `ifExists` flag for v2 `ALTER TABLE ... UNSET TBLPROPERTIES` command. Currently, the flag is not respected and the command behaves as `ifExists = true` where the command always succeeds when the properties do not exist.
### Why are the changes needed?
To support `ifExists` flag and align with v1 command behavior.
### Does this PR introduce _any_ user-facing change?
Yes, now if the property does not exist and `IF EXISTS` is not specified, the command will fail:
```
ALTER TABLE t UNSET TBLPROPERTIES ('unknown') // Fails with "Attempted to unset non-existent property 'unknown'"
ALTER TABLE t UNSET TBLPROPERTIES IF EXISTS ('unknown') // OK
```
### How was this patch tested?
Added new test
Closes#31494 from imback82/AlterTableUnsetPropertiesIfExists.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Move `PartitionTransforms.scala` from `sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions` to `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions`.
### Why are the changes needed?
We should put java/scala files to their corresponding directories.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
N/A
Closes#31546 from sunchao/SPARK-34419.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
Creating a Pandas dataframe via Apache Arrow currently can use twice as much memory as the final result, because during the conversion, both Pandas and Arrow retain a copy of the data. Arrow has a "self-destruct" mode now (Arrow >= 0.16) to avoid this, by freeing each column after conversion. This PR integrates support for this in toPandas, handling a couple of edge cases:
self_destruct has no effect unless the memory is allocated appropriately, which is handled in the Arrow serializer here. Essentially, the issue is that self_destruct frees memory column-wise, but Arrow record batches are oriented row-wise:
```
Record batch 0: allocation 0: column 0 chunk 0, column 1 chunk 0, ...
Record batch 1: allocation 1: column 0 chunk 1, column 1 chunk 1, ...
```
In this scenario, Arrow will drop references to all of column 0's chunks, but no memory will actually be freed, as the chunks were just slices of an underlying allocation. The PR copies each column into its own allocation so that memory is instead arranged as so:
```
Record batch 0: allocation 0 column 0 chunk 0, allocation 1 column 1 chunk 0, ...
Record batch 1: allocation 2 column 0 chunk 1, allocation 3 column 1 chunk 1, ...
```
The optimization is disabled by default, and can be enabled with the Spark SQL conf "spark.sql.execution.arrow.pyspark.selfDestruct.enabled" set to "true". We can't always apply this optimization because it's more likely to generate a dataframe with immutable buffers, which Pandas doesn't always handle well, and because it is slower overall (since it only converts one column at a time instead of in parallel).
### Why are the changes needed?
This lets us load larger datasets - in particular, with N bytes of memory, before we could never load a dataset bigger than N/2 bytes; now the overhead is more like N/1.25 or so.
### Does this PR introduce _any_ user-facing change?
Yes - it adds a new SQL conf "spark.sql.execution.arrow.pyspark.selfDestruct.enabled"
### How was this patch tested?
See the [mailing list](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Reducing-memory-usage-of-toPandas-with-Arrow-quot-self-destruct-quot-option-td30149.html) - it was tested with Python memory_profiler. Unit tests added to check memory within certain bounds and correctness with the option enabled.
Closes#29818 from lidavidm/spark-32953.
Authored-by: David Li <li.davidm96@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
### What changes were proposed in this pull request?
`TreeNodeException` causes the error msg not clear and it didn't work well.
Because the `TreeNodeException` looks redundancy, we could remove it.
There are show a case:
```
val df = Seq(("1", 1), ("1", 2), ("2", 3), ("2", 4)).toDF("x", "y")
val hashAggDF = df.groupBy("x").agg(c, sum("y"))
```
The above code will use `HashAggregateExec`. In order to ensure that an exception will be thrown when executing `HashAggregateExec`, I added `throw new RuntimeException("calculate error")` into 72b7f8abfb/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala (L85)
So, if the above code is executed, `RuntimeException("calculate error")` will be thrown.
Before this PR, the error is:
```
execute, tree:
ShuffleQueryStage 0
+- Exchange hashpartitioning(x#105, 5), ENSURE_REQUIREMENTS, [id=#168]
+- HashAggregate(keys=[x#105], functions=[partial_sum(y#106)], output=[x#105, sum#118L])
+- Project [_1#100 AS x#105, _2#101 AS y#106]
+- LocalTableScan [_1#100, _2#101]
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
ShuffleQueryStage 0
+- Exchange hashpartitioning(x#105, 5), ENSURE_REQUIREMENTS, [id=#168]
+- HashAggregate(keys=[x#105], functions=[partial_sum(y#106)], output=[x#105, sum#118L])
+- Project [_1#100 AS x#105, _2#101 AS y#106]
+- LocalTableScan [_1#100, _2#101]
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
at org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.doMaterialize(QueryStageExec.scala:163)
at org.apache.spark.sql.execution.adaptive.QueryStageExec.$anonfun$materialize$1(QueryStageExec.scala:81)
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.adaptive.QueryStageExec.materialize(QueryStageExec.scala:79)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5(AdaptiveSparkPlanExec.scala:207)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5$adapted(AdaptiveSparkPlanExec.scala:205)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$1(AdaptiveSparkPlanExec.scala:205)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.getFinalPhysicalPlan(AdaptiveSparkPlanExec.scala:179)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.executeCollect(AdaptiveSparkPlanExec.scala:289)
at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3708)
at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:2977)
at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3699)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3697)
at org.apache.spark.sql.Dataset.collect(Dataset.scala:2977)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$3(DataFrameAggregateSuite.scala:665)
at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38)
at org.apache.spark.sql.DataFrameAggregateSuite.org$apache$spark$sql$test$SQLTestUtilsBase$$super$withSQLConf(DataFrameAggregateSuite.scala:37)
at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf(SQLTestUtils.scala:246)
at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf$(SQLTestUtils.scala:244)
at org.apache.spark.sql.DataFrameAggregateSuite.withSQLConf(DataFrameAggregateSuite.scala:37)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$2(DataFrameAggregateSuite.scala:659)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$2$adapted(DataFrameAggregateSuite.scala:655)
at scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:877)
at scala.collection.immutable.List.foreach(List.scala:392)
at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:876)
at org.apache.spark.sql.DataFrameAggregateSuite.assertNoExceptions(DataFrameAggregateSuite.scala:655)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$126(DataFrameAggregateSuite.scala:695)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$126$adapted(DataFrameAggregateSuite.scala:695)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$125(DataFrameAggregateSuite.scala:695)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
at org.scalatest.Transformer.apply(Transformer.scala:22)
at org.scalatest.Transformer.apply(Transformer.scala:20)
at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:190)
at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:176)
at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:188)
at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:200)
at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:200)
at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:182)
at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:61)
at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:61)
at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:233)
at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:233)
at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:232)
at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1563)
at org.scalatest.Suite.run(Suite.scala:1112)
at org.scalatest.Suite.run$(Suite.scala:1094)
at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1563)
at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:237)
at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:237)
at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:236)
at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:61)
at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:61)
at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45)
at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1320)
at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1314)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1314)
at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:993)
at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:971)
at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1480)
at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:971)
at org.scalatest.tools.Runner$.run(Runner.scala:798)
at org.scalatest.tools.Runner.run(Runner.scala)
at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2(ScalaTestRunner.java:131)
at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:28)
Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
HashAggregate(keys=[x#105], functions=[partial_sum(y#106)], output=[x#105, sum#118L])
+- Project [_1#100 AS x#105, _2#101 AS y#106]
+- LocalTableScan [_1#100, _2#101]
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doExecute(HashAggregateExec.scala:84)
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.ShuffleExchangeExec.inputRDD$lzycompute(ShuffleExchangeExec.scala:118)
at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD(ShuffleExchangeExec.scala:118)
at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture$lzycompute(ShuffleExchangeExec.scala:122)
at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture(ShuffleExchangeExec.scala:121)
at org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.$anonfun$doMaterialize$1(QueryStageExec.scala:163)
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
... 91 more
Caused by: java.lang.RuntimeException: calculate error
at org.apache.spark.sql.execution.aggregate.HashAggregateExec.$anonfun$doExecute$1(HashAggregateExec.scala:85)
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
... 103 more
```
After this PR, the error is:
```
calculate error
java.lang.RuntimeException: calculate error
at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doExecute(HashAggregateExec.scala:84)
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.ShuffleExchangeExec.inputRDD$lzycompute(ShuffleExchangeExec.scala:117)
at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD(ShuffleExchangeExec.scala:117)
at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture$lzycompute(ShuffleExchangeExec.scala:121)
at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture(ShuffleExchangeExec.scala:120)
at org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.doMaterialize(QueryStageExec.scala:161)
at org.apache.spark.sql.execution.adaptive.QueryStageExec.$anonfun$materialize$1(QueryStageExec.scala:80)
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.adaptive.QueryStageExec.materialize(QueryStageExec.scala:78)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5(AdaptiveSparkPlanExec.scala:207)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5$adapted(AdaptiveSparkPlanExec.scala:205)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$1(AdaptiveSparkPlanExec.scala:205)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.getFinalPhysicalPlan(AdaptiveSparkPlanExec.scala:179)
at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.executeCollect(AdaptiveSparkPlanExec.scala:289)
at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3708)
at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:2977)
at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3699)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3697)
at org.apache.spark.sql.Dataset.collect(Dataset.scala:2977)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$3(DataFrameAggregateSuite.scala:665)
at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38)
at org.apache.spark.sql.DataFrameAggregateSuite.org$apache$spark$sql$test$SQLTestUtilsBase$$super$withSQLConf(DataFrameAggregateSuite.scala:37)
at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf(SQLTestUtils.scala:246)
at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf$(SQLTestUtils.scala:244)
at org.apache.spark.sql.DataFrameAggregateSuite.withSQLConf(DataFrameAggregateSuite.scala:37)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$2(DataFrameAggregateSuite.scala:659)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$assertNoExceptions$2$adapted(DataFrameAggregateSuite.scala:655)
at scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:877)
at scala.collection.immutable.List.foreach(List.scala:392)
at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:876)
at org.apache.spark.sql.DataFrameAggregateSuite.assertNoExceptions(DataFrameAggregateSuite.scala:655)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$126(DataFrameAggregateSuite.scala:695)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$126$adapted(DataFrameAggregateSuite.scala:695)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.apache.spark.sql.DataFrameAggregateSuite.$anonfun$new$125(DataFrameAggregateSuite.scala:695)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
at org.scalatest.Transformer.apply(Transformer.scala:22)
at org.scalatest.Transformer.apply(Transformer.scala:20)
at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:190)
at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:176)
at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:188)
at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:200)
at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:200)
at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:182)
at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:61)
at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:61)
at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:233)
at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:233)
at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:232)
at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1563)
at org.scalatest.Suite.run(Suite.scala:1112)
at org.scalatest.Suite.run$(Suite.scala:1094)
at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1563)
at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:237)
at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:237)
at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:236)
at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:61)
at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:61)
at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45)
at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1320)
at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1314)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1314)
at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:993)
at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:971)
at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1480)
at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:971)
at org.scalatest.tools.Runner$.run(Runner.scala:798)
at org.scalatest.tools.Runner.run(Runner.scala)
at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2(ScalaTestRunner.java:131)
at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:28)
```
### Why are the changes needed?
`TreeNodeException` didn't work well.
### Does this PR introduce _any_ user-facing change?
'No'.
### How was this patch tested?
Jenkins test.
Closes#31337 from beliefer/SPARK-34234.
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?
This PR includes the following changes:
1. in `CatalogImpl.uncacheTable`, invalidate caches in cascade when the target table is
a temp view, and `spark.sql.legacy.storeAnalyzedPlanForView` is false (default value).
2. make `SessionCatalog.lookupTempView` public and return processed temp view plan (i.e., with `View` op).
### Why are the changes needed?
Following [SPARK-34052](https://issues.apache.org/jira/browse/SPARK-34052) (#31107), we should invalidate in cascade for `CatalogImpl.uncacheTable` when the table is a temp view, so that the behavior is consistent.
### Does this PR introduce _any_ user-facing change?
Yes, now `SQLContext.uncacheTable` will drop temp view in cascade by default.
### How was this patch tested?
Added a UT
Closes#31462 from sunchao/SPARK-34347.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
Passing around the output attributes should have more benefits like keeping the exprID unchanged to avoid bugs when we apply more operators above the command output DataFrame.
This PR did 2 things :
1. After this pr, a `SHOW TBLPROPERTIES` clause's output shows `key` and `value` columns whether you specify the table property `key`. Before this pr, a `SHOW TBLPROPERTIES` clause's output only show a `value` column when you specify the table property `key`..
2. Keep `SHOW TBLPROPERTIES` command's output attribute exprId unchanged.
### Why are the changes needed?
1. Keep `SHOW TBLPROPERTIES`'s output schema consistence
2. Keep `SHOW TBLPROPERTIES` command's output attribute exprId unchanged.
### Does this PR introduce _any_ user-facing change?
After this pr, a `SHOW TBLPROPERTIES` clause's output shows `key` and `value` columns whether you specify the table property `key`. Before this pr, a `SHOW TBLPROPERTIES` clause's output only show a `value` column when you specify the table property `key`.
Before this PR:
```
sql > SHOW TBLPROPERTIES tabe_name('key')
value
value_of_key
```
After this PR
```
sql > SHOW TBLPROPERTIES tabe_name('key')
key value
key value_of_key
```
### How was this patch tested?
Added UT
Closes#31378 from AngersZhuuuu/SPARK-34240.
Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Keep consistence with other `SHOW` command according to https://github.com/apache/spark/pull/31341#issuecomment-774613080
### Why are the changes needed?
Keep consistence
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Not need
Closes#31516 from AngersZhuuuu/SPARK-34238-follow-up.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Delegate table name validation to the session catalog
### Why are the changes needed?
Queerying of tables with nested namespaces.
### Does this PR introduce _any_ user-facing change?
SQL queries of nested namespace queries
### How was this patch tested?
Unit tests updated.
Closes#31427 from holdenk/SPARK-34209-delegate-table-name-validation-to-the-catalog.
Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
### What changes were proposed in this pull request?
Currently, we pass the default value `EmptyRow` to method `checkEvaluation` in the `StringExpressionsSuite`, but the default value of the 'checkEvaluation' method parameter is the `emptyRow`.
We can clean the parameter for Code Simplifications.
### Why are the changes needed?
for Code Simplifications
**before**:
```
def testConcat(inputs: String*): Unit = {
val expected = if (inputs.contains(null)) null else inputs.mkString
checkEvaluation(Concat(inputs.map(Literal.create(_, StringType))), expected, EmptyRow)
}
```
**after**:
```
def testConcat(inputs: String*): Unit = {
val expected = if (inputs.contains(null)) null else inputs.mkString
checkEvaluation(Concat(inputs.map(Literal.create(_, StringType))), expected)
}
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or Github action.
Closes#31510 from yikf/master.
Authored-by: yikf <13468507104@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
This PR proposes to propagate the name used for registering UDFs to `ScalaUDF`, `ScalaUDAF` and `ScaalAggregator`.
Note that `PythonUDF` gets the name correctly: 466c045bfa/python/pyspark/sql/udf.py (L358-L359)
, and same for Hive UDFs:
466c045bfa/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala (L67)
### Why are the changes needed?
This PR can help in the following scenarios:
1) Better EXPLAIN output
2) By adding `def name: String` to `UserDefinedExpression`, we can match an expression by `UserDefinedExpression` and look up the catalog, an use case needed for #31273.
### Does this PR introduce _any_ user-facing change?
The EXPLAIN output involving udfs will be changed to use the name used for UDF registration.
For example, for the following:
```
sql("CREATE TEMPORARY FUNCTION test_udf AS 'org.apache.spark.examples.sql.Spark33084'")
sql("SELECT test_udf(col1) FROM VALUES (1), (2), (3)").explain(true)
```
The output of the optimized plan will change from:
```
Aggregate [spark33084(cast(col1#223 as bigint), org.apache.spark.examples.sql.Spark330846906be0f, 1, 1) AS spark33084(col1)#237]
+- LocalRelation [col1#223]
```
to
```
Aggregate [test_udf(cast(col1#223 as bigint), org.apache.spark.examples.sql.Spark330847a62d697, 1, 1, Some(test_udf)) AS test_udf(col1)#237]
+- LocalRelation [col1#223]
```
### How was this patch tested?
Added new tests.
Closes#31500 from imback82/udaf_name.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Adds an assert to `FixedLengthRowBasedKeyValueBatch#appendRow` method to check the incoming vlen and klen by comparing them with the lengths stored as member variables as followup to https://github.com/apache/spark/pull/30788
### Why are the changes needed?
Add assert statement to catch similar bugs in future.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Ran some tests locally, though not easy to test.
Closes#31447 from yliou/SPARK-33726-Assert.
Authored-by: yliou <yliou@berkeley.edu>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
In Spark, `set -v` is defined as "Queries all properties that are defined in the SQLConf of the sparkSession".
But there are other external modules that also define properties and register them to SQLConf. In this case,
it can't be displayed by `set -v` until the conf object is initiated (i.e. calling the object at least once).
In this PR, I propose to eagerly initiate all the objects registered to SQLConf, so that `set -v` will always output
the completed properties.
### Why are the changes needed?
Improve the `set -v` command to produces completed and deterministic results
### Does this PR introduce _any_ user-facing change?
`set -v` command will dump more configs
### How was this patch tested?
existing tests
Closes#30363 from linhongliu-db/set-v.
Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Keep consistence with other `SHOW` command according to https://github.com/apache/spark/pull/31341#issuecomment-774613080
### Why are the changes needed?
Keep consistence
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Not need
Closes#31518 from AngersZhuuuu/SPARK-34239-followup.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>