Skip to content

Conversation

@thepinetree
Copy link
Contributor

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. 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 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 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.

@dongjoon-hyun
Copy link
Member

Could you fix the compilation?

[error] /home/runner/work/spark/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala:787:27: not found: value toSQLId
[error]         "functionName" -> toSQLId("sequence"),
[error]                           

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM. Thank you, @thepinetree .
Merged to branch-3.5.

dongjoon-hyun pushed a commit that referenced this pull request Nov 17, 2023
### 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 <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
turboFei pushed a commit to turboFei/spark that referenced this pull request Nov 6, 2025
…che#360)

### 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 apache#43820 from thepinetree/spark-sequence-overflow-3.5.

Authored-by: Deepayan Patra <[email protected]>

Signed-off-by: Dongjoon Hyun <[email protected]>
Co-authored-by: Deepayan Patra <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants