Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
3.5.0
Description
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.