https://github.com/apache/spark
Revision f3baf086acdf166445aef81181d13d4884d44e92 authored by Deepayan Patra on 17 November 2023, 21:17:43 UTC, committed by Dongjoon Hyun on 17 November 2023, 21:17:43 UTC
### What changes were proposed in this pull request?
Spark has a (long-standing) overflow bug in the `sequence` expression.

Consider the following operations:
```
spark.sql("CREATE TABLE foo (l LONG);")
spark.sql(s"INSERT INTO foo VALUES (${Long.MaxValue});")
spark.sql("SELECT sequence(0, l) FROM foo;").collect()
```

The result of these operations will be:
```
Array[org.apache.spark.sql.Row] = Array([WrappedArray()])
```
an unintended consequence of overflow.

The sequence is applied to values `0` and `Long.MaxValue` with a step size of `1` which uses a length computation defined [here](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3451). In this calculation, with `start = 0`, `stop = Long.MaxValue`, and `step = 1`, the calculated `len` overflows to `Long.MinValue`. The computation, in binary looks like:

```
  0111111111111111111111111111111111111111111111111111111111111111
- 0000000000000000000000000000000000000000000000000000000000000000
------------------------------------------------------------------
  0111111111111111111111111111111111111111111111111111111111111111
/ 0000000000000000000000000000000000000000000000000000000000000001
------------------------------------------------------------------
  0111111111111111111111111111111111111111111111111111111111111111
+ 0000000000000000000000000000000000000000000000000000000000000001
------------------------------------------------------------------
  1000000000000000000000000000000000000000000000000000000000000000
```

The following [check](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3454) passes as the negative `Long.MinValue` is still `<= MAX_ROUNDED_ARRAY_LENGTH`. The following cast to `toInt` uses this representation and [truncates the upper bits](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3457) resulting in an empty length of `0`.

Other overflows are similarly problematic.

This PR addresses the issue by checking numeric operations in the length computation for overflow.

### Why are the changes needed?
There is a correctness bug from overflow in the `sequence` expression.

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

### How was this patch tested?
Tests added in `CollectionExpressionsSuite.scala`.

Closes #43820 from thepinetree/spark-sequence-overflow-3.5.

Authored-by: Deepayan Patra <deepayan.patra@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
1 parent 9e492b7
History
Tip revision: f3baf086acdf166445aef81181d13d4884d44e92 authored by Deepayan Patra on 17 November 2023, 21:17:43 UTC
[SPARK-43393][SQL][3.5] Address sequence expression overflow bug
Tip revision: f3baf08

README.md

back to top