### What changes were proposed in this pull request?
This patch proposes to make StaticInvoke able to find method with given method name even the parameter types do not exactly match to argument classes.
### Why are the changes needed?
Unlike `Invoke`, `StaticInvoke` only tries to get the method with exact argument classes. If the calling method's parameter types are not exactly matched with the argument classes, `StaticInvoke` cannot find the method.
`StaticInvoke` should be able to find the method under the cases too.
### Does this PR introduce _any_ user-facing change?
Yes. `StaticInvoke` can find a method even the argument classes are not exactly matched.
### How was this patch tested?
Unit test.
Closes#32413 from viirya/static-invoke.
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?
This PR group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog`.
### 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#32377 from beliefer/SPARK-35021.
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?
Added the following TreePattern enums:
- APPEND_COLUMNS
- DESERIALIZE_TO_OBJECT
- LAMBDA_VARIABLE
- MAP_OBJECTS
- SERIALIZE_FROM_OBJECT
- PROJECT
- TYPED_FILTER
Added tree traversal pruning to the following rules dealing with objects:
- EliminateSerialization
- CombineTypedFilters
- EliminateMapObjects
- ObjectSerializerPruning
### Why are the changes needed?
Reduce the number of tree traversals and hence improve the query compilation latency.
### How was this patch tested?
Existing tests.
Closes#32451 from sigmod/object.
Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
### What changes were proposed in this pull request?
If `targetObject` is not nullable, we don't need the object null check in `Invoke`.
### Why are the changes needed?
small perf improvement
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
existing tests
Closes#32466 from cloud-fan/invoke.
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 group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util`.
### 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#32367 from beliefer/SPARK-35020.
Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This is a follow up to https://github.com/apache/spark/pull/32032#discussion_r620928086. Basically, `children`/`innerChildren` should be mutually exclusive for `AlterViewAsCommand` and `CreateViewCommand`, which extend `AnalysisOnlyCommand`. Otherwise, there could be an issue in the `EXPLAIN` command. Currently, this is not an issue, because these commands will be analyzed (children will always be empty) when the `EXPLAIN` command is run.
### Why are the changes needed?
To be future-proof where these commands are directly used.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added new tsts
Closes#32447 from imback82/SPARK-34701-followup.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Hide internal view properties for describe table command, because those
properties are generated by spark and should be transparent to the end-user.
### Why are the changes needed?
Avoid internal properties confusing the users.
### Does this PR introduce _any_ user-facing change?
Yes
Before this change, the user will see below output for `describe formatted test_view`
```
....
Table Properties [view.catalogAndNamespace.numParts=2, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=default, view.query.out.col.0=c, view.query.out.col.1=v, view.query.out.numCols=2, view.referredTempFunctionsNames=[], view.referredTempViewNames=[]]
...
```
After this change, the internal properties will be hidden for `describe formatted test_view`
```
...
Table Properties []
...
```
### How was this patch tested?
existing UT
Closes#32441 from linhongliu-db/hide-properties.
Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This pull request proposes a new API for streaming sources to signal that they can report metrics, and adds a use case to support Kafka micro batch stream to report the stats of # of offsets for the current offset falling behind the latest.
A public interface is added.
`metrics`: returns the metrics reported by the streaming source with given offset.
### Why are the changes needed?
The new API can expose any custom metrics for the "current" offset for streaming sources. Different from #31398, this PR makes metrics available to user through progress report, not through spark UI. A use case is that people want to know how the current offset falls behind the latest offset.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit test for Kafka micro batch source v2 are added to test the Kafka use case.
Closes#31944 from yijiacui-db/SPARK-34297.
Authored-by: Yijia Cui <yijia.cui@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
To fix lambda variable name issues in nested DataFrame functions, this PR modifies code to use a global counter for `LambdaVariables` names created by higher order functions.
This is the rework of #31887. Closes#31887.
### Why are the changes needed?
This moves away from the current hard-coded variable names which break on nested function calls. There is currently a bug where nested transforms in particular fail (the inner variable shadows the outer variable)
For this query:
```
val df = Seq(
(Seq(1,2,3), Seq("a", "b", "c"))
).toDF("numbers", "letters")
df.select(
f.flatten(
f.transform(
$"numbers",
(number: Column) => { f.transform(
$"letters",
(letter: Column) => { f.struct(
number.as("number"),
letter.as("letter")
) }
) }
)
).as("zipped")
).show(10, false)
```
This is the current (incorrect) output:
```
+------------------------------------------------------------------------+
|zipped |
+------------------------------------------------------------------------+
|[{a, a}, {b, b}, {c, c}, {a, a}, {b, b}, {c, c}, {a, a}, {b, b}, {c, c}]|
+------------------------------------------------------------------------+
```
And this is the correct output after fix:
```
+------------------------------------------------------------------------+
|zipped |
+------------------------------------------------------------------------+
|[{1, a}, {1, b}, {1, c}, {2, a}, {2, b}, {2, c}, {3, a}, {3, b}, {3, c}]|
+------------------------------------------------------------------------+
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added the new test in `DataFrameFunctionsSuite`.
Closes#32424 from maropu/pr31887.
Lead-authored-by: dsolow <dsolow@sayari.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: dmsolow <dsolow@sayarianalytics.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Added the following TreePattern enums:
- CREATE_NAMED_STRUCT
- EXTRACT_VALUE
- JSON_TO_STRUCT
- OUTER_REFERENCE
- AGGREGATE
- LOCAL_RELATION
- EXCEPT
- LIMIT
- WINDOW
Used them in the following rules:
- DecorrelateInnerQuery
- LimitPushDownThroughWindow
- OptimizeCsvJsonExprs
- PropagateEmptyRelation
- PullOutGroupingExpressions
- PushLeftSemiLeftAntiThroughJoin
- ReplaceExceptWithFilter
- RewriteDistinctAggregates
- SimplifyConditionalsInPredicate
- UnwrapCastInBinaryComparison
### Why are the changes needed?
Reduce the number of tree traversals and hence improve the query compilation latency.
### How was this patch tested?
Existing tests.
Closes#32421 from sigmod/opt.
Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
### What changes were proposed in this pull request?
In `StaticInvoke`, when result is nullable, don't box the return value if its type is primitive.
### Why are the changes needed?
It is unnecessary to apply boxing when the method return value is of primitive type, and it would hurt performance a lot if the method is simple. The check is done in `Invoke` but not in `StaticInvoke`.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added a UT.
Closes#32416 from sunchao/SPARK-35281.
Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
1. Extend Spark SQL parser to support parsing of:
- `INTERVAL YEAR TO MONTH` to `YearMonthIntervalType`
- `INTERVAL DAY TO SECOND` to `DayTimeIntervalType`
2. Assign new names to the ANSI interval types according to the SQL standard to be able to parse the names back by Spark SQL parser. Override the `typeName()` name of `YearMonthIntervalType`/`DayTimeIntervalType`.
### Why are the changes needed?
To be able to use new ANSI interval types in SQL. The SQL standard requires the types to be defined according to the rules:
```
<interval type> ::= INTERVAL <interval qualifier>
<interval qualifier> ::= <start field> TO <end field> | <single datetime field>
<start field> ::= <non-second primary datetime field> [ <left paren> <interval leading field precision> <right paren> ]
<end field> ::= <non-second primary datetime field> | SECOND [ <left paren> <interval fractional seconds precision> <right paren> ]
<primary datetime field> ::= <non-second primary datetime field | SECOND
<non-second primary datetime field> ::= YEAR | MONTH | DAY | HOUR | MINUTE
<interval fractional seconds precision> ::= <unsigned integer>
<interval leading field precision> ::= <unsigned integer>
```
Currently, Spark SQL supports only `YEAR TO MONTH` and `DAY TO SECOND` as `<interval qualifier>`.
### Does this PR introduce _any_ user-facing change?
Should not since the types has not been released yet.
### How was this patch tested?
By running the affected tests such as:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "test:testOnly *ExpressionTypeCheckingSuite"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z windowFrameCoercion.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z literals.sql"
```
Closes#32409 from MaxGekk/parse-ansi-interval-types.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Support Cast string to day-seconds interval
### Why are the changes needed?
Users can cast day-second interval string to DayTimeIntervalType.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UT
Closes#32271 from AngersZhuuuu/SPARK-35112.
Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
This PR adds a new rule `PullOutGroupingExpressions` to pull out complex grouping expressions to a `Project` node under an `Aggregate`. These expressions are then referenced in both grouping expressions and aggregate expressions without aggregate functions to ensure that optimization rules don't change the aggregate expressions to invalid ones that no longer refer to any grouping expressions.
### Why are the changes needed?
If aggregate expressions (without aggregate functions) in an `Aggregate` node are complex then the `Optimizer` can optimize out grouping expressions from them and so making aggregate expressions invalid.
Here is a simple example:
```
SELECT not(t.id IS NULL) , count(*)
FROM t
GROUP BY t.id IS NULL
```
In this case the `BooleanSimplification` rule does this:
```
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.BooleanSimplification ===
!Aggregate [isnull(id#222)], [NOT isnull(id#222) AS (NOT (id IS NULL))#226, count(1) AS c#224L] Aggregate [isnull(id#222)], [isnotnull(id#222) AS (NOT (id IS NULL))#226, count(1) AS c#224L]
+- Project [value#219 AS id#222] +- Project [value#219 AS id#222]
+- LocalRelation [value#219] +- LocalRelation [value#219]
```
where `NOT isnull(id#222)` is optimized to `isnotnull(id#222)` and so it no longer refers to any grouping expression.
Before this PR:
```
== Optimized Logical Plan ==
Aggregate [isnull(id#222)], [isnotnull(id#222) AS (NOT (id IS NULL))#234, count(1) AS c#232L]
+- Project [value#219 AS id#222]
+- LocalRelation [value#219]
```
and running the query throws an error:
```
Couldn't find id#222 in [isnull(id#222)#230,count(1)#226L]
java.lang.IllegalStateException: Couldn't find id#222 in [isnull(id#222)#230,count(1)#226L]
```
After this PR:
```
== Optimized Logical Plan ==
Aggregate [_groupingexpression#233], [NOT _groupingexpression#233 AS (NOT (id IS NULL))#230, count(1) AS c#228L]
+- Project [isnull(value#219) AS _groupingexpression#233]
+- LocalRelation [value#219]
```
and the query works.
### Does this PR introduce _any_ user-facing change?
Yes, the query works.
### How was this patch tested?
Added new UT.
Closes#32396 from peter-toth/SPARK-34581-keep-grouping-expressions-2.
Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This patch fixes `Invoke` expression when the target object has more than one method with the given method name.
### Why are the changes needed?
`Invoke` will find out the method on the target object with given method name. If there are more than one method with the name, currently it is undeterministic which method will be used. We should add the condition of parameter number when finding the method.
### Does this PR introduce _any_ user-facing change?
Yes, fixed a bug when using `Invoke` on a object where more than one method with the given method name.
### How was this patch tested?
Unit test.
Closes#32404 from viirya/verify-invoke-param-len.
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?
This pr makes `CombineFilters` support non-deterministic expressions. For example:
```sql
spark.sql("CREATE TABLE t1(id INT, dt STRING) using parquet PARTITIONED BY (dt)")
spark.sql("CREATE VIEW v1 AS SELECT * FROM t1 WHERE dt NOT IN ('2020-01-01', '2021-01-01')")
spark.sql("SELECT * FROM v1 WHERE dt = '2021-05-01' AND rand() <= 0.01").explain()
```
Before this pr:
```
== Physical Plan ==
*(1) Filter (isnotnull(dt#1) AND ((dt#1 = 2021-05-01) AND (rand(-6723800298719475098) <= 0.01)))
+- *(1) ColumnarToRow
+- FileScan parquet default.t1[id#0,dt#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(0 paths)[], PartitionFilters: [NOT dt#1 IN (2020-01-01,2021-01-01)], PushedFilters: [], ReadSchema: struct<id:int>
```
After this pr:
```
== Physical Plan ==
*(1) Filter (rand(-2400509328955813273) <= 0.01)
+- *(1) ColumnarToRow
+- FileScan parquet default.t1[id#0,dt#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(0 paths)[], PartitionFilters: [isnotnull(dt#1), NOT dt#1 IN (2020-01-01,2021-01-01), (dt#1 = 2021-05-01)], PushedFilters: [], ReadSchema: struct<id:int>
```
### 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#32405 from wangyum/SPARK-35273.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
~~This PR aims to add a new AQE optimizer rule `DynamicJoinSelection`. Like other AQE partition number configs, this rule add a new broadcast threshold config `spark.sql.adaptive.autoBroadcastJoinThreshold`.~~
This PR amis to add a flag in `Statistics` to distinguish AQE stats or normal stats, so that we can make some sql configs isolation between AQE and normal.
### Why are the changes needed?
The main idea here is that make join config isolation between normal planner and aqe planner which shared the same code path.
Actually we do not very trust using the static stats to consider if it can build broadcast hash join. In our experience it's very common that Spark throw broadcast timeout or driver side OOM exception when execute a bit large plan. And due to braodcast join is not reversed which means if we covert join to braodcast hash join at first time, we(AQE) can not optimize it again, so it should make sense to decide if we can do broadcast at aqe side using different sql config.
### Does this PR introduce _any_ user-facing change?
Yes, a new config `spark.sql.adaptive.autoBroadcastJoinThreshold` added.
### How was this patch tested?
Add new test.
Closes#32391 from ulysses-you/SPARK-35264.
Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Support Cast string to year-month interval
Supported format as below
```
ANSI_STYLE, like
INTERVAL -'-10-1' YEAR TO MONTH
HIVE_STYLE like
10-1 or -10-1
Rules from the SQL standard about ANSI_STYLE:
<interval literal> ::=
INTERVAL [ <sign> ] <interval string> <interval qualifier>
<interval string> ::=
<quote> <unquoted interval string> <quote>
<unquoted interval string> ::=
[ <sign> ] { <year-month literal> | <day-time literal> }
<year-month literal> ::=
<years value> [ <minus sign> <months value> ]
| <months value>
<years value> ::=
<datetime value>
<months value> ::=
<datetime value>
<datetime value> ::=
<unsigned integer>
<unsigned integer> ::= <digit>...
```
### Why are the changes needed?
Support Cast string to year-month interval
### Does this PR introduce _any_ user-facing change?
User can cast year month interval string to YearMonthIntervalType
### How was this patch tested?
Added UT
Closes#32266 from AngersZhuuuu/SPARK-SPARK-35111.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
This PR proposes to enable the JSON datasources to write non-ascii characters as codepoints.
To enable/disable this feature, I introduce a new option `writeNonAsciiCharacterAsCodePoint` for JSON datasources.
### Why are the changes needed?
JSON specification allows codepoints as literal but Spark SQL's JSON datasources don't support the way to do it.
It's great if we can write non-ascii characters as codepoints, which is a platform neutral representation.
### Does this PR introduce _any_ user-facing change?
Yes. Users can write non-ascii characters as codepoints with JSON datasources.
### How was this patch tested?
New test.
Closes#32147 from sarutak/json-unicode-write.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Make sure we re-throw an exception that is not null.
### Why are the changes needed?
to be super safe
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
N/A
Closes#32387 from cloud-fan/minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-Authored-By: Chao Sun <sunchaoapple.com>
Co-Authored-By: Ryan Blue <rbluenetflix.com>
### What changes were proposed in this pull request?
This implements function resolution and evaluation for functions registered through V2 FunctionCatalog [SPARK-27658](https://issues.apache.org/jira/browse/SPARK-27658). In particular:
- Added documentation for how to define the "magic method" in `ScalarFunction`.
- Added a new expression `ApplyFunctionExpression` which evaluates input by delegating to `ScalarFunction.produceResult` method.
- added a new expression `V2Aggregator` which is a type of `TypedImperativeAggregate`. It's a wrapper of V2 `AggregateFunction` and mostly delegate methods to the implementation of the latter. It also uses plain Java serde for intermediate state.
- Added function resolution logic for `ScalarFunction` and `AggregateFunction` in `Analyzer`.
+ For `ScalarFunction` this checks if the magic method is implemented through Java reflection, and create a `Invoke` expression if so. Otherwise, it checks if the default `produceResult` is overridden. If so, it creates a `ApplyFunctionExpression` which evaluates through `InternalRow`. Otherwise an analysis exception is thrown.
+ For `AggregateFunction`, this checks if the `update` method is overridden. If so, it converts it to `V2Aggregator`. Otherwise an analysis exception is thrown similar to the case of `ScalarFunction`.
- Extended existing `InMemoryTableCatalog` to add the function catalog capability. Also renamed it to `InMemoryCatalog` since it no longer only covers tables.
**Note**: this currently can successfully detect whether a subclass overrides the default `produceResult` or `update` method from the parent interface **only for Java implementations**. It seems in Scala it's hard to differentiate whether a subclass overrides a default method from its parent interface. In this case, it will be a runtime error instead of analysis error.
A few TODOs:
- Extend `V2SessionCatalog` with function catalog. This seems a little tricky since API such V2 `FunctionCatalog`'s `loadFunction` is different from V1 `SessionCatalog`'s `lookupFunction`.
- Add magic method for `AggregateFunction`.
- Type coercion when looking up functions
### Why are the changes needed?
As V2 FunctionCatalog APIs are finalized, we should integrate it with function resolution and evaluation process so that they are actually useful.
### Does this PR introduce _any_ user-facing change?
Yes, now a function exposed through V2 FunctionCatalog can be analyzed and evaluated.
### How was this patch tested?
Added new unit tests.
Closes#32082 from sunchao/resolve-func-v2.
Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Co-authored-by: Chao Sun <sunchao@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add `ShuffledHashJoin` pattern check in `OptimizeSkewedJoin` so that we can optimize it.
### Why are the changes needed?
Currently, we have already supported all type of join through hint that make it easy to choose the join implementation.
We would choose `ShuffledHashJoin` if one table is not big but over the broadcast threshold. It's better that we can support optimize it in `OptimizeSkewedJoin`.
### Does this PR introduce _any_ user-facing change?
Probably yes, the execute plan in AQE mode may be changed.
### How was this patch tested?
Improve exists test in `AdaptiveQueryExecSuite`
Closes#32328 from ulysses-you/SPARK-35214.
Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
The UnsupportedOperationChecker shouldn't allow streaming-batch intersects. As described in the ticket, they can't actually be planned correctly, and even simple cases like the below will fail:
```
test("intersect") {
val input = MemoryStream[Long]
val df = input.toDS().intersect(spark.range(10).as[Long])
testStream(df) (
AddData(input, 1L),
CheckAnswer(1)
)
}
```
### Why are the changes needed?
Users will be confused by the cryptic errors produced from trying to run an invalid query plan.
### Does this PR introduce _any_ user-facing change?
Some queries which previously failed with a poor error will now fail with a better one.
### How was this patch tested?
modified unit test
Closes#32371 from jose-torres/ossthing.
Authored-by: Jose Torres <joseph.torres@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR updates the interpreted code path of invoke expressions, to unwrap the `InvocationTargetException`
### Why are the changes needed?
Make interpreted and codegen path consistent for invoke expressions.
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
new UT
Closes#32370 from cloud-fan/minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
In this PR, we add extract/date_part support for ANSI Intervals
The `extract` is an ANSI expression and `date_part` is NON-ANSI but exists as an equivalence for `extract`
#### expression
```
<extract expression> ::=
EXTRACT <left paren> <extract field> FROM <extract source> <right paren>
```
#### <extract field> for interval source
```
<primary datetime field> ::=
<non-second primary datetime field>
| SECOND
<non-second primary datetime field> ::=
YEAR
| MONTH
| DAY
| HOUR
| MINUTE
```
#### dataType
```
If <extract field> is a <primary datetime field> that does not specify SECOND or <extract field> is not a <primary datetime field>, then the declared type of the result is an implementation-defined exact numeric type with scale 0 (zero)
Otherwise, the declared type of the result is an implementation-defined exact numeric type with scale not less than the specified or implied <time fractional seconds precision> or <interval fractional seconds precision>, as appropriate, of the SECOND <primary datetime field> of the <extract source>.
```
### Why are the changes needed?
Subtask of ANSI Intervals Support
### Does this PR introduce _any_ user-facing change?
Yes
1. extract/date_part support ANSI intervals
2. for non-ansi intervals, the return type is changed from long to byte when extracting hours
### How was this patch tested?
new added tests
Closes#32351 from yaooqinn/SPARK-35091.
Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/32229 support ANSI SQL intervals by the aggregate function `avg`.
But have not treat that the input zero rows. so this will lead to:
```
Caused by: java.lang.ArithmeticException: / by zero
at com.google.common.math.LongMath.divide(LongMath.java:367)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1864)
at org.apache.spark.rdd.RDD.$anonfun$count$1(RDD.scala:1253)
at org.apache.spark.rdd.RDD.$anonfun$count$1$adapted(RDD.scala:1253)
at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2248)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:131)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
```
### Why are the changes needed?
Fix a bug.
### Does this PR introduce _any_ user-facing change?
No. Just new feature.
### How was this patch tested?
new tests.
Closes#32358 from beliefer/SPARK-34837-followup.
Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
Before this patch
```
scala> Seq(java.time.Period.ofMonths(Int.MinValue)).toDF("i").select($"i" / -1).show(false)
+-------------------------------------+
|(i / -1) |
+-------------------------------------+
|INTERVAL '-178956970-8' YEAR TO MONTH|
+-------------------------------------+
scala> Seq(java.time.Duration.of(Long.MinValue, java.time.temporal.ChronoUnit.MICROS)).toDF("i").select($"i" / -1).show(false)
+---------------------------------------------------+
|(i / -1) |
+---------------------------------------------------+
|INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND|
+---------------------------------------------------+
```
Wrong result of min ANSI interval division by -1, this pr fix this
### Why are the changes needed?
Fix bug
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UT
Closes#32314 from AngersZhuuuu/SPARK-35169.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Support YearMonthIntervalType and DayTimeIntervalType to extend ArrowColumnVector
### Why are the changes needed?
https://issues.apache.org/jira/browse/SPARK-35139
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
1. By checking coding style via:
$ ./dev/scalastyle
$ ./dev/lint-java
2. Run the test "ArrowWriterSuite"
Closes#32340 from Peng-Lei/SPARK-35139.
Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This is a re-proposal of https://github.com/apache/spark/pull/23163. Currently spark always requires a [local sort](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala#L188) before writing to output table with dynamic partition/bucket columns. The sort can be unnecessary if cardinality of partition/bucket values is small, and can be avoided by keeping multiple output writers concurrently.
This PR introduces a config `spark.sql.maxConcurrentOutputFileWriters` (which disables this feature by default), where user can tune the maximal number of concurrent writers. The config is needed here as we cannot keep arbitrary number of writers in task memory which can cause OOM (especially for Parquet/ORC vectorization writer).
The feature is to first use concurrent writers to write rows. If the number of writers exceeds the above config specified limit. Sort rest of rows and write rows one by one (See `DynamicPartitionDataConcurrentWriter.writeWithIterator()`).
In addition, interface `WriteTaskStatsTracker` and its implementation `BasicWriteTaskStatsTracker` are also changed because previously they are relying on the assumption that only one writer is active for writing dynamic partitions and bucketed table.
### Why are the changes needed?
Avoid the sort before writing output for dynamic partitioned query and bucketed table.
Help improve CPU and IO performance for these queries.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added unit test in `DataFrameReaderWriterSuite.scala`.
Closes#32198 from c21/writer.
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 group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/types`.
### 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#32244 from beliefer/SPARK-35060.
Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This patch proposes an improvement on nested column pruning if the pruning target is generator's output. Previously we disallow such case. This patch allows to prune on it if there is only one single nested column is accessed after `Generate`.
E.g., `df.select(explode($"items").as('item)).select($"item.itemId")`. As we only need `itemId` from `item`, we can prune other fields out and only keep `itemId`.
In this patch, we only address explode-like generators. We will address other generators in followups.
### Why are the changes needed?
This helps to extend the availability of nested column pruning.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test
Closes#31966 from viirya/SPARK-34638.
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?
This patch moves DS v2 custom metric classes to `org.apache.spark.sql.connector.metric` package. Moving `CustomAvgMetric` and `CustomSumMetric` to above package and make them as public java abstract class too.
### Why are the changes needed?
`CustomAvgMetric` and `CustomSumMetric` should be public APIs for developers to extend. As there are a few metric classes, we should put them together in one package.
### Does this PR introduce _any_ user-facing change?
No, dev only and they are not released yet.
### How was this patch tested?
Unit tests.
Closes#32348 from viirya/move-custom-metric-classes.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR makes `Sequence` expression supports ANSI intervals as step expression.
If the start and stop expression is `TimestampType,` then the step expression could select year-month or day-time interval.
If the start and stop expression is `DateType,` then the step expression must be year-month.
### Why are the changes needed?
Extends the function of `Sequence` expression.
### Does this PR introduce _any_ user-facing change?
'Yes'. Users could use ANSI intervals as step expression for `Sequence` expression.
### How was this patch tested?
New tests.
Closes#32311 from beliefer/SPARK-35088.
Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
Modifies the UpdateFields optimizer to fix correctness issues with certain nested and chained withField operations. Examples for recreating the issue are in the new unit tests as well as the JIRA issue.
### Why are the changes needed?
Certain withField patterns can cause Exceptions or even incorrect results. It appears to be a result of the additional UpdateFields optimization added in https://github.com/apache/spark/pull/29812. It traverses fieldOps in reverse order to take the last one per field, but this can cause nested structs to change order which leads to mismatches between the schema and the actual data. This updates the optimization to maintain the initial ordering of nested structs to match the generated schema.
### Does this PR introduce _any_ user-facing change?
It fixes exceptions and incorrect results for valid uses in the latest Spark release.
### How was this patch tested?
Added new unit tests for these edge cases.
Closes#32338 from Kimahriman/bug/optimize-with-fields.
Authored-by: Adam Binford <adamq43@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
In the test `"unsafe buffer with NO_CODEGEN"` of `MutableProjectionSuite`, fix unsafe buffer size calculation to be able to place all input fields without buffer overflow + meta-data.
### Why are the changes needed?
To make the test suite `MutableProjectionSuite` more stable.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running the affected test suite:
```
$ build/sbt "test:testOnly *MutableProjectionSuite"
```
Closes#32339 from MaxGekk/fix-buffer-overflow-MutableProjectionSuite.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
`CatalystTypeConverters` is useful when the type of the input data classes are not known statically (otherwise we can use `ExpressionEncoder`). However, the current `CatalystTypeConverters` requires you to know the datetime data class statically, which makes it hard to use.
This PR improves the `CatalystTypeConverters` for date/timestamp, to support the old and new Java time classes at the same time.
### Why are the changes needed?
Make `CatalystTypeConverters` easier to use.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
new test
Closes#32312 from cloud-fan/minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
Format empty grouping set exception in CUBE/ROLLUP
### Why are the changes needed?
Format empty grouping set exception in CUBE/ROLLUP
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Not need
Closes#32307 from AngersZhuuuu/SPARK-35201.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Added the following TreePattern enums:
- AND_OR
- BINARY_ARITHMETIC
- BINARY_COMPARISON
- CASE_WHEN
- CAST
- CONCAT
- COUNT
- IF
- LIKE_FAMLIY
- NOT
- NULL_CHECK
- UNARY_POSITIVE
- UPPER_OR_LOWER
Used them in the following rules:
- ConstantPropagation
- ReorderAssociativeOperator
- BooleanSimplification
- SimplifyBinaryComparison
- SimplifyCaseConversionExpressions
- SimplifyConditionals
- PushFoldableIntoBranches
- LikeSimplification
- NullPropagation
- SimplifyCasts
- RemoveDispensableExpressions
- CombineConcats
### Why are the changes needed?
Reduce the number of tree traversals and hence improve the query compilation latency.
### How was this patch tested?
Existing tests.
Closes#32280 from sigmod/expression.
Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
### What changes were proposed in this pull request?
For partial hash aggregation (code-gen path), we have two level of hash map for aggregation. First level is from `RowBasedHashMapGenerator`, which is computation faster compared to the second level from `UnsafeFixedWidthAggregationMap`. The introducing of two level hash map can help improve CPU performance of query as the first level hash map normally fits in hardware cache and has cheaper hash function for key lookup.
For final hash aggregation, we can also support two level of hash map, to improve query performance further.
The original two level of hash map code works for final aggregation mostly out of box. The major change here is to support testing fall back of final aggregation (see change related to `bitMaxCapacity` and `checkFallbackForGeneratedHashMap`).
Example:
An aggregation query:
```
spark.sql(
"""
|SELECT key, avg(value)
|FROM agg1
|GROUP BY key
""".stripMargin)
```
The generated code for final aggregation is [here](https://gist.github.com/c21/20c10cc8e2c7e561aafbe9b8da055242).
An aggregation query with testing fallback:
```
withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" -> "2, 3") {
spark.sql(
"""
|SELECT key, avg(value)
|FROM agg1
|GROUP BY key
""".stripMargin)
}
```
The generated code for final aggregation is [here](https://gist.github.com/c21/dabf176cbc18a5e2138bc0a29e81c878). Note the no more counter condition for first level fast map.
### Why are the changes needed?
Improve the CPU performance of hash aggregation query in general.
For `AggregateBenchmark."Aggregate w multiple keys"`, seeing query performance improved by 10%.
`codegen = T` means whole stage code-gen is enabled.
`hashmap = T` means two level maps is enabled for partial aggregation.
`finalhashmap = T` means two level maps is enabled for final aggregation.
```
Running benchmark: Aggregate w multiple keys
Running case: codegen = F
Stopped after 2 iterations, 8284 ms
Running case: codegen = T hashmap = F
Stopped after 2 iterations, 5424 ms
Running case: codegen = T hashmap = T finalhashmap = F
Stopped after 2 iterations, 4753 ms
Running case: codegen = T hashmap = T finalhashmap = T
Stopped after 2 iterations, 4508 ms
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU 2.40GHz
Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
codegen = F 3881 4142 370 5.4 185.1 1.0X
codegen = T hashmap = F 2701 2712 16 7.8 128.8 1.4X
codegen = T hashmap = T finalhashmap = F 2363 2377 19 8.9 112.7 1.6X
codegen = T hashmap = T finalhashmap = T 2252 2254 3 9.3 107.4 1.7X
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing unit test in `HashAggregationQuerySuite` and `HashAggregationQueryWithControlledFallbackSuite` already cover the test.
Closes#32242 from c21/agg.
Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Move the following classes:
- `InMemoryAtomicPartitionTable`
- `InMemoryPartitionTable`
- `InMemoryPartitionTableCatalog`
- `InMemoryTable`
- `InMemoryTableCatalog`
- `StagingInMemoryTableCatalog`
from `org.apache.spark.sql.connector` to `org.apache.spark.sql.connector.catalog`.
### Why are the changes needed?
These classes implement catalog related interfaces but reside in `org.apache.spark.sql.connector`. A more suitable place should be `org.apache.spark.sql.connector.catalog`.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
N/A
Closes#32302 from sunchao/SPARK-35195.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
Added the following TreePattern enums:
- DYNAMIC_PRUNING_SUBQUERY
- EXISTS_SUBQUERY
- IN_SUBQUERY
- LIST_SUBQUERY
- PLAN_EXPRESSION
- SCALAR_SUBQUERY
- FILTER
Used them in the following rules:
- ResolveSubquery
- UpdateOuterReferences
- OptimizeSubqueries
- RewritePredicateSubquery
- PullupCorrelatedPredicates
- RewriteCorrelatedScalarSubquery (not the rule itself but an internal transform call, the full support is in SPARK-35148)
- InsertAdaptiveSparkPlan
- PlanAdaptiveSubqueries
### Why are the changes needed?
Reduce the number of tree traversals and hence improve the query compilation latency.
### How was this patch tested?
Existing tests.
Closes#32247 from sigmod/subquery.
Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
### What changes were proposed in this pull request?
If the sign '-' inside of interval string, everything is fine after bb5459fb26:
```
spark-sql> SELECT INTERVAL '-178956970-8' YEAR TO MONTH;
-178956970-8
```
but the sign outside of interval string is not handled properly:
```
spark-sql> SELECT INTERVAL -'178956970-8' YEAR TO MONTH;
Error in query:
Error parsing interval year-month string: integer overflow(line 1, pos 16)
== SQL ==
SELECT INTERVAL -'178956970-8' YEAR TO MONTH
----------------^^^
```
This pr fix this issue
### Why are the changes needed?
Fix bug
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UT
Closes#32296 from AngersZhuuuu/SPARK-35187.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
This PR makes window frame could support `YearMonthIntervalType` and `DayTimeIntervalType`.
### Why are the changes needed?
Extend the function of window frame
### Does this PR introduce _any_ user-facing change?
Yes. Users could use `YearMonthIntervalType` or `DayTimeIntervalType` as the sort expression for window frame.
### How was this patch tested?
New tests
Closes#32294 from beliefer/SPARK-35110.
Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
Use transformAllExpressions instead of transformExpressionsDown in CombineConcats. The latter only transforms the root plan node.
### Why are the changes needed?
It allows CombineConcats to cover more cases where `concat` are not in the root plan node.
### How was this patch tested?
Unit test. The updated tests would fail without the code change.
Closes#32290 from sigmod/concat.
Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
PG and Oracle both support use CUBE/ROLLUP/GROUPING SETS in GROUPING SETS's grouping set as a sugar syntax.
![image](https://user-images.githubusercontent.com/46485123/114975588-139a1180-9eb7-11eb-8f53-498c1db934e0.png)
In this PR, we support it in Spark SQL too
### Why are the changes needed?
Keep consistent with PG and oracle
### Does this PR introduce _any_ user-facing change?
User can write grouping analytics like
```
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b));
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ());
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()));
```
### How was this patch tested?
Added Test
Closes#32201 from AngersZhuuuu/SPARK-35026.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
IntervalUtils.fromYearMonthString should handle Int.MinValue months correctly.
In current logic, just use `Math.addExact(Math.multiplyExact(years, 12), months)` to calculate negative total months will overflow when actual total months is Int.MinValue, this pr fixes this bug.
### Why are the changes needed?
IntervalUtils.fromYearMonthString should handle Int.MinValue months correctly
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UT
Closes#32281 from AngersZhuuuu/SPARK-35177.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
### What changes were proposed in this pull request?
It will remove `StructField` when [pruning nested columns](0f2c0b53e8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala (L28-L42)). For example:
```scala
spark.sql(
"""
|CREATE TABLE t1 (
| _col0 INT,
| _col1 STRING,
| _col2 STRUCT<c1: STRING, c2: STRING, c3: STRING, c4: BIGINT>)
|USING ORC
|""".stripMargin)
spark.sql("INSERT INTO t1 values(1, '2', struct('a', 'b', 'c', 10L))")
spark.sql("SELECT _col0, _col2.c1 FROM t1").show
```
Before this pr. The returned schema is: ``` `_col0` INT,`_col2` STRUCT<`c1`: STRING> ``` add it will throw exception:
```
java.lang.AssertionError: assertion failed: The given data schema struct<_col0:int,_col2:struct<c1:string>> has less fields than the actual ORC physical schema, no idea which columns were dropped, fail to read.
at scala.Predef$.assert(Predef.scala:223)
at org.apache.spark.sql.execution.datasources.orc.OrcUtils$.requestedColumnIds(OrcUtils.scala:160)
```
After this pr. The returned schema is: ``` `_col0` INT,`_col1` STRING,`_col2` STRUCT<`c1`: STRING> ```.
The finally schema is ``` `_col0` INT,`_col2` STRUCT<`c1`: STRING> ``` after the complete column pruning:
7a5647a93a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala (L208-L213)e64eb75aed/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala (L96-L97)
### Why are the changes needed?
Fix bug.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit test.
Closes#31993 from wangyum/SPARK-34897.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>