---
.../advanced_configs.md | 1 +
docs/supported_ops.md | 1465 +++++++++--------
.../src/main/python/date_time_test.py | 35 +-
.../nvidia/spark/rapids/GpuOverrides.scala | 11 +
.../sql/rapids/datetimeExpressions.scala | 240 ++-
tools/generated_files/320/operatorsScore.csv | 1 +
tools/generated_files/320/supportedExprs.csv | 4 +
tools/generated_files/321/operatorsScore.csv | 1 +
tools/generated_files/321/supportedExprs.csv | 4 +
.../generated_files/321cdh/operatorsScore.csv | 1 +
.../generated_files/321cdh/supportedExprs.csv | 4 +
tools/generated_files/322/operatorsScore.csv | 1 +
tools/generated_files/322/supportedExprs.csv | 4 +
tools/generated_files/323/operatorsScore.csv | 1 +
tools/generated_files/323/supportedExprs.csv | 4 +
tools/generated_files/324/operatorsScore.csv | 1 +
tools/generated_files/324/supportedExprs.csv | 4 +
tools/generated_files/330/operatorsScore.csv | 1 +
tools/generated_files/330/supportedExprs.csv | 4 +
.../generated_files/330cdh/operatorsScore.csv | 1 +
.../generated_files/330cdh/supportedExprs.csv | 4 +
tools/generated_files/331/operatorsScore.csv | 1 +
tools/generated_files/331/supportedExprs.csv | 4 +
tools/generated_files/332/operatorsScore.csv | 1 +
tools/generated_files/332/supportedExprs.csv | 4 +
.../generated_files/332cdh/operatorsScore.csv | 1 +
.../generated_files/332cdh/supportedExprs.csv | 4 +
tools/generated_files/333/operatorsScore.csv | 1 +
tools/generated_files/333/supportedExprs.csv | 4 +
tools/generated_files/334/operatorsScore.csv | 1 +
tools/generated_files/334/supportedExprs.csv | 4 +
tools/generated_files/340/operatorsScore.csv | 1 +
tools/generated_files/340/supportedExprs.csv | 4 +
tools/generated_files/341/operatorsScore.csv | 1 +
tools/generated_files/341/supportedExprs.csv | 4 +
tools/generated_files/342/operatorsScore.csv | 1 +
tools/generated_files/342/supportedExprs.csv | 4 +
tools/generated_files/343/operatorsScore.csv | 1 +
tools/generated_files/343/supportedExprs.csv | 4 +
tools/generated_files/344/operatorsScore.csv | 1 +
tools/generated_files/344/supportedExprs.csv | 4 +
tools/generated_files/350/operatorsScore.csv | 1 +
tools/generated_files/350/supportedExprs.csv | 4 +
tools/generated_files/351/operatorsScore.csv | 1 +
tools/generated_files/351/supportedExprs.csv | 4 +
tools/generated_files/352/operatorsScore.csv | 1 +
tools/generated_files/352/supportedExprs.csv | 4 +
tools/generated_files/353/operatorsScore.csv | 1 +
tools/generated_files/353/supportedExprs.csv | 4 +
tools/generated_files/operatorsScore.csv | 1 +
tools/generated_files/supportedExprs.csv | 4 +
51 files changed, 1179 insertions(+), 688 deletions(-)
diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md
index 0298d4f0375..f3157b46099 100644
--- a/docs/additional-functionality/advanced_configs.md
+++ b/docs/additional-functionality/advanced_configs.md
@@ -311,6 +311,7 @@ Name | SQL Function(s) | Description | Default Value | Notes
spark.rapids.sql.expression.Minute|`minute`|Returns the minute component of the string/timestamp|true|None|
spark.rapids.sql.expression.MonotonicallyIncreasingID|`monotonically_increasing_id`|Returns monotonically increasing 64-bit integers|true|None|
spark.rapids.sql.expression.Month|`month`|Returns the month from a date or timestamp|true|None|
+spark.rapids.sql.expression.MonthsBetween|`months_between`|If `timestamp1` is later than `timestamp2`, then the result is positive. If `timestamp1` and `timestamp2` are on the same day of month, or both are the last day of month, time of day will be ignored. Otherwise, the difference is calculated based on 31 days per month, and rounded to 8 digits unless roundOff=false.|true|None|
spark.rapids.sql.expression.Multiply|`*`|Multiplication|true|None|
spark.rapids.sql.expression.Murmur3Hash|`hash`|Murmur3 hash operator|true|None|
spark.rapids.sql.expression.NaNvl|`nanvl`|Evaluates to `left` iff left is not NaN, `right` otherwise|true|None|
diff --git a/docs/supported_ops.md b/docs/supported_ops.md
index 60fc6ea9b9a..2fa11f8aa6e 100644
--- a/docs/supported_ops.md
+++ b/docs/supported_ops.md
@@ -11493,6 +11493,103 @@ are limited.
|
+MonthsBetween |
+`months_between` |
+If `timestamp1` is later than `timestamp2`, then the result is positive. If `timestamp1` and `timestamp2` are on the same day of month, or both are the last day of month, time of day will be ignored. Otherwise, the difference is calculated based on 31 days per month, and rounded to 8 digits unless roundOff=false. |
+None |
+project |
+timestamp1 |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+PS UTC is only supported TZ for TIMESTAMP |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+
+
+timestamp2 |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+PS UTC is only supported TZ for TIMESTAMP |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+
+
+round |
+PS Literal value only |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+
+
+result |
+ |
+ |
+ |
+ |
+ |
+ |
+S |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+
+
Multiply |
`*` |
Multiplication |
@@ -11637,6 +11734,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Murmur3Hash |
`hash` |
Murmur3 hash operator |
@@ -11762,34 +11887,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
NamedLambdaVariable |
|
A parameter to a higher order SQL function |
@@ -12041,6 +12138,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Or |
`or` |
Logical OR |
@@ -12185,34 +12310,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
ParseUrl |
`parse_url` |
Extracts a part from a URL |
@@ -12435,6 +12532,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
PosExplode |
`posexplode_outer`, `posexplode` |
Given an input array produces a sequence of rows for each value in the array |
@@ -12630,34 +12755,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
PreciseTimestampConversion |
|
Expression used internally to convert the TimestampType to Long and back without losing precision, i.e. in microseconds. Used in time windowing |
@@ -12952,6 +13049,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Quarter |
`quarter` |
Returns the quarter of the year for date, in the range 1 to 4 |
@@ -13077,34 +13202,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
RaiseError |
`raise_error` |
Throw an exception |
@@ -13355,6 +13452,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
RegExpExtractAll |
`regexp_extract_all` |
Extract all strings matching a regular expression corresponding to the regex group index |
@@ -13572,34 +13697,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
Remainder |
`%`, `mod` |
Remainder or modulo |
@@ -13776,6 +13873,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Rint |
`rint` |
Rounds up a double value to the nearest double equal to an integer |
@@ -13966,42 +14091,14 @@ are limited.
|
|
|
- |
- |
- |
- |
- |
- |
- |
- |
-
-
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
ScalaUDF |
@@ -14254,6 +14351,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
ShiftLeft |
`shiftleft` |
Bitwise shift left (<<) |
@@ -14402,34 +14527,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
ShiftRightUnsigned |
`shiftrightunsigned` |
Bitwise unsigned shift right (>>>) |
@@ -14653,6 +14750,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Sinh |
`sinh` |
Hyperbolic sine |
@@ -14802,34 +14927,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
SortArray |
`sort_array` |
Returns a sorted array with the input array and the ascending / descending order |
@@ -15057,6 +15154,34 @@ are limited.
NS |
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Sqrt |
`sqrt` |
Square root |
@@ -15229,34 +15354,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
StartsWith |
|
Starts with |
@@ -15502,6 +15599,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
StringLocate |
`locate`, `position` |
Substring search operator |
@@ -15670,58 +15795,30 @@ are limited.
|
|
|
- |
-
-
-result |
- |
- |
- |
- |
- |
- |
- |
- |
- |
-S |
- |
- |
- |
- |
- |
- |
- |
- |
- |
- |
-
-
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
+ |
+
+
+result |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+S |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
StringRepeat |
@@ -15895,6 +15992,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
StringSplit |
`split` |
Splits `str` around occurrences that match `regex` |
@@ -16089,34 +16214,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
StringTranslate |
`translate` |
StringTranslate operator |
@@ -16288,6 +16385,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
StringTrimLeft |
`ltrim` |
StringTrimLeft operator |
@@ -16487,34 +16612,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
Substring |
`substr`, `substring` |
Substring operator |
@@ -16709,6 +16806,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Subtract |
`-` |
Subtraction |
@@ -16951,34 +17076,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
Tanh |
`tanh` |
Hyperbolic tangent |
@@ -17151,6 +17248,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
ToDegrees |
`degrees` |
Converts radians to degrees |
@@ -17391,42 +17516,14 @@ are limited.
|
|
|
- |
- |
- |
- |
- |
- |
- |
- |
-
-
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
TransformKeys |
@@ -17577,6 +17674,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
UnaryMinus |
`negative` |
Negate a numeric value |
@@ -17801,34 +17926,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
UnboundedPreceding$ |
|
Special boundary for a window frame, indicating all rows preceding the current row |
@@ -17982,6 +18079,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Upper |
`ucase`, `upper` |
String uppercase operator |
@@ -18232,34 +18357,6 @@ are limited.
NS |
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
XxHash64 |
`xxhash64` |
xxhash64 hash operator |
@@ -18576,6 +18673,34 @@ are limited.
S |
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
ApproximatePercentile |
`approx_percentile`, `percentile_approx` |
Approximate percentile |
@@ -18766,34 +18891,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
Average |
`avg`, `mean` |
Average aggregate operator |
@@ -19084,6 +19181,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
CollectSet |
`collect_set` |
Collect a set of unique elements, not supported in reduction |
@@ -19229,34 +19354,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
Count |
`count` |
Count aggregate operator |
@@ -19547,6 +19644,34 @@ are limited.
NS |
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Last |
`last_value`, `last` |
last aggregate operator |
@@ -19692,34 +19817,6 @@ are limited.
NS |
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
Max |
`max` |
Max aggregate operator |
@@ -20009,6 +20106,34 @@ are limited.
NS |
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Min |
`min` |
Min aggregate operator |
@@ -20154,34 +20279,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
MinBy |
`min_by` |
MinBy aggregate operator. It may produce different results than CPU when multiple rows in a group have same minimum value in the ordering column and different associated values in the value column. |
@@ -20516,6 +20613,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
PivotFirst |
|
PivotFirst operator |
@@ -20660,34 +20785,6 @@ are limited.
NS |
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
StddevPop |
`stddev_pop` |
Aggregation computing population standard deviation |
@@ -20978,6 +21075,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Sum |
`sum` |
Sum aggregate operator |
@@ -21123,34 +21248,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
VariancePop |
`var_pop` |
Aggregation computing population variance |
@@ -21441,6 +21538,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
NormalizeNaNAndZero |
|
Normalize NaN and zero |
@@ -21520,34 +21645,6 @@ are limited.
NS |
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
HiveGenericUDF |
|
Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to get better performance |
diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py
index 5781f0b6df4..5a98e06fadc 100644
--- a/integration_tests/src/main/python/date_time_test.py
+++ b/integration_tests/src/main/python/date_time_test.py
@@ -17,7 +17,7 @@
from conftest import is_utc, is_supported_time_zone, get_test_tz
from data_gen import *
from datetime import date, datetime, timezone
-from marks import allow_non_gpu, datagen_overrides, disable_ansi_mode, ignore_order, incompat, tz_sensitive_test
+from marks import allow_non_gpu, approximate_float, datagen_overrides, disable_ansi_mode, ignore_order, incompat, tz_sensitive_test
from pyspark.sql.types import *
from spark_session import with_cpu_session, is_before_spark_330, is_before_spark_350
import pyspark.sql.functions as f
@@ -139,6 +139,39 @@ def test_datediff(data_gen):
hms_fallback = ['ProjectExec'] if not is_supported_time_zone() else []
+@allow_non_gpu(*hms_fallback)
+def test_months_between():
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : binary_op_df(spark, timestamp_gen).selectExpr('months_between(a, b, false)'))
+
+@allow_non_gpu(*hms_fallback)
+def test_months_between_first_day():
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('months_between(a, timestamp"2024-01-01", false)'))
+
+@allow_non_gpu(*hms_fallback)
+def test_months_between_last_day():
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('months_between(a, timestamp"2023-12-31", false)'))
+
+@allow_non_gpu(*hms_fallback)
+@approximate_float()
+def test_months_between_round():
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : binary_op_df(spark, timestamp_gen).selectExpr('months_between(a, b, true)'))
+
+@allow_non_gpu(*hms_fallback)
+@approximate_float()
+def test_months_between_first_day_round():
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('months_between(a, timestamp"2024-01-01", true)'))
+
+@allow_non_gpu(*hms_fallback)
+@approximate_float()
+def test_months_between_last_day_round():
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('months_between(a, timestamp"2023-12-31", true)'))
+
@allow_non_gpu(*hms_fallback)
def test_hour():
assert_gpu_and_cpu_are_equal_collect(
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
index 19aff51664a..bdeebaabbfc 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
@@ -1810,6 +1810,17 @@ object GpuOverrides extends Logging {
TypeSig.lit(TypeEnum.STRING))),
(a, conf, p, r) => new ToUTCTimestampExprMeta(a, conf, p, r)
),
+ expr[MonthsBetween](
+ "If `timestamp1` is later than `timestamp2`, then the result " +
+ "is positive. If `timestamp1` and `timestamp2` are on the same day of month, or both " +
+ "are the last day of month, time of day will be ignored. Otherwise, the difference is " +
+ "calculated based on 31 days per month, and rounded to 8 digits unless roundOff=false.",
+ ExprChecks.projectOnly(TypeSig.DOUBLE, TypeSig.DOUBLE,
+ Seq(ParamCheck("timestamp1", TypeSig.TIMESTAMP, TypeSig.TIMESTAMP),
+ ParamCheck("timestamp2", TypeSig.TIMESTAMP, TypeSig.TIMESTAMP),
+ ParamCheck("round", TypeSig.lit(TypeEnum.BOOLEAN), TypeSig.BOOLEAN))),
+ (a, conf, p, r) => new MonthsBetweenExprMeta(a, conf, p, r)
+ ),
expr[Pmod](
"Pmod",
// Decimal support disabled https://github.com/NVIDIA/spark-rapids/issues/7553
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
index 2cac7dcf024..8ed4c50ac3b 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
@@ -19,15 +19,18 @@ package org.apache.spark.sql.rapids
import java.time.ZoneId
import java.util.concurrent.TimeUnit
-import ai.rapids.cudf.{BinaryOp, CaptureGroups, ColumnVector, ColumnView, DType, RegexProgram, Scalar}
+import scala.concurrent.duration.DAYS
+
+import ai.rapids.cudf.{BinaryOp, CaptureGroups, ColumnVector, ColumnView, DateTimeRoundingFrequency, DType, RegexProgram, Scalar}
import com.nvidia.spark.rapids.{BinaryExprMeta, BoolUtils, DataFromReplacementRule, DateUtils, GpuBinaryExpression, GpuBinaryExpressionArgsAnyScalar, GpuCast, GpuColumnVector, GpuExpression, GpuOverrides, GpuScalar, GpuUnaryExpression, RapidsConf, RapidsMeta}
import com.nvidia.spark.rapids.Arm._
+import com.nvidia.spark.rapids.ExprMeta
import com.nvidia.spark.rapids.GpuOverrides.{extractStringLit, getTimeParserPolicy}
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.jni.GpuTimeZoneDB
-import com.nvidia.spark.rapids.shims.{NullIntolerantShim, ShimBinaryExpression}
+import com.nvidia.spark.rapids.shims.{NullIntolerantShim, ShimBinaryExpression, ShimExpression}
-import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, FromUnixTime, FromUTCTimestamp, ImplicitCastInputTypes, TimeZoneAwareExpression, ToUTCTimestamp}
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, FromUnixTime, FromUTCTimestamp, ImplicitCastInputTypes, MonthsBetween, TimeZoneAwareExpression, ToUTCTimestamp}
import org.apache.spark.sql.catalyst.util.DateTimeConstants
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
@@ -1208,6 +1211,237 @@ case class GpuToUTCTimestamp(
}
}
+class MonthsBetweenExprMeta(expr: MonthsBetween,
+ override val conf: RapidsConf,
+ override val parent: Option[RapidsMeta[_, _, _]],
+ rule: DataFromReplacementRule)
+ extends ExprMeta[MonthsBetween](expr, conf, parent, rule) {
+
+ override def isTimeZoneSupported = true
+
+ override def convertToGpu(): GpuExpression = {
+ val gpuChildren = childExprs.map(_.convertToGpu())
+ assert(gpuChildren.length == 3)
+ GpuMonthsBetween(gpuChildren(0), gpuChildren(1), gpuChildren(2), expr.timeZoneId)
+ }
+}
+
+object GpuMonthsBetween {
+ val UTC = GpuOverrides.UTC_TIMEZONE_ID
+
+ /**
+ * Convert the given timestamp in UTC to a specific time zone and close the original input.
+ * @param micros the timestamp in micros to convert
+ * @param normalizedZoneId the time zone to convert it to. Note that this should have
+ * already been normalized.
+ * @return the converted timestamp.
+ */
+ private def convertToZoneAndClose(micros: GpuColumnVector,
+ normalizedZoneId: ZoneId): ColumnVector = {
+ withResource(micros) { _ =>
+ if (normalizedZoneId.equals(UTC)) {
+ micros.getBase.incRefCount()
+ } else {
+ GpuTimeZoneDB.fromUtcTimestampToTimestamp(micros.getBase, normalizedZoneId)
+ }
+ }
+ }
+
+ private def calcMonths(converted: ColumnVector): ColumnVector = {
+ val yearInMonths = withResource(converted.year()) { year =>
+ withResource(Scalar.fromInt(12)) { monthsPerYear =>
+ year.mul(monthsPerYear)
+ }
+ }
+ withResource(yearInMonths) { _ =>
+ withResource(converted.month()) { month =>
+ yearInMonths.add(month)
+ }
+ }
+ }
+
+ /**
+ * When a timestamp is truncated to a month, calculate how many months are different
+ * between the two timestamps.
+ * @param converted1 the first timestamp (in the desired time zone)
+ * @param converted2 the second timestamp (in the desired time zone)
+ * @return the number of months different as a float64
+ */
+ private def calcMonthDiff(converted1: ColumnVector, converted2: ColumnVector): ColumnVector = {
+ withResource(calcMonths(converted1)) { months1 =>
+ withResource(calcMonths(converted2)) { months2 =>
+ months1.sub(months2, DType.FLOAT64)
+ }
+ }
+ }
+
+ private def isLastDayOfTheMonth(converted: ColumnVector, day: ColumnVector): ColumnVector = {
+ val lastDay = withResource(converted.lastDayOfMonth()) { ldm =>
+ ldm.day()
+ }
+ withResource(lastDay) { _ =>
+ lastDay.equalTo(day)
+ }
+ }
+
+ private def calcSecondsInDay(converted: ColumnVector): ColumnVector = {
+ // Find the number of seconds that are not counted for in a day
+
+ // find the micros over by finding the part that is not days
+ val microsInDay = withResource(converted.dateTimeFloor(DateTimeRoundingFrequency.DAY)) { days =>
+ // But we cannot subtract timestamps directly. They are both micros
+ assert(days.getType == DType.TIMESTAMP_MICROSECONDS)
+ assert(converted.getType == DType.TIMESTAMP_MICROSECONDS)
+ withResource(days.bitCastTo(DType.INT64)) { longDays =>
+ withResource(converted.bitCastTo(DType.INT64)) { longConverted =>
+ longConverted.sub(longDays)
+ }
+ }
+ }
+
+ // Then convert that to seconds (java does not round so we can be simple about it)
+ withResource(microsInDay) { _ =>
+ withResource(Scalar.fromLong(DateTimeConstants.MICROS_PER_SECOND)) { mps =>
+ microsInDay.div(mps, DType.INT64)
+ }
+ }
+ }
+
+ /**
+ * In Spark if both dates have the same day of the month, or if both are
+ * the end of the month then we ignore diffs for days and below, otherwise
+ * we need to calculate that partial part of the month.
+ *
+ * @param converted1 the first timestamp (in the desired time zone)
+ * @param converted2 the second timestamp (in the desired time zone)
+ * @return a boolean column where true is return just the whole number months diff
+ * and false is return the diff with days/time taken into account.
+ */
+ private def calcJustMonth(converted1: ColumnVector,
+ converted2: ColumnVector): ColumnVector = {
+ withResource(converted1.day()) { dayOfMonth1 =>
+ withResource(converted2.day()) { dayOfMonth2 =>
+ val bothLastDay = withResource(isLastDayOfTheMonth(converted1, dayOfMonth1)) { isLastDay1 =>
+ withResource(isLastDayOfTheMonth(converted2, dayOfMonth2)) { isLastDay2 =>
+ isLastDay1.and(isLastDay2)
+ }
+ }
+ withResource(bothLastDay) { _ =>
+ withResource(dayOfMonth1.equalTo(dayOfMonth2)) { sameDayOfMonth =>
+ sameDayOfMonth.or(bothLastDay)
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Calculate the number of seconds that are different between the two timestamps
+ * ignoring the year and the month. This is because calcMonthDiff will have
+ * already calculated that part.
+ *
+ * @param converted1 the first timestamp (in the desired time zone)
+ * @param converted2 the second timestamp (in the desired time zone)
+ * @return an INT64 column containing the diff in seconds.
+ */
+ private def calcSecondsDiff(converted1: ColumnVector,
+ converted2: ColumnVector): ColumnVector = {
+ // In theory, we could go directly to seconds in the month, but there
+ // may be some overflow issues according to Spark. Also,
+ // CUDF does not have a way to floor a timestamp to MONTHS, so it would
+ // be a two-step process anyway.
+ val daysDiffAsSeconds = withResource(converted1.day()) { day1 =>
+ withResource(converted2.day()) { day2 =>
+ withResource(day1.sub(day2)) { daysDiff =>
+ withResource(Scalar.fromLong(DateTimeConstants.SECONDS_PER_DAY)) { secsPerDay =>
+ daysDiff.mul(secsPerDay)
+ }
+ }
+ }
+ }
+ withResource(daysDiffAsSeconds) { _ =>
+ val secsInDayDiff = withResource(calcSecondsInDay(converted1)) { sid1 =>
+ withResource(calcSecondsInDay(converted2)) { sid2 =>
+ sid1.sub(sid2)
+ }
+ }
+ withResource(secsInDayDiff) { _ =>
+ daysDiffAsSeconds.add(secsInDayDiff)
+ }
+ }
+ }
+}
+
+case class GpuMonthsBetween(ts1: Expression,
+ ts2: Expression,
+ roundOff: Expression,
+ timeZoneId: Option[String] = None) extends GpuExpression
+ with ShimExpression with TimeZoneAwareExpression with ImplicitCastInputTypes
+ with NullIntolerantShim {
+ import GpuMonthsBetween._
+
+ override def columnarEval(batch: ColumnarBatch): GpuColumnVector = {
+ val needsRoundOff = withResourceIfAllowed(roundOff.columnarEvalAny(batch)) {
+ case s: GpuScalar if (s.isValid) => Some(s.getBase.getBoolean)
+ case _: GpuScalar => None
+ case other =>
+ throw new IllegalArgumentException(s"Only literal roundoff values are supported $other")
+ }
+ if (needsRoundOff.isEmpty) {
+ // Special case so we always return null for this.
+ withResource(Scalar.fromNull(DType.FLOAT64)) { s =>
+ closeOnExcept(ColumnVector.fromScalar(s, batch.numRows())) { result =>
+ return GpuColumnVector.from(result, dataType)
+ }
+ }
+ }
+
+ val zoneId = timeZoneId.map(s => ZoneId.of(s).normalized()).getOrElse(UTC)
+ withResource(convertToZoneAndClose(ts1.columnarEval(batch), zoneId)) { converted1 =>
+ withResource(convertToZoneAndClose(ts2.columnarEval(batch), zoneId)) { converted2 =>
+ withResource(calcMonthDiff(converted1, converted2)) { monthDiff =>
+ withResource(calcJustMonth(converted1, converted2)) { justMonthDiff =>
+ withResource(calcSecondsDiff(converted1, converted2)) { secondsDiff =>
+ val partialMonth = withResource(Scalar.fromDouble(DAYS.toSeconds(31))) {
+ secondsInMonth =>
+ secondsDiff.trueDiv(secondsInMonth)
+ }
+ val roundedPartialMonth = if (needsRoundOff.get) {
+ withResource(partialMonth) { _ =>
+ partialMonth.round(8)
+ }
+ } else {
+ partialMonth
+ }
+ val diff = withResource(roundedPartialMonth) { _ =>
+ roundedPartialMonth.add(monthDiff)
+ }
+ withResource(diff) { _ =>
+ GpuColumnVector.from(justMonthDiff.ifElse(monthDiff, diff), dataType)
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+
+ override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
+ copy(timeZoneId = Option(timeZoneId))
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, TimestampType, BooleanType)
+
+ override def dataType: DataType = DoubleType
+
+ override def foldable: Boolean = children.forall(_.foldable)
+
+ override def nullable: Boolean = children.exists(_.nullable)
+
+ override def children: Seq[Expression] = Seq(ts1, ts2, roundOff)
+
+ override def prettyName: String = "months_between"
+}
+
trait GpuDateMathBase extends GpuBinaryExpression with ExpectsInputTypes {
override def inputTypes: Seq[AbstractDataType] =
Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType))
diff --git a/tools/generated_files/320/operatorsScore.csv b/tools/generated_files/320/operatorsScore.csv
index 09056a7a285..19c999aa796 100644
--- a/tools/generated_files/320/operatorsScore.csv
+++ b/tools/generated_files/320/operatorsScore.csv
@@ -184,6 +184,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
Murmur3Hash,4
NaNvl,4
diff --git a/tools/generated_files/320/supportedExprs.csv b/tools/generated_files/320/supportedExprs.csv
index 6aeb2eccd8c..808d8fb4df3 100644
--- a/tools/generated_files/320/supportedExprs.csv
+++ b/tools/generated_files/320/supportedExprs.csv
@@ -371,6 +371,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/321/operatorsScore.csv b/tools/generated_files/321/operatorsScore.csv
index 09056a7a285..19c999aa796 100644
--- a/tools/generated_files/321/operatorsScore.csv
+++ b/tools/generated_files/321/operatorsScore.csv
@@ -184,6 +184,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
Murmur3Hash,4
NaNvl,4
diff --git a/tools/generated_files/321/supportedExprs.csv b/tools/generated_files/321/supportedExprs.csv
index 6aeb2eccd8c..808d8fb4df3 100644
--- a/tools/generated_files/321/supportedExprs.csv
+++ b/tools/generated_files/321/supportedExprs.csv
@@ -371,6 +371,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/321cdh/operatorsScore.csv b/tools/generated_files/321cdh/operatorsScore.csv
index 09056a7a285..19c999aa796 100644
--- a/tools/generated_files/321cdh/operatorsScore.csv
+++ b/tools/generated_files/321cdh/operatorsScore.csv
@@ -184,6 +184,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
Murmur3Hash,4
NaNvl,4
diff --git a/tools/generated_files/321cdh/supportedExprs.csv b/tools/generated_files/321cdh/supportedExprs.csv
index 6aeb2eccd8c..808d8fb4df3 100644
--- a/tools/generated_files/321cdh/supportedExprs.csv
+++ b/tools/generated_files/321cdh/supportedExprs.csv
@@ -371,6 +371,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/322/operatorsScore.csv b/tools/generated_files/322/operatorsScore.csv
index 09056a7a285..19c999aa796 100644
--- a/tools/generated_files/322/operatorsScore.csv
+++ b/tools/generated_files/322/operatorsScore.csv
@@ -184,6 +184,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
Murmur3Hash,4
NaNvl,4
diff --git a/tools/generated_files/322/supportedExprs.csv b/tools/generated_files/322/supportedExprs.csv
index 6aeb2eccd8c..808d8fb4df3 100644
--- a/tools/generated_files/322/supportedExprs.csv
+++ b/tools/generated_files/322/supportedExprs.csv
@@ -371,6 +371,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/323/operatorsScore.csv b/tools/generated_files/323/operatorsScore.csv
index 09056a7a285..19c999aa796 100644
--- a/tools/generated_files/323/operatorsScore.csv
+++ b/tools/generated_files/323/operatorsScore.csv
@@ -184,6 +184,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
Murmur3Hash,4
NaNvl,4
diff --git a/tools/generated_files/323/supportedExprs.csv b/tools/generated_files/323/supportedExprs.csv
index 6aeb2eccd8c..808d8fb4df3 100644
--- a/tools/generated_files/323/supportedExprs.csv
+++ b/tools/generated_files/323/supportedExprs.csv
@@ -371,6 +371,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/324/operatorsScore.csv b/tools/generated_files/324/operatorsScore.csv
index 09056a7a285..19c999aa796 100644
--- a/tools/generated_files/324/operatorsScore.csv
+++ b/tools/generated_files/324/operatorsScore.csv
@@ -184,6 +184,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
Murmur3Hash,4
NaNvl,4
diff --git a/tools/generated_files/324/supportedExprs.csv b/tools/generated_files/324/supportedExprs.csv
index 6aeb2eccd8c..808d8fb4df3 100644
--- a/tools/generated_files/324/supportedExprs.csv
+++ b/tools/generated_files/324/supportedExprs.csv
@@ -371,6 +371,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/330/operatorsScore.csv b/tools/generated_files/330/operatorsScore.csv
index b2a85a45dcb..e5978fb9f1a 100644
--- a/tools/generated_files/330/operatorsScore.csv
+++ b/tools/generated_files/330/operatorsScore.csv
@@ -190,6 +190,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/330/supportedExprs.csv b/tools/generated_files/330/supportedExprs.csv
index aa3db4be4ab..fcea9c8cb40 100644
--- a/tools/generated_files/330/supportedExprs.csv
+++ b/tools/generated_files/330/supportedExprs.csv
@@ -380,6 +380,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/330cdh/operatorsScore.csv b/tools/generated_files/330cdh/operatorsScore.csv
index b2a85a45dcb..e5978fb9f1a 100644
--- a/tools/generated_files/330cdh/operatorsScore.csv
+++ b/tools/generated_files/330cdh/operatorsScore.csv
@@ -190,6 +190,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/330cdh/supportedExprs.csv b/tools/generated_files/330cdh/supportedExprs.csv
index aa3db4be4ab..fcea9c8cb40 100644
--- a/tools/generated_files/330cdh/supportedExprs.csv
+++ b/tools/generated_files/330cdh/supportedExprs.csv
@@ -380,6 +380,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/331/operatorsScore.csv b/tools/generated_files/331/operatorsScore.csv
index ca40757f61a..b988344e702 100644
--- a/tools/generated_files/331/operatorsScore.csv
+++ b/tools/generated_files/331/operatorsScore.csv
@@ -191,6 +191,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/331/supportedExprs.csv b/tools/generated_files/331/supportedExprs.csv
index 4d53f6b86f0..4eccb898337 100644
--- a/tools/generated_files/331/supportedExprs.csv
+++ b/tools/generated_files/331/supportedExprs.csv
@@ -382,6 +382,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/332/operatorsScore.csv b/tools/generated_files/332/operatorsScore.csv
index ca40757f61a..b988344e702 100644
--- a/tools/generated_files/332/operatorsScore.csv
+++ b/tools/generated_files/332/operatorsScore.csv
@@ -191,6 +191,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/332/supportedExprs.csv b/tools/generated_files/332/supportedExprs.csv
index 4d53f6b86f0..4eccb898337 100644
--- a/tools/generated_files/332/supportedExprs.csv
+++ b/tools/generated_files/332/supportedExprs.csv
@@ -382,6 +382,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/332cdh/operatorsScore.csv b/tools/generated_files/332cdh/operatorsScore.csv
index ca40757f61a..b988344e702 100644
--- a/tools/generated_files/332cdh/operatorsScore.csv
+++ b/tools/generated_files/332cdh/operatorsScore.csv
@@ -191,6 +191,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/332cdh/supportedExprs.csv b/tools/generated_files/332cdh/supportedExprs.csv
index 4d53f6b86f0..4eccb898337 100644
--- a/tools/generated_files/332cdh/supportedExprs.csv
+++ b/tools/generated_files/332cdh/supportedExprs.csv
@@ -382,6 +382,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/333/operatorsScore.csv b/tools/generated_files/333/operatorsScore.csv
index ca40757f61a..b988344e702 100644
--- a/tools/generated_files/333/operatorsScore.csv
+++ b/tools/generated_files/333/operatorsScore.csv
@@ -191,6 +191,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/333/supportedExprs.csv b/tools/generated_files/333/supportedExprs.csv
index 4d53f6b86f0..4eccb898337 100644
--- a/tools/generated_files/333/supportedExprs.csv
+++ b/tools/generated_files/333/supportedExprs.csv
@@ -382,6 +382,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/334/operatorsScore.csv b/tools/generated_files/334/operatorsScore.csv
index ca40757f61a..b988344e702 100644
--- a/tools/generated_files/334/operatorsScore.csv
+++ b/tools/generated_files/334/operatorsScore.csv
@@ -191,6 +191,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/334/supportedExprs.csv b/tools/generated_files/334/supportedExprs.csv
index 4d53f6b86f0..4eccb898337 100644
--- a/tools/generated_files/334/supportedExprs.csv
+++ b/tools/generated_files/334/supportedExprs.csv
@@ -382,6 +382,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/340/operatorsScore.csv b/tools/generated_files/340/operatorsScore.csv
index 91b7bad076b..b1e9198e58b 100644
--- a/tools/generated_files/340/operatorsScore.csv
+++ b/tools/generated_files/340/operatorsScore.csv
@@ -193,6 +193,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/340/supportedExprs.csv b/tools/generated_files/340/supportedExprs.csv
index 77da153357f..80bc405b058 100644
--- a/tools/generated_files/340/supportedExprs.csv
+++ b/tools/generated_files/340/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/341/operatorsScore.csv b/tools/generated_files/341/operatorsScore.csv
index 91b7bad076b..b1e9198e58b 100644
--- a/tools/generated_files/341/operatorsScore.csv
+++ b/tools/generated_files/341/operatorsScore.csv
@@ -193,6 +193,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/341/supportedExprs.csv b/tools/generated_files/341/supportedExprs.csv
index 77da153357f..80bc405b058 100644
--- a/tools/generated_files/341/supportedExprs.csv
+++ b/tools/generated_files/341/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/342/operatorsScore.csv b/tools/generated_files/342/operatorsScore.csv
index 91b7bad076b..b1e9198e58b 100644
--- a/tools/generated_files/342/operatorsScore.csv
+++ b/tools/generated_files/342/operatorsScore.csv
@@ -193,6 +193,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/342/supportedExprs.csv b/tools/generated_files/342/supportedExprs.csv
index 77da153357f..80bc405b058 100644
--- a/tools/generated_files/342/supportedExprs.csv
+++ b/tools/generated_files/342/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/343/operatorsScore.csv b/tools/generated_files/343/operatorsScore.csv
index 91b7bad076b..b1e9198e58b 100644
--- a/tools/generated_files/343/operatorsScore.csv
+++ b/tools/generated_files/343/operatorsScore.csv
@@ -193,6 +193,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/343/supportedExprs.csv b/tools/generated_files/343/supportedExprs.csv
index 77da153357f..80bc405b058 100644
--- a/tools/generated_files/343/supportedExprs.csv
+++ b/tools/generated_files/343/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/344/operatorsScore.csv b/tools/generated_files/344/operatorsScore.csv
index 91b7bad076b..b1e9198e58b 100644
--- a/tools/generated_files/344/operatorsScore.csv
+++ b/tools/generated_files/344/operatorsScore.csv
@@ -193,6 +193,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/344/supportedExprs.csv b/tools/generated_files/344/supportedExprs.csv
index 77da153357f..80bc405b058 100644
--- a/tools/generated_files/344/supportedExprs.csv
+++ b/tools/generated_files/344/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/350/operatorsScore.csv b/tools/generated_files/350/operatorsScore.csv
index e6c3269b115..3b0b82d58bf 100644
--- a/tools/generated_files/350/operatorsScore.csv
+++ b/tools/generated_files/350/operatorsScore.csv
@@ -194,6 +194,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/350/supportedExprs.csv b/tools/generated_files/350/supportedExprs.csv
index cbd9627125a..f45289388fc 100644
--- a/tools/generated_files/350/supportedExprs.csv
+++ b/tools/generated_files/350/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/351/operatorsScore.csv b/tools/generated_files/351/operatorsScore.csv
index e6c3269b115..3b0b82d58bf 100644
--- a/tools/generated_files/351/operatorsScore.csv
+++ b/tools/generated_files/351/operatorsScore.csv
@@ -194,6 +194,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/351/supportedExprs.csv b/tools/generated_files/351/supportedExprs.csv
index cbd9627125a..f45289388fc 100644
--- a/tools/generated_files/351/supportedExprs.csv
+++ b/tools/generated_files/351/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/352/operatorsScore.csv b/tools/generated_files/352/operatorsScore.csv
index e6c3269b115..3b0b82d58bf 100644
--- a/tools/generated_files/352/operatorsScore.csv
+++ b/tools/generated_files/352/operatorsScore.csv
@@ -194,6 +194,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/352/supportedExprs.csv b/tools/generated_files/352/supportedExprs.csv
index cbd9627125a..f45289388fc 100644
--- a/tools/generated_files/352/supportedExprs.csv
+++ b/tools/generated_files/352/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/353/operatorsScore.csv b/tools/generated_files/353/operatorsScore.csv
index e6c3269b115..3b0b82d58bf 100644
--- a/tools/generated_files/353/operatorsScore.csv
+++ b/tools/generated_files/353/operatorsScore.csv
@@ -194,6 +194,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/353/supportedExprs.csv b/tools/generated_files/353/supportedExprs.csv
index cbd9627125a..f45289388fc 100644
--- a/tools/generated_files/353/supportedExprs.csv
+++ b/tools/generated_files/353/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/operatorsScore.csv b/tools/generated_files/operatorsScore.csv
index 09056a7a285..19c999aa796 100644
--- a/tools/generated_files/operatorsScore.csv
+++ b/tools/generated_files/operatorsScore.csv
@@ -184,6 +184,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
Murmur3Hash,4
NaNvl,4
diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv
index 6aeb2eccd8c..808d8fb4df3 100644
--- a/tools/generated_files/supportedExprs.csv
+++ b/tools/generated_files/supportedExprs.csv
@@ -371,6 +371,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
From d3cda269a2261f7301d7f17334ba5d048bd17354 Mon Sep 17 00:00:00 2001
From: Jason Lowe
Date: Thu, 21 Nov 2024 00:19:59 -0600
Subject: [PATCH 071/103] Update to Spark 4.0 changing signature of
SupportsV1Write.writeWithV1 (#11739)
* Update to Spark 4.0 changing signature of SupportsV1Write.writeWithV1
Signed-off-by: Jason Lowe
* Update Spark 4 tools support files
---------
Signed-off-by: Jason Lowe
---
.../scala/com/nvidia/spark/rapids/v1FallbackWriters.scala | 4 ++--
tools/generated_files/400/operatorsScore.csv | 1 +
tools/generated_files/400/supportedExprs.csv | 4 ++++
3 files changed, 7 insertions(+), 2 deletions(-)
diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala
index deff51b88f1..f9767504d99 100644
--- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala
+++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala
@@ -99,9 +99,9 @@ trait GpuV1FallbackWriters extends LeafV2CommandExec with SupportsV1Write with G
def write: V1Write
override def run(): Seq[InternalRow] = {
- val writtenRows = writeWithV1(write.toInsertableRelation)
+ writeWithV1(write.toInsertableRelation)
refreshCache()
- writtenRows
+ Nil
}
override def internalDoExecuteColumnar(): RDD[ColumnarBatch] = {
diff --git a/tools/generated_files/400/operatorsScore.csv b/tools/generated_files/400/operatorsScore.csv
index 50eae950857..53791a06705 100644
--- a/tools/generated_files/400/operatorsScore.csv
+++ b/tools/generated_files/400/operatorsScore.csv
@@ -194,6 +194,7 @@ MinBy,4
Minute,4
MonotonicallyIncreasingID,4
Month,4
+MonthsBetween,4
Multiply,4
MultiplyDTInterval,4
MultiplyYMInterval,4
diff --git a/tools/generated_files/400/supportedExprs.csv b/tools/generated_files/400/supportedExprs.csv
index 847fd98e6f1..890f959eab5 100644
--- a/tools/generated_files/400/supportedExprs.csv
+++ b/tools/generated_files/400/supportedExprs.csv
@@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N
MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA
From f2ea943dfd911bd92d3342f91e966ade3dcc5510 Mon Sep 17 00:00:00 2001
From: Renjie Liu
Date: Fri, 22 Nov 2024 08:42:09 +0800
Subject: [PATCH 072/103] Integrate with kudo (#11724)
---
.../delta/GpuOptimizeWriteExchangeExec.scala | 8 +-
.../src/main/python/hash_aggregate_test.py | 667 ++++++++++++------
.../src/main/python/join_test.py | 464 ++++++++----
.../src/main/python/repart_test.py | 100 ++-
.../nvidia/spark/rapids/GpuColumnVector.java | 49 ++
.../rapids/GpuColumnarBatchSerializer.scala | 274 ++++++-
.../com/nvidia/spark/rapids/GpuExec.scala | 4 +
.../spark/rapids/GpuShuffleCoalesceExec.scala | 116 ++-
.../rapids/GpuShuffledHashJoinExec.scala | 17 +-
.../rapids/GpuShuffledSizedHashJoinExec.scala | 35 +-
.../com/nvidia/spark/rapids/RapidsConf.scala | 9 +
.../RapidsShuffleInternalManagerBase.scala | 9 +-
.../GpuShuffleExchangeExecBase.scala | 6 +-
.../execution/GpuBroadcastHashJoinExec.scala | 4 +-
.../GpuBroadcastNestedLoopJoinExec.scala | 4 +-
.../GpuExecutorBroadcastHelper.scala | 16 +-
.../RapidsShuffleThreadedReaderSuite.scala | 2 +-
17 files changed, 1318 insertions(+), 466 deletions(-)
diff --git a/delta-lake/common/src/main/databricks/scala/org/apache/spark/sql/rapids/delta/GpuOptimizeWriteExchangeExec.scala b/delta-lake/common/src/main/databricks/scala/org/apache/spark/sql/rapids/delta/GpuOptimizeWriteExchangeExec.scala
index 1a9936ea808..0c212d6842a 100644
--- a/delta-lake/common/src/main/databricks/scala/org/apache/spark/sql/rapids/delta/GpuOptimizeWriteExchangeExec.scala
+++ b/delta-lake/common/src/main/databricks/scala/org/apache/spark/sql/rapids/delta/GpuOptimizeWriteExchangeExec.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2023, NVIDIA CORPORATION.
+ * Copyright (c) 2023-2024, NVIDIA CORPORATION.
*
* This file was derived from OptimizeWriteExchange.scala
* in the Delta Lake project at https://github.com/delta-io/delta
@@ -26,7 +26,7 @@ import scala.concurrent.Future
import scala.concurrent.duration.Duration
import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf
-import com.nvidia.spark.rapids.{GpuColumnarBatchSerializer, GpuExec, GpuMetric, GpuPartitioning, GpuRoundRobinPartitioning}
+import com.nvidia.spark.rapids.{GpuColumnarBatchSerializer, GpuExec, GpuMetric, GpuPartitioning, GpuRoundRobinPartitioning, RapidsConf}
import com.nvidia.spark.rapids.delta.RapidsDeltaSQLConf
import org.apache.spark.{MapOutputStatistics, ShuffleDependency}
@@ -98,7 +98,9 @@ case class GpuOptimizeWriteExchangeExec(
}
private lazy val serializer: Serializer =
- new GpuColumnarBatchSerializer(gpuLongMetric("dataSize"))
+ new GpuColumnarBatchSerializer(gpuLongMetric("dataSize"),
+ child.output.map(_.dataType).toArray,
+ RapidsConf.SHUFFLE_KUDO_SERIALIZER_ENABLED.get(child.conf))
@transient lazy val inputRDD: RDD[ColumnarBatch] = child.executeColumnar()
diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py
index 734b4dfb708..444e4131724 100644
--- a/integration_tests/src/main/python/hash_aggregate_test.py
+++ b/integration_tests/src/main/python/hash_aggregate_test.py
@@ -204,6 +204,8 @@
_decimal_gen_36_neg5 = DecimalGen(precision=36, scale=-5)
_decimal_gen_38_10 = DecimalGen(precision=38, scale=10)
+kudo_enabled_conf_key = "spark.rapids.shuffle.kudo.serializer.enabled"
+
def get_params(init_list, marked_params=[]):
"""
@@ -307,7 +309,8 @@ def get_params(init_list, marked_params=[]):
@nightly_gpu_mem_consuming_case
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('precision', [38, 37, 36, 35, 34, 33, 32, 31], ids=idfn)
-def test_hash_reduction_decimal_overflow_sum(precision):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_reduction_decimal_overflow_sum(precision, kudo_enabled):
constant = '9' * precision
count = pow(10, 38 - precision)
assert_gpu_and_cpu_are_equal_collect(
@@ -318,16 +321,20 @@ def test_hash_reduction_decimal_overflow_sum(precision):
# run out of memory in some setups. These should not happen in production, because
# we really are just doing a really bad job at multiplying to get this result so
# some optimizations are conspiring against us.
- conf = {'spark.rapids.sql.batchSizeBytes': '128m'})
+ conf = {'spark.rapids.sql.batchSizeBytes': '128m',
+ kudo_enabled_conf_key: kudo_enabled})
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [_longs_with_nulls], ids=idfn)
@pytest.mark.parametrize('override_split_until_size', [None, 1], ids=idfn)
@pytest.mark.parametrize('override_batch_size_bytes', [None, 1], ids=idfn)
-def test_hash_grpby_sum_count_action(data_gen, override_split_until_size, override_batch_size_bytes):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_grpby_sum_count_action(data_gen, override_split_until_size,
+ override_batch_size_bytes, kudo_enabled):
conf = {
- 'spark.rapids.sql.test.overrides.splitUntilSize': override_split_until_size
+ 'spark.rapids.sql.test.overrides.splitUntilSize': override_split_until_size,
+ kudo_enabled_conf_key: kudo_enabled
}
if override_batch_size_bytes is not None:
conf["spark.rapids.sql.batchSizeBytes"] = override_batch_size_bytes
@@ -340,23 +347,29 @@ def test_hash_grpby_sum_count_action(data_gen, override_split_until_size, overri
@allow_non_gpu("SortAggregateExec", "SortExec", "ShuffleExchangeExec")
@ignore_order
@pytest.mark.parametrize('data_gen', _grpkey_nested_structs_with_array_basic_child + _grpkey_list_with_non_nested_children, ids=idfn)
-def test_hash_grpby_list_min_max(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_grpby_list_min_max(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark: gen_df(spark, data_gen, length=100).coalesce(1).groupby('a').agg(f.min('b'), f.max('b'))
- )
+ lambda spark: gen_df(spark, data_gen, length=100).coalesce(1).groupby('a').agg(f.min(
+ 'b'), f.max('b')),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [_longs_with_nulls], ids=idfn)
-def test_hash_reduction_sum_count_action(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_reduction_sum_count_action(data_gen, kudo_enabled):
assert_gpu_and_cpu_row_counts_equal(
- lambda spark: gen_df(spark, data_gen, length=100).agg(f.sum('b'))
+ lambda spark: gen_df(spark, data_gen, length=100).agg(f.sum('b')),
+ conf = {kudo_enabled_conf_key: kudo_enabled}
)
# Make sure that we can do computation in the group by columns
@ignore_order
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
-def test_computation_in_grpby_columns():
- conf = {'spark.rapids.sql.batchSizeBytes' : '250'}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_computation_in_grpby_columns(kudo_enabled):
+ conf = {'spark.rapids.sql.batchSizeBytes' : '250',
+ kudo_enabled_conf_key: kudo_enabled}
data_gen = [
('a', RepeatSeqGen(StringGen('a{1,20}'), length=50)),
('b', short_gen)]
@@ -371,10 +384,12 @@ def test_computation_in_grpby_columns():
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list_with_decimalbig, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_grpby_sum(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_grpby_sum(data_gen, conf, kudo_enabled):
+ new_conf = copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.sum('b')),
- conf = conf)
+ conf = new_conf)
@shuffle_test
@approximate_float
@@ -383,10 +398,12 @@ def test_hash_grpby_sum(data_gen, conf):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [_grpkey_short_sum_full_decimals, _grpkey_short_sum_full_neg_scale_decimals], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_grpby_sum_full_decimal(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_grpby_sum_full_decimal(data_gen, conf, kudo_enabled):
+ new_conf = copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.sum('b')),
- conf = conf)
+ conf = new_conf)
@approximate_float
@datagen_overrides(seed=0, reason="https://github.com/NVIDIA/spark-rapids/issues/9822")
@@ -394,10 +411,12 @@ def test_hash_grpby_sum_full_decimal(data_gen, conf):
@incompat
@pytest.mark.parametrize('data_gen', numeric_gens + decimal_gens + [DecimalGen(precision=36, scale=5)], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_reduction_sum(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_reduction_sum(data_gen, conf, kudo_enabled):
+ new_conf = copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_collect(
lambda spark: unary_op_df(spark, data_gen, length=100).selectExpr("SUM(a)"),
- conf = conf)
+ conf = new_conf)
@approximate_float
@ignore_order
@@ -406,11 +425,13 @@ def test_hash_reduction_sum(data_gen, conf):
@pytest.mark.parametrize('data_gen', numeric_gens + decimal_gens + [
DecimalGen(precision=38, scale=0), DecimalGen(precision=38, scale=-10)], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@datagen_overrides(seed=0, permanent=True, reason='https://github.com/NVIDIA/spark-rapids/issues/9779')
-def test_hash_reduction_sum_full_decimal(data_gen, conf):
+def test_hash_reduction_sum_full_decimal(data_gen, conf, kudo_enabled):
+ new_conf = copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_collect(
lambda spark: unary_op_df(spark, data_gen, length=100).selectExpr("SUM(a)"),
- conf = conf)
+ conf = new_conf)
@approximate_float
@ignore_order
@@ -419,10 +440,12 @@ def test_hash_reduction_sum_full_decimal(data_gen, conf):
@pytest.mark.parametrize('data_gen', _init_list + [_grpkey_short_mid_decimals,
_grpkey_short_big_decimals, _grpkey_short_very_big_decimals, _grpkey_short_sum_full_decimals], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_grpby_avg(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_grpby_avg(data_gen, conf, kudo_enabled):
+ new_conf = copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=200).groupby('a').agg(f.avg('b')),
- conf=conf
+ conf=new_conf
)
# tracks https://github.com/NVIDIA/spark-rapids/issues/154
@@ -438,30 +461,38 @@ def test_hash_grpby_avg(data_gen, conf):
@pytest.mark.parametrize('data_gen', [
StructGen(children=[('a', int_gen), ('b', int_gen)],nullable=False,
special_cases=[((None, None), 400.0), ((None, -1542301795), 100.0)])], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@pytest.mark.xfail(condition=is_databricks104_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/4963')
-def test_hash_avg_nulls_partial_only(data_gen):
+def test_hash_avg_nulls_partial_only(data_gen, kudo_enabled):
+ conf = copy_and_update(_float_conf_partial, {kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=2).agg(f.avg('b')),
- conf=_float_conf_partial
- )
+ conf=conf)
@approximate_float
@ignore_order
@incompat
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list_with_decimalbig, ids=idfn)
-def test_intersect_all(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_intersect_all(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark : gen_df(spark, data_gen, length=100).intersectAll(gen_df(spark, data_gen, length=100)))
+ lambda spark : gen_df(spark, data_gen, length=100).intersectAll(gen_df(spark, data_gen,
+ length=100)),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@approximate_float
@ignore_order
@incompat
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list_with_decimalbig, ids=idfn)
-def test_exceptAll(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_exceptAll(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark : gen_df(spark, data_gen, length=100).exceptAll(gen_df(spark, data_gen, length=100).filter('a != b')))
+ lambda spark : (gen_df(spark, data_gen, length=100)
+ .exceptAll(gen_df(spark, data_gen, length=100)
+ .filter('a != b'))),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# Spark fails to sort some decimal values due to overflow when calculating the sorting prefix.
# See https://issues.apache.org/jira/browse/SPARK-40129
@@ -488,13 +519,14 @@ def test_exceptAll(data_gen):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _pivot_gens_with_decimals, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_grpby_pivot(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_grpby_pivot(data_gen, conf, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.groupby('a')
.pivot('b')
.agg(f.sum('c')),
- conf = conf)
+ conf = copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@approximate_float
@ignore_order(local=True)
@@ -503,13 +535,14 @@ def test_hash_grpby_pivot(data_gen, conf):
@pytest.mark.parametrize('data_gen', _init_list, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
@datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/10062')
-def test_hash_multiple_grpby_pivot(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_multiple_grpby_pivot(data_gen, conf, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.groupby('a','b')
.pivot('b')
.agg(f.sum('c'), f.max('c')),
- conf=conf)
+ conf=copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@approximate_float
@ignore_order(local=True)
@@ -517,13 +550,14 @@ def test_hash_multiple_grpby_pivot(data_gen, conf):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_reduction_pivot(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_reduction_pivot(data_gen, conf, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.groupby()
.pivot('b')
.agg(f.sum('c')),
- conf=conf)
+ conf=copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@approximate_float
@@ -533,7 +567,8 @@ def test_hash_reduction_pivot(data_gen, conf):
@incompat
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [_grpkey_floats_with_nulls_and_nans], ids=idfn)
-def test_hash_pivot_groupby_duplicates_fallback(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_pivot_groupby_duplicates_fallback(data_gen, kudo_enabled):
# PivotFirst will not work on the GPU when pivot_values has duplicates
assert_gpu_fallback_collect(
lambda spark: gen_df(spark, data_gen, length=100)
@@ -541,7 +576,7 @@ def test_hash_pivot_groupby_duplicates_fallback(data_gen):
.pivot('b', ['10.0', '10.0'])
.agg(f.sum('c')),
"PivotFirst",
- conf=_float_conf)
+ conf=copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}) )
_repeat_agg_column_for_collect_op = [
RepeatSeqGen(BooleanGen(), length=15),
@@ -610,43 +645,53 @@ def test_hash_pivot_groupby_duplicates_fallback(data_gen):
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn)
-def test_decimal128_count_reduction(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_decimal128_count_reduction(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark: unary_op_df(spark, data_gen).selectExpr('count(a)'))
+ lambda spark: unary_op_df(spark, data_gen).selectExpr('count(a)'),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# very simple test for just a count on decimals 128 values until we can support more with them
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn)
-def test_decimal128_count_group_by(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_decimal128_count_group_by(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: two_col_df(spark, byte_gen, data_gen)
.groupby('a')
- .agg(f.count('b')))
+ .agg(f.count('b')),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# very simple test for just a min/max on decimals 128 values until we can support more with them
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn)
-def test_decimal128_min_max_reduction(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_decimal128_min_max_reduction(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark: unary_op_df(spark, data_gen).selectExpr('min(a)', 'max(a)'))
+ lambda spark: unary_op_df(spark, data_gen).selectExpr('min(a)', 'max(a)'),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# very simple test for just a min/max on decimals 128 values until we can support more with them
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn)
-def test_decimal128_min_max_group_by(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_decimal128_min_max_group_by(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: two_col_df(spark, byte_gen, data_gen)
.groupby('a')
- .agg(f.min('b'), f.max('b')))
+ .agg(f.min('b'), f.max('b')),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', _all_basic_gens_with_all_nans_cases, ids=idfn)
-def test_min_max_group_by(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_min_max_group_by(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: two_col_df(spark, byte_gen, data_gen)
.groupby('a')
- .agg(f.min('b'), f.max('b')))
+ .agg(f.min('b'), f.max('b')),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# To avoid ordering issues with collect_list, sorting the arrays that are returned.
# NOTE: sorting the arrays locally, because sort_array() does not yet
@@ -657,18 +702,21 @@ def test_min_max_group_by(data_gen):
@ignore_order(local=True, arrays=["blist"])
@pytest.mark.parametrize('data_gen', _gen_data_for_collect_list_op, ids=idfn)
@pytest.mark.parametrize('use_obj_hash_agg', [True, False], ids=idfn)
-def test_hash_groupby_collect_list(data_gen, use_obj_hash_agg):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_collect_list(data_gen, use_obj_hash_agg, kudo_enabled):
def doit(spark):
return gen_df(spark, data_gen, length=100)\
.groupby('a')\
.agg(f.collect_list('b').alias("blist"))
assert_gpu_and_cpu_are_equal_collect(
doit,
- conf={'spark.sql.execution.useObjectHashAggregateExec': str(use_obj_hash_agg).lower()})
+ conf={'spark.sql.execution.useObjectHashAggregateExec': str(use_obj_hash_agg).lower(),
+ kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('use_obj_hash_agg', [True, False], ids=idfn)
-def test_hash_groupby_collect_list_of_maps(use_obj_hash_agg):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_collect_list_of_maps(use_obj_hash_agg, kudo_enabled):
gens = [("a", RepeatSeqGen(LongGen(), length=20)), ("b", simple_string_to_string_map_gen)]
def doit(spark):
df = gen_df(spark, gens, length=100) \
@@ -680,27 +728,32 @@ def doit(spark):
return spark.createDataFrame(df.rdd, schema=df.schema).select("a", f.explode("blist"))
assert_gpu_and_cpu_are_equal_collect(
doit,
- conf={'spark.sql.execution.useObjectHashAggregateExec': str(use_obj_hash_agg).lower()})
+ conf={'spark.sql.execution.useObjectHashAggregateExec': str(use_obj_hash_agg).lower(),
+ kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_groupby_collect_set(data_gen):
+def test_hash_groupby_collect_set(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.groupby('a')
- .agg(f.sort_array(f.collect_set('b')), f.count('b')))
+ .agg(f.sort_array(f.collect_set('b')), f.count('b')),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_groupby_collect_set_on_nested_type(data_gen):
+def test_hash_groupby_collect_set_on_nested_type(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.groupby('a')
- .agg(f.sort_array(f.collect_set('b'))))
+ .agg(f.sort_array(f.collect_set('b'))),
+ conf= {kudo_enabled_conf_key: kudo_enabled})
# NOTE: sorting the arrays locally, because sort_array() does not yet
@@ -710,9 +763,11 @@ def test_hash_groupby_collect_set_on_nested_type(data_gen):
@ignore_order(local=True, arrays=["collect_set"])
@allow_non_gpu("ProjectExec", *non_utc_allow)
@pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op_nested, ids=idfn)
-def test_hash_groupby_collect_set_on_nested_array_type(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_collect_set_on_nested_array_type(data_gen, kudo_enabled):
conf = copy_and_update(_float_conf, {
"spark.rapids.sql.castFloatToString.enabled": "true",
+ kudo_enabled_conf_key: kudo_enabled
})
def do_it(spark):
@@ -726,19 +781,23 @@ def do_it(spark):
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_reduction_collect_set(data_gen):
+def test_hash_reduction_collect_set(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
- .agg(f.sort_array(f.collect_set('b')), f.count('b')))
+ .agg(f.sort_array(f.collect_set('b')), f.count('b')),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_reduction_collect_set_on_nested_type(data_gen):
+def test_hash_reduction_collect_set_on_nested_type(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
- .agg(f.sort_array(f.collect_set('b'))))
+ .agg(f.sort_array(f.collect_set('b'))),
+ conf= {kudo_enabled_conf_key: kudo_enabled})
# NOTE: sorting the arrays locally, because sort_array() does not yet
@@ -748,9 +807,11 @@ def test_hash_reduction_collect_set_on_nested_type(data_gen):
@ignore_order(local=True, arrays=["collect_set"])
@allow_non_gpu("ProjectExec", *non_utc_allow)
@pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op_nested, ids=idfn)
-def test_hash_reduction_collect_set_on_nested_array_type(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_reduction_collect_set_on_nested_array_type(data_gen, kudo_enabled):
conf = copy_and_update(_float_conf, {
"spark.rapids.sql.castFloatToString.enabled": "true",
+ kudo_enabled_conf_key: kudo_enabled
})
def do_it(spark):
@@ -763,8 +824,9 @@ def do_it(spark):
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_groupby_collect_with_single_distinct(data_gen):
+def test_hash_groupby_collect_with_single_distinct(data_gen, kudo_enabled):
# test collect_ops with other distinct aggregations
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
@@ -772,7 +834,8 @@ def test_hash_groupby_collect_with_single_distinct(data_gen):
.agg(f.sort_array(f.collect_list('b')),
f.sort_array(f.collect_set('b')),
f.countDistinct('c'),
- f.count('c')))
+ f.count('c')),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
def hash_groupby_single_distinct_collect_impl(data_gen, conf):
@@ -798,41 +861,46 @@ def hash_groupby_single_distinct_collect_impl(data_gen, conf):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', _gen_data_for_collect_op, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_groupby_single_distinct_collect(data_gen):
+def test_hash_groupby_single_distinct_collect(data_gen, kudo_enabled):
"""
Tests distinct collect, with ANSI disabled.
The corresponding ANSI-enabled condition is tested in
test_hash_groupby_single_distinct_collect_ansi_enabled
"""
- ansi_disabled_conf = {'spark.sql.ansi.enabled': False}
+ ansi_disabled_conf = {'spark.sql.ansi.enabled': False,
+ kudo_enabled_conf_key: kudo_enabled}
hash_groupby_single_distinct_collect_impl(data_gen=data_gen, conf=ansi_disabled_conf)
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [_gen_data_for_collect_op[0]], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
@allow_non_gpu('ObjectHashAggregateExec', 'ShuffleExchangeExec')
-def test_hash_groupby_single_distinct_collect_ansi_enabled(data_gen):
+def test_hash_groupby_single_distinct_collect_ansi_enabled(data_gen, kudo_enabled):
"""
Tests distinct collect, with ANSI enabled.
Enabling ANSI mode causes the plan to include ObjectHashAggregateExec, which runs on CPU.
"""
- hash_groupby_single_distinct_collect_impl(data_gen=data_gen, conf=ansi_enabled_conf)
+ hash_groupby_single_distinct_collect_impl(data_gen=data_gen,
+ conf=copy_and_update(ansi_enabled_conf, {kudo_enabled_conf_key: kudo_enabled}))
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _gen_data_for_collect_op, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_groupby_collect_with_multi_distinct(data_gen):
+def test_hash_groupby_collect_with_multi_distinct(data_gen, kudo_enabled):
def spark_fn(spark_session):
return gen_df(spark_session, data_gen, length=100).groupby('a').agg(
f.sort_array(f.collect_list('b')),
f.sort_array(f.collect_set('b')),
f.countDistinct('b'),
f.countDistinct('c'))
- assert_gpu_and_cpu_are_equal_collect(spark_fn)
+ assert_gpu_and_cpu_are_equal_collect(spark_fn, conf = {kudo_enabled_conf_key: kudo_enabled})
_replace_modes_non_distinct = [
# Spark: GPU(Final) -> CPU(Partial)
@@ -851,13 +919,16 @@ def spark_fn(spark_session):
@pytest.mark.parametrize('replace_mode', _replace_modes_non_distinct, ids=idfn)
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
@pytest.mark.parametrize('use_obj_hash_agg', ['false', 'true'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
def test_hash_groupby_collect_partial_replace_fallback(data_gen,
replace_mode,
aqe_enabled,
- use_obj_hash_agg):
+ use_obj_hash_agg,
+ kudo_enabled):
conf = {'spark.rapids.sql.hashAgg.replaceMode': replace_mode,
'spark.sql.adaptive.enabled': aqe_enabled,
- 'spark.sql.execution.useObjectHashAggregateExec': use_obj_hash_agg}
+ 'spark.sql.execution.useObjectHashAggregateExec': use_obj_hash_agg,
+ kudo_enabled_conf_key: kudo_enabled}
cpu_clz, gpu_clz = ['CollectList', 'CollectSet'], ['GpuCollectList', 'GpuCollectSet']
exist_clz, non_exist_clz = [], []
@@ -901,14 +972,17 @@ def test_hash_groupby_collect_partial_replace_fallback(data_gen,
@pytest.mark.parametrize('replace_mode', _replace_modes_single_distinct, ids=idfn)
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
@pytest.mark.parametrize('use_obj_hash_agg', ['false', 'true'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@pytest.mark.xfail(condition=is_databricks104_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/4963')
def test_hash_groupby_collect_partial_replace_with_distinct_fallback(data_gen,
replace_mode,
aqe_enabled,
- use_obj_hash_agg):
+ use_obj_hash_agg,
+ kudo_enabled):
conf = {'spark.rapids.sql.hashAgg.replaceMode': replace_mode,
'spark.sql.adaptive.enabled': aqe_enabled,
- 'spark.sql.execution.useObjectHashAggregateExec': use_obj_hash_agg}
+ 'spark.sql.execution.useObjectHashAggregateExec': use_obj_hash_agg,
+ kudo_enabled_conf_key: kudo_enabled}
# test with single Distinct
assert_cpu_and_gpu_are_equal_collect_with_capture(
lambda spark: gen_df(spark, data_gen, length=100)
@@ -975,10 +1049,11 @@ def exact_percentile_reduction(df):
@datagen_overrides(seed=0, reason="https://github.com/NVIDIA/spark-rapids/issues/10233")
@pytest.mark.parametrize('data_gen', exact_percentile_reduction_data_gen, ids=idfn)
-def test_exact_percentile_reduction(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_exact_percentile_reduction(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark: exact_percentile_reduction(gen_df(spark, data_gen))
- )
+ lambda spark: exact_percentile_reduction(gen_df(spark, data_gen)),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
exact_percentile_reduction_cpu_fallback_data_gen = [
[('val', data_gen),
@@ -992,9 +1067,10 @@ def test_exact_percentile_reduction(data_gen):
@pytest.mark.parametrize('data_gen', exact_percentile_reduction_cpu_fallback_data_gen, ids=idfn)
@pytest.mark.parametrize('replace_mode', ['partial', 'final|complete'], ids=idfn)
@pytest.mark.parametrize('use_obj_hash_agg', ['false', 'true'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@pytest.mark.xfail(condition=is_databricks104_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/9494')
def test_exact_percentile_reduction_partial_fallback_to_cpu(data_gen, replace_mode,
- use_obj_hash_agg):
+ use_obj_hash_agg, kudo_enabled):
cpu_clz, gpu_clz = ['Percentile'], ['GpuPercentileDefault']
exist_clz, non_exist_clz = [], []
# For aggregations without distinct, Databricks runtime removes the partial Aggregate stage (
@@ -1017,7 +1093,8 @@ def test_exact_percentile_reduction_partial_fallback_to_cpu(data_gen, replace_m
exist_classes=','.join(exist_clz),
non_exist_classes=','.join(non_exist_clz),
conf={'spark.rapids.sql.hashAgg.replaceMode': replace_mode,
- 'spark.sql.execution.useObjectHashAggregateExec': use_obj_hash_agg}
+ 'spark.sql.execution.useObjectHashAggregateExec': use_obj_hash_agg,
+ kudo_enabled_conf_key: kudo_enabled}
)
@@ -1051,10 +1128,11 @@ def exact_percentile_groupby(df):
@ignore_order
@pytest.mark.parametrize('data_gen', exact_percentile_groupby_data_gen, ids=idfn)
-def test_exact_percentile_groupby(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_exact_percentile_groupby(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark: exact_percentile_groupby(gen_df(spark, data_gen))
- )
+ lambda spark: exact_percentile_groupby(gen_df(spark, data_gen)),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
exact_percentile_groupby_cpu_fallback_data_gen = [
[('key', RepeatSeqGen(IntegerGen(), length=100)),
@@ -1070,8 +1148,10 @@ def test_exact_percentile_groupby(data_gen):
@pytest.mark.parametrize('data_gen', exact_percentile_groupby_cpu_fallback_data_gen, ids=idfn)
@pytest.mark.parametrize('replace_mode', ['partial', 'final|complete'], ids=idfn)
@pytest.mark.parametrize('use_obj_hash_agg', ['false', 'true'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@pytest.mark.xfail(condition=is_databricks104_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/9494')
-def test_exact_percentile_groupby_partial_fallback_to_cpu(data_gen, replace_mode, use_obj_hash_agg):
+def test_exact_percentile_groupby_partial_fallback_to_cpu(data_gen, replace_mode,
+ use_obj_hash_agg, kudo_enabled):
cpu_clz, gpu_clz = ['Percentile'], ['GpuPercentileDefault']
exist_clz, non_exist_clz = [], []
# For aggregations without distinct, Databricks runtime removes the partial Aggregate stage (
@@ -1094,15 +1174,16 @@ def test_exact_percentile_groupby_partial_fallback_to_cpu(data_gen, replace_mode
exist_classes=','.join(exist_clz),
non_exist_classes=','.join(non_exist_clz),
conf={'spark.rapids.sql.hashAgg.replaceMode': replace_mode,
- 'spark.sql.execution.useObjectHashAggregateExec': use_obj_hash_agg}
- )
+ 'spark.sql.execution.useObjectHashAggregateExec': use_obj_hash_agg,
+ kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu('ObjectHashAggregateExec', 'ShuffleExchangeExec',
'HashAggregateExec', 'HashPartitioning',
'ApproximatePercentile', 'Alias', 'Literal', 'AggregateExpression')
-def test_hash_groupby_typed_imperative_agg_without_gpu_implementation_fallback():
+def test_hash_groupby_typed_imperative_agg_without_gpu_implementation_fallback(kudo_enabled):
assert_cpu_and_gpu_are_equal_sql_with_capture(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(LongGen(), length=20)),
('v', UniqueLongGen())], length=100),
@@ -1110,7 +1191,8 @@ def test_hash_groupby_typed_imperative_agg_without_gpu_implementation_fallback()
non_exist_classes='GpuApproximatePercentile,GpuObjectHashAggregateExec',
table_name='table',
sql="""select k,
- approx_percentile(v, array(0.25, 0.5, 0.75)) from table group by k""")
+ approx_percentile(v, array(0.25, 0.5, 0.75)) from table group by k""",
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@approximate_float
@ignore_order
@@ -1118,7 +1200,8 @@ def test_hash_groupby_typed_imperative_agg_without_gpu_implementation_fallback()
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_multiple_mode_query(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_multiple_mode_query(data_gen, conf, kudo_enabled):
print_params(data_gen)
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
@@ -1132,7 +1215,7 @@ def test_hash_multiple_mode_query(data_gen, conf):
f.max('a'),
f.sumDistinct('b'),
f.countDistinct('c')
- ), conf=conf)
+ ), conf=copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@approximate_float
@@ -1143,11 +1226,12 @@ def test_hash_multiple_mode_query(data_gen, conf):
@pytest.mark.parametrize('data_gen', _init_list, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs),
ids=idfn)
-def test_hash_multiple_mode_query_avg_distincts(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_multiple_mode_query_avg_distincts(data_gen, conf, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.selectExpr('avg(distinct a)', 'avg(distinct b)','avg(distinct c)'),
- conf=conf)
+ conf=copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@approximate_float
@@ -1157,8 +1241,11 @@ def test_hash_multiple_mode_query_avg_distincts(data_gen, conf):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_query_multiple_distincts_with_non_distinct(data_gen, conf):
- local_conf = copy_and_update(conf, {'spark.sql.legacy.allowParameterlessCount': 'true'})
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_query_multiple_distincts_with_non_distinct(data_gen, conf, kudo_enabled):
+ local_conf = copy_and_update(conf,
+ {'spark.sql.legacy.allowParameterlessCount': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, data_gen, length=100),
"hash_agg_table",
@@ -1181,8 +1268,10 @@ def test_hash_query_multiple_distincts_with_non_distinct(data_gen, conf):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_query_max_with_multiple_distincts(data_gen, conf):
- local_conf = copy_and_update(conf, {'spark.sql.legacy.allowParameterlessCount': 'true'})
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_query_max_with_multiple_distincts(data_gen, conf, kudo_enabled):
+ local_conf = copy_and_update(conf, {'spark.sql.legacy.allowParameterlessCount': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, data_gen, length=100),
"hash_agg_table",
@@ -1196,11 +1285,12 @@ def test_hash_query_max_with_multiple_distincts(data_gen, conf):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_count_with_filter(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_count_with_filter(data_gen, conf, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.selectExpr('count(a) filter (where c > 50)'),
- conf=conf)
+ conf=copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@approximate_float
@@ -1209,7 +1299,8 @@ def test_hash_count_with_filter(data_gen, conf):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', _init_list + [_grpkey_short_mid_decimals, _grpkey_short_big_decimals], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_multiple_filters(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_multiple_filters(data_gen, conf, kudo_enabled):
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, data_gen, length=100),
"hash_agg_table",
@@ -1217,15 +1308,17 @@ def test_hash_multiple_filters(data_gen, conf):
'count(b) filter (where c > 100),' +
'avg(b) filter (where b > 20),' +
'min(a), max(b) filter (where c > 250) from hash_agg_table group by a',
- conf)
+ conf = copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@approximate_float
@ignore_order
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [_grpkey_floats_with_nan_zero_grouping_keys,
_grpkey_doubles_with_nan_zero_grouping_keys], ids=idfn)
-def test_hash_agg_with_nan_keys(data_gen):
- local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'})
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_agg_with_nan_keys(data_gen, kudo_enabled):
+ local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, data_gen, length=1024),
"hash_agg_table",
@@ -1245,8 +1338,10 @@ def test_hash_agg_with_nan_keys(data_gen):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [_grpkey_structs_with_non_nested_children,
_grpkey_nested_structs], ids=idfn)
-def test_hash_agg_with_struct_keys(data_gen):
- local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'})
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_agg_with_struct_keys(data_gen, kudo_enabled):
+ local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, data_gen, length=1024),
"hash_agg_table",
@@ -1267,8 +1362,10 @@ def test_hash_agg_with_struct_keys(data_gen):
'Cast', 'Literal', 'Alias', 'AggregateExpression',
'ShuffleExchangeExec', 'HashPartitioning')
@pytest.mark.parametrize('data_gen', [_grpkey_nested_structs_with_array_child], ids=idfn)
-def test_hash_agg_with_struct_of_array_fallback(data_gen):
- local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'})
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_agg_with_struct_of_array_fallback(data_gen, kudo_enabled):
+ local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_cpu_and_gpu_are_equal_sql_with_capture(
lambda spark : gen_df(spark, data_gen, length=100),
'select a, '
@@ -1290,12 +1387,13 @@ def test_hash_agg_with_struct_of_array_fallback(data_gen):
@ignore_order
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [ _grpkey_floats_with_nulls_and_nans ], ids=idfn)
-def test_count_distinct_with_nan_floats(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_count_distinct_with_nan_floats(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, data_gen, length=1024),
"hash_agg_table",
'select a, count(distinct b) as count_distinct_bees from hash_agg_table group by a',
- _float_conf)
+ copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}))
# TODO: Literal tests
@@ -1304,27 +1402,33 @@ def test_count_distinct_with_nan_floats(data_gen):
_nested_gens = array_gens_sample + struct_gens_sample + map_gens_sample + [binary_gen]
@pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn)
-def test_first_last_reductions_decimal_types(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_first_last_reductions_decimal_types(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
# Coalesce and sort are to make sure that first and last, which are non-deterministic
# become deterministic
lambda spark: unary_op_df(spark, data_gen).coalesce(1).selectExpr(
- 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)'))
+ 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)'),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', _nested_gens, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_first_last_reductions_nested_types(data_gen):
+def test_first_last_reductions_nested_types(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
# Coalesce and sort are to make sure that first and last, which are non-deterministic
# become deterministic
lambda spark: unary_op_df(spark, data_gen).coalesce(1).selectExpr(
- 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)'))
+ 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)'),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', _all_basic_gens_with_all_nans_cases, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@allow_non_gpu(*non_utc_allow)
-def test_generic_reductions(data_gen):
- local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'})
+def test_generic_reductions(data_gen, kudo_enabled):
+ local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_collect(
# Coalesce and sort are to make sure that first and last, which are non-deterministic
# become deterministic
@@ -1342,43 +1446,50 @@ def test_generic_reductions(data_gen):
# min_by and max_by are supported for pyspark since 3.3.0 so tested with sql
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', all_basic_gens + nested_gens_sample, ids=idfn)
-def test_hash_groupby_min_max_by_unique(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_min_max_by_unique(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_sql(
lambda spark: three_col_df(spark, byte_gen, data_gen, UniqueLongGen()),
"tbl",
- "SELECT a, min_by(b, c), max_by(b, c) FROM tbl GROUP BY a")
+ "SELECT a, min_by(b, c), max_by(b, c) FROM tbl GROUP BY a",
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# When the ordering column is not unique this gpu will always return the minimal/maximal value
# while spark's result is non-deterministic. So we need to set the column b and c to be
# the same to make the result comparable.
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', basic_gen_no_floats + struct_gens_sample_with_decimal128 + array_gens_sample, ids=idfn)
-def test_hash_groupby_min_max_by_same(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_min_max_by_same(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_sql(
lambda spark: two_col_df(spark, byte_gen, data_gen),
"tbl",
- "SELECT a, min_by(b, b), max_by(b, b) FROM tbl GROUP BY a")
+ "SELECT a, min_by(b, b), max_by(b, b) FROM tbl GROUP BY a",
+ conf = {kudo_enabled_conf_key: kudo_enabled})
-def test_reduction_with_min_max_by_unique():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_reduction_with_min_max_by_unique(kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: two_col_df(spark, int_gen, UniqueLongGen()).selectExpr(
- "min_by(a, b)", "max_by(a, b)")
- )
+ "min_by(a, b)", "max_by(a, b)"),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# When the ordering column is not unique this gpu will always return the minimal/maximal value
# while spark's result is non-deterministic. So we need to set the column b and c to be
# the same to make the result comparable.
@pytest.mark.parametrize('data_gen', basic_gen_no_floats + struct_gens_sample_with_decimal128 + array_gens_sample, ids=idfn)
-def test_reduction_with_max_by_same(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_reduction_with_max_by_same(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: unary_op_df(spark, data_gen).selectExpr(
- "min_by(a, a)", "max_by(a, a)")
- )
+ "min_by(a, a)", "max_by(a, a)"),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', all_gen + _nested_gens, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@allow_non_gpu(*non_utc_allow)
-def test_count(data_gen):
+def test_count(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark : unary_op_df(spark, data_gen) \
.selectExpr(
@@ -1386,42 +1497,49 @@ def test_count(data_gen):
'count()',
'count()',
'count(1)'),
- conf = {'spark.sql.legacy.allowParameterlessCount': 'true'})
+ conf = {'spark.sql.legacy.allowParameterlessCount': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', all_basic_gens, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_distinct_count_reductions(data_gen):
+def test_distinct_count_reductions(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark : binary_op_df(spark, data_gen).selectExpr(
- 'count(DISTINCT a)'))
+ 'count(DISTINCT a)'),
+ conf= {kudo_enabled_conf_key: kudo_enabled})
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn)
-def test_distinct_float_count_reductions(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_distinct_float_count_reductions(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark : binary_op_df(spark, data_gen).selectExpr(
- 'count(DISTINCT a)'))
+ 'count(DISTINCT a)'),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@approximate_float
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', numeric_gens + [decimal_gen_64bit, decimal_gen_128bit], ids=idfn)
-def test_arithmetic_reductions(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_arithmetic_reductions(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark : unary_op_df(spark, data_gen).selectExpr(
'sum(a)',
'avg(a)'),
- conf = _float_conf)
+ conf = copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}))
@pytest.mark.parametrize('data_gen',
all_basic_gens + decimal_gens + _nested_gens,
ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_collect_list_reductions(data_gen):
+def test_collect_list_reductions(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
# coalescing because collect_list is not deterministic
lambda spark: unary_op_df(spark, data_gen).coalesce(1).selectExpr('collect_list(a)'),
- conf=_float_conf)
+ conf= copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}) )
_no_neg_zero_all_basic_gens = [byte_gen, short_gen, int_gen, long_gen,
# -0.0 cannot work because of -0.0 == 0.0 in cudf for distinct and
@@ -1435,11 +1553,12 @@ def test_collect_list_reductions(data_gen):
@pytest.mark.parametrize('data_gen',
_no_neg_zero_all_basic_gens + decimal_gens + _struct_only_nested_gens,
ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_collect_set_reductions(data_gen):
+def test_collect_set_reductions(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: unary_op_df(spark, data_gen).selectExpr('sort_array(collect_set(a))'),
- conf=_float_conf)
+ conf=copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}))
def test_collect_empty():
assert_gpu_and_cpu_are_equal_collect(
@@ -1449,8 +1568,9 @@ def test_collect_empty():
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', all_gen + _nested_gens, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_groupby_first_last(data_gen):
+def test_groupby_first_last(data_gen, kudo_enabled):
gen_fn = [('a', RepeatSeqGen(LongGen(), length=20)), ('b', data_gen)]
agg_fn = lambda df: df.groupBy('a').agg(
f.first('b'), f.last('b'), f.first('b', True), f.last('b', True))
@@ -1459,12 +1579,14 @@ def test_groupby_first_last(data_gen):
# We set parallelism 1 to prevent nondeterministic results because of distributed setup.
lambda spark: agg_fn(gen_df(spark, gen_fn, num_slices=1)),
# Disable RADIX sort as the CPU sort is not stable if it is
- conf={'spark.sql.sort.enableRadixSort': False})
+ conf={'spark.sql.sort.enableRadixSort': False,
+ kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', all_gen + _struct_only_nested_gens, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_sorted_groupby_first_last(data_gen):
+def test_sorted_groupby_first_last(data_gen, kudo_enabled):
gen_fn = [('a', RepeatSeqGen(LongGen(), length=20)), ('b', data_gen)]
# sort by more than the group by columns to be sure that first/last don't remove the ordering
agg_fn = lambda df: df.orderBy('a', 'b').groupBy('a').agg(
@@ -1474,7 +1596,8 @@ def test_sorted_groupby_first_last(data_gen):
# We set parallelism and partitions to 1 to prevent nondeterministic results because
# of distributed setups.
lambda spark: agg_fn(gen_df(spark, gen_fn, num_slices=1)),
- conf = {'spark.sql.shuffle.partitions': '1'})
+ conf = {'spark.sql.shuffle.partitions': '1',
+ kudo_enabled_conf_key: kudo_enabled})
# Spark has a sorting bug with decimals, see https://issues.apache.org/jira/browse/SPARK-40129.
# Have pytest do the sorting rather than Spark as a workaround.
@@ -1482,11 +1605,13 @@ def test_sorted_groupby_first_last(data_gen):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', all_gen, ids=idfn)
@pytest.mark.parametrize('count_func', [f.count, f.countDistinct])
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_agg_count(data_gen, count_func):
+def test_agg_count(data_gen, count_func, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark : gen_df(spark, [('a', data_gen), ('b', data_gen)],
- length=1024).groupBy('a').agg(count_func("b")))
+ length=1024).groupBy('a').agg(count_func("b")),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# Spark has a sorting bug with decimals, see https://issues.apache.org/jira/browse/SPARK-40129.
# Have pytest do the sorting rather than Spark as a workaround.
@@ -1497,11 +1622,13 @@ def test_agg_count(data_gen, count_func):
[ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]]))
, binary_gen], ids=idfn)
@pytest.mark.parametrize('count_func', [f.count, f.countDistinct])
-def test_groupby_list_types_fallback(data_gen, count_func):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_groupby_list_types_fallback(data_gen, count_func, kudo_enabled):
assert_gpu_fallback_collect(
lambda spark : gen_df(spark, [('a', data_gen), ('b', data_gen)],
length=1024).groupBy('a').agg(count_func("b")),
- "HashAggregateExec")
+ "HashAggregateExec",
+ conf = {kudo_enabled_conf_key: kudo_enabled})
def subquery_create_temp_views(spark, expr):
t1 = "select * from values (1,2) as t1(a,b)"
@@ -1525,10 +1652,12 @@ def subquery_create_temp_views(spark, expr):
"select sum(distinct(if(c > (select sum(distinct(a)) from t1), d, 0))) as csum " +
"from t2 group by c"
])
-def test_subquery_in_agg(adaptive, expr):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_subquery_in_agg(adaptive, expr, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: subquery_create_temp_views(spark, expr),
- conf = {"spark.sql.adaptive.enabled" : adaptive})
+ conf = {"spark.sql.adaptive.enabled" : adaptive,
+ kudo_enabled_conf_key: kudo_enabled})
# TODO support multi-level structs https://github.com/NVIDIA/spark-rapids/issues/2438
@@ -1558,12 +1687,13 @@ def workaround_dedupe_by_value(df, num_cols):
], nullable=False),
], ids=idfn)
@ignore_order(local=True)
-def test_struct_groupby_count(key_data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_struct_groupby_count(key_data_gen, kudo_enabled):
def group_by_count(spark):
df = two_col_df(spark, key_data_gen, IntegerGen())
assert_single_level_struct(df)
return workaround_dedupe_by_value(df.groupBy(df.a).count(), 3)
- assert_gpu_and_cpu_are_equal_collect(group_by_count)
+ assert_gpu_and_cpu_are_equal_collect(group_by_count, conf = {kudo_enabled_conf_key: kudo_enabled})
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@@ -1578,13 +1708,15 @@ def group_by_count(spark):
], nullable=False)
], ids=idfn)
@ignore_order(local=True)
-def test_struct_cast_groupby_count(cast_struct_tostring, key_data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_struct_cast_groupby_count(cast_struct_tostring, key_data_gen, kudo_enabled):
def _group_by_struct_or_cast(spark):
df = two_col_df(spark, key_data_gen, IntegerGen())
assert_single_level_struct(df)
return df.groupBy(df.a.cast(StringType())).count()
assert_gpu_and_cpu_are_equal_collect(_group_by_struct_or_cast, {
- 'spark.sql.legacy.castComplexTypesToString.enabled': cast_struct_tostring == 'LEGACY'
+ 'spark.sql.legacy.castComplexTypesToString.enabled': cast_struct_tostring == 'LEGACY',
+ kudo_enabled_conf_key: kudo_enabled
})
@@ -1601,12 +1733,13 @@ def _group_by_struct_or_cast(spark):
]))], nullable=False),
], ids=idfn)
@ignore_order(local=True)
-def test_struct_count_distinct(key_data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_struct_count_distinct(key_data_gen, kudo_enabled):
def _count_distinct_by_struct(spark):
df = gen_df(spark, key_data_gen)
assert_single_level_struct(df)
return df.agg(f.countDistinct(df.a))
- assert_gpu_and_cpu_are_equal_collect(_count_distinct_by_struct)
+ assert_gpu_and_cpu_are_equal_collect(_count_distinct_by_struct, conf = {kudo_enabled_conf_key: kudo_enabled})
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@@ -1623,96 +1756,112 @@ def _count_distinct_by_struct(spark):
]))], nullable=False),
], ids=idfn)
@ignore_order(local=True)
-def test_struct_count_distinct_cast(cast_struct_tostring, key_data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_struct_count_distinct_cast(cast_struct_tostring, key_data_gen, kudo_enabled):
def _count_distinct_by_struct(spark):
df = gen_df(spark, key_data_gen)
assert_single_level_struct(df)
return df.agg(f.countDistinct(df.a.cast(StringType())))
assert_gpu_and_cpu_are_equal_collect(_count_distinct_by_struct, {
- 'spark.sql.legacy.castComplexTypesToString.enabled': cast_struct_tostring == 'LEGACY'
+ 'spark.sql.legacy.castComplexTypesToString.enabled': cast_struct_tostring == 'LEGACY',
+ kudo_enabled_conf_key: kudo_enabled
})
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@ignore_order(local=True)
-def test_reduction_nested_struct():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_reduction_nested_struct(kudo_enabled):
def do_it(spark):
df = unary_op_df(spark, StructGen([('aa', StructGen([('aaa', IntegerGen(min_val=0, max_val=4))]))]))
return df.agg(f.sum(df.a.aa.aaa))
- assert_gpu_and_cpu_are_equal_collect(do_it)
+ assert_gpu_and_cpu_are_equal_collect(do_it, conf = {kudo_enabled_conf_key: kudo_enabled})
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@ignore_order(local=True)
-def test_reduction_nested_array():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_reduction_nested_array(kudo_enabled):
def do_it(spark):
df = unary_op_df(spark, ArrayGen(StructGen([('aa', IntegerGen(min_val=0, max_val=4))])))
return df.agg(f.sum(df.a[1].aa))
- assert_gpu_and_cpu_are_equal_collect(do_it)
+ assert_gpu_and_cpu_are_equal_collect(do_it, conf = {kudo_enabled_conf_key: kudo_enabled})
# The map here is a child not a top level, because we only support GetMapValue on String to String maps.
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@ignore_order(local=True)
-def test_reduction_nested_map():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_reduction_nested_map(kudo_enabled):
def do_it(spark):
df = unary_op_df(spark, ArrayGen(MapGen(StringGen('a{1,5}', nullable=False), StringGen('[ab]{1,5}'))))
return df.agg(f.min(df.a[1]["a"]))
- assert_gpu_and_cpu_are_equal_collect(do_it)
+ assert_gpu_and_cpu_are_equal_collect(do_it, conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
-def test_agg_nested_struct():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_agg_nested_struct(kudo_enabled):
def do_it(spark):
df = two_col_df(spark, StringGen('k{1,5}'), StructGen([('aa', StructGen([('aaa', IntegerGen(min_val=0, max_val=4))]))]))
return df.groupBy('a').agg(f.sum(df.b.aa.aaa))
- assert_gpu_and_cpu_are_equal_collect(do_it)
+ assert_gpu_and_cpu_are_equal_collect(do_it, conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
-def test_agg_nested_array():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_agg_nested_array(kudo_enabled):
def do_it(spark):
df = two_col_df(spark, StringGen('k{1,5}'), ArrayGen(StructGen([('aa', IntegerGen(min_val=0, max_val=4))])))
return df.groupBy('a').agg(f.sum(df.b[1].aa))
- assert_gpu_and_cpu_are_equal_collect(do_it)
+ assert_gpu_and_cpu_are_equal_collect(do_it, conf = {kudo_enabled_conf_key: kudo_enabled})
# The map here is a child not a top level, because we only support GetMapValue on String to String maps.
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
-def test_agg_nested_map():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_agg_nested_map(kudo_enabled):
def do_it(spark):
df = two_col_df(spark, StringGen('k{1,5}'), ArrayGen(MapGen(StringGen('a{1,5}', nullable=False), StringGen('[ab]{1,5}'))))
return df.groupBy('a').agg(f.min(df.b[1]["a"]))
- assert_gpu_and_cpu_are_equal_collect(do_it)
+ assert_gpu_and_cpu_are_equal_collect(do_it, conf = {kudo_enabled_conf_key: kudo_enabled})
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_reduction(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_reduction(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('v', DoubleGen())], length=100),
[0.05, 0.25, 0.5, 0.75, 0.95], conf, reduction = True)
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_reduction_single_row(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_reduction_single_row(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('v', DoubleGen())], length=1),
[0.05, 0.25, 0.5, 0.75, 0.95], conf, reduction = True)
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_reduction_no_rows(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_reduction_no_rows(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('v', DoubleGen())], length=0),
[0.05, 0.25, 0.5, 0.75, 0.95], conf, reduction = True)
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_byte(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_byte(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', StringGen(nullable=False)),
('v', ByteGen())], length=100),
@@ -1721,8 +1870,10 @@ def test_hash_groupby_approx_percentile_byte(aqe_enabled):
@incompat
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/11198
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_byte_scalar(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_byte_scalar(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', StringGen(nullable=False)),
('v', ByteGen())], length=100),
@@ -1730,8 +1881,10 @@ def test_hash_groupby_approx_percentile_byte_scalar(aqe_enabled):
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_long_repeated_keys(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_long_repeated_keys(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(LongGen(), length=20)),
('v', UniqueLongGen())], length=100),
@@ -1739,8 +1892,10 @@ def test_hash_groupby_approx_percentile_long_repeated_keys(aqe_enabled):
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_long(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_long(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', StringGen(nullable=False)),
('v', UniqueLongGen())], length=100),
@@ -1749,8 +1904,10 @@ def test_hash_groupby_approx_percentile_long(aqe_enabled):
@incompat
@disable_ansi_mode # ANSI mode is tested in test_hash_groupby_approx_percentile_long_single_ansi
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_long_single(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_long_single(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', StringGen(nullable=False)),
('v', UniqueLongGen())], length=100),
@@ -1760,13 +1917,15 @@ def test_hash_groupby_approx_percentile_long_single(aqe_enabled):
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
@allow_non_gpu('ObjectHashAggregateExec', 'ShuffleExchangeExec')
-def test_hash_groupby_approx_percentile_long_single_ansi(aqe_enabled):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_long_single_ansi(aqe_enabled, kudo_enabled):
"""
Tests approx_percentile with ANSI mode enabled.
Note: In ANSI mode, the test query exercises ObjectHashAggregateExec and ShuffleExchangeExec,
which fall back to CPU.
"""
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
conf.update(ansi_enabled_conf)
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', StringGen(nullable=False)),
@@ -1776,8 +1935,10 @@ def test_hash_groupby_approx_percentile_long_single_ansi(aqe_enabled):
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_double(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_double(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', StringGen(nullable=False)),
('v', DoubleGen())], length=100),
@@ -1785,8 +1946,10 @@ def test_hash_groupby_approx_percentile_double(aqe_enabled):
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
-def test_hash_groupby_approx_percentile_double_single(aqe_enabled):
- conf = {'spark.sql.adaptive.enabled': aqe_enabled}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_double_single(aqe_enabled, kudo_enabled):
+ conf = {'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled}
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', StringGen(nullable=False)),
('v', DoubleGen())], length=100),
@@ -1794,13 +1957,15 @@ def test_hash_groupby_approx_percentile_double_single(aqe_enabled):
@incompat
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@ignore_order(local=True)
@allow_non_gpu('TakeOrderedAndProjectExec', 'Alias', 'Cast', 'ObjectHashAggregateExec', 'AggregateExpression',
'ApproximatePercentile', 'Literal', 'ShuffleExchangeExec', 'HashPartitioning', 'CollectLimitExec')
-def test_hash_groupby_approx_percentile_partial_fallback_to_cpu(aqe_enabled):
+def test_hash_groupby_approx_percentile_partial_fallback_to_cpu(aqe_enabled, kudo_enabled):
conf = {
'spark.rapids.sql.hashAgg.replaceMode': 'partial',
- 'spark.sql.adaptive.enabled': aqe_enabled
+ 'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled
}
def approx_percentile_query(spark):
@@ -1813,66 +1978,80 @@ def approx_percentile_query(spark):
@incompat
@ignore_order(local=True)
-def test_hash_groupby_approx_percentile_decimal32():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_decimal32(kudo_enabled):
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(ByteGen(nullable=False), length=2)),
('v', DecimalGen(6, 2))]),
- [0.05, 0.25, 0.5, 0.75, 0.95])
+ [0.05, 0.25, 0.5, 0.75, 0.95],
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@incompat
@ignore_order(local=True)
@disable_ansi_mode # ANSI mode is tested with test_hash_groupby_approx_percentile_decimal_single_ansi.
-def test_hash_groupby_approx_percentile_decimal32_single():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_decimal32_single(kudo_enabled):
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(ByteGen(nullable=False), length=2)),
('v', DecimalGen(6, 2))]),
- 0.05)
+ 0.05,
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@incompat
@ignore_order(local=True)
@allow_non_gpu('ObjectHashAggregateExec', 'ShuffleExchangeExec')
-def test_hash_groupby_approx_percentile_decimal_single_ansi():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_decimal_single_ansi(kudo_enabled):
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(ByteGen(nullable=False), length=2)),
('v', DecimalGen(6, 2))]),
- 0.05, conf=ansi_enabled_conf)
+ 0.05,
+ conf=copy_and_update(ansi_enabled_conf, {kudo_enabled_conf_key: kudo_enabled}))
@incompat
@ignore_order(local=True)
-def test_hash_groupby_approx_percentile_decimal64():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_decimal64(kudo_enabled):
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(ByteGen(nullable=False), length=2)),
('v', DecimalGen(10, 9))]),
- [0.05, 0.25, 0.5, 0.75, 0.95])
+ [0.05, 0.25, 0.5, 0.75, 0.95],
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@incompat
@disable_ansi_mode # ANSI mode is tested with test_hash_groupby_approx_percentile_decimal_single_ansi.
@ignore_order(local=True)
-def test_hash_groupby_approx_percentile_decimal64_single():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_decimal64_single(kudo_enabled):
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(ByteGen(nullable=False), length=2)),
('v', DecimalGen(10, 9))]),
- 0.05)
+ 0.05,
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@incompat
@ignore_order(local=True)
-def test_hash_groupby_approx_percentile_decimal128():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_decimal128(kudo_enabled):
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(ByteGen(nullable=False), length=2)),
('v', DecimalGen(19, 18))]),
- [0.05, 0.25, 0.5, 0.75, 0.95])
+ [0.05, 0.25, 0.5, 0.75, 0.95],
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@incompat
@disable_ansi_mode # ANSI mode is tested with test_hash_groupby_approx_percentile_decimal_single_ansi.
@ignore_order(local=True)
-def test_hash_groupby_approx_percentile_decimal128_single():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_groupby_approx_percentile_decimal128_single(kudo_enabled):
compare_percentile_approx(
lambda spark: gen_df(spark, [('k', RepeatSeqGen(ByteGen(nullable=False), length=2)),
('v', DecimalGen(19, 18))]),
- 0.05)
+ 0.05,
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# The percentile approx tests differ from other tests because we do not expect the CPU and GPU to produce the same
# results due to the different algorithms being used. Instead we compute an exact percentile on the CPU and then
@@ -1967,20 +2146,22 @@ def create_percentile_sql(func_name, percentiles, reduction):
@disable_ansi_mode # ANSI mode is tested in test_hash_grpby_avg_nulls_ansi
@pytest.mark.parametrize('data_gen', [_grpkey_strings_with_extra_nulls], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_grpby_avg_nulls(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_grpby_avg_nulls(data_gen, conf, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100).groupby('a')
.agg(f.avg('c')),
- conf=conf
- )
+ conf=copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@ignore_order
@allow_non_gpu('HashAggregateExec', 'Alias', 'AggregateExpression', 'Cast',
'HashPartitioning', 'ShuffleExchangeExec', 'Average')
@pytest.mark.parametrize('data_gen', [_grpkey_strings_with_extra_nulls], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_grpby_avg_nulls_ansi(data_gen, conf):
- local_conf = copy_and_update(conf, {'spark.sql.ansi.enabled': 'true'})
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_grpby_avg_nulls_ansi(data_gen, conf, kudo_enabled):
+ local_conf = copy_and_update(conf, {'spark.sql.ansi.enabled': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_fallback_collect(
lambda spark: gen_df(spark, data_gen, length=100).groupby('a')
.agg(f.avg('c')),
@@ -1992,20 +2173,22 @@ def test_hash_grpby_avg_nulls_ansi(data_gen, conf):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('data_gen', [_grpkey_strings_with_extra_nulls], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_reduction_avg_nulls(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_reduction_avg_nulls(data_gen, conf, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.agg(f.avg('c')),
- conf=conf
- )
+ conf=copy_and_update(conf, {kudo_enabled_conf_key: kudo_enabled}))
@ignore_order
@allow_non_gpu('HashAggregateExec', 'Alias', 'AggregateExpression', 'Cast',
'HashPartitioning', 'ShuffleExchangeExec', 'Average')
@pytest.mark.parametrize('data_gen', [_grpkey_strings_with_extra_nulls], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_hash_reduction_avg_nulls_ansi(data_gen, conf):
- local_conf = copy_and_update(conf, {'spark.sql.ansi.enabled': 'true'})
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_reduction_avg_nulls_ansi(data_gen, conf, kudo_enabled):
+ local_conf = copy_and_update(conf, {'spark.sql.ansi.enabled': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_fallback_collect(
lambda spark: gen_df(spark, data_gen, length=100)
.agg(f.avg('c')),
@@ -2018,43 +2201,47 @@ def test_hash_reduction_avg_nulls_ansi(data_gen, conf):
@allow_non_gpu('HashAggregateExec', 'Alias', 'AggregateExpression', 'Cast',
'HashPartitioning', 'ShuffleExchangeExec', 'Sum')
@pytest.mark.parametrize('data_gen', _no_overflow_ansi_gens, ids=idfn)
-def test_sum_fallback_when_ansi_enabled(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sum_fallback_when_ansi_enabled(data_gen, kudo_enabled):
def do_it(spark):
df = gen_df(spark, [('a', data_gen), ('b', data_gen)], length=100)
return df.groupBy('a').agg(f.sum("b"))
assert_gpu_fallback_collect(do_it, 'Sum',
- conf={'spark.sql.ansi.enabled': 'true'})
+ conf={'spark.sql.ansi.enabled': 'true', kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@allow_non_gpu('HashAggregateExec', 'Alias', 'AggregateExpression', 'Cast',
'HashPartitioning', 'ShuffleExchangeExec', 'Average')
@pytest.mark.parametrize('data_gen', _no_overflow_ansi_gens, ids=idfn)
-def test_avg_fallback_when_ansi_enabled(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_avg_fallback_when_ansi_enabled(data_gen, kudo_enabled):
def do_it(spark):
df = gen_df(spark, [('a', data_gen), ('b', data_gen)], length=100)
return df.groupBy('a').agg(f.avg("b"))
assert_gpu_fallback_collect(do_it, 'Average',
- conf={'spark.sql.ansi.enabled': 'true'})
+ conf={'spark.sql.ansi.enabled': 'true', kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@allow_non_gpu('HashAggregateExec', 'Alias', 'AggregateExpression',
'HashPartitioning', 'ShuffleExchangeExec', 'Count', 'Literal')
@pytest.mark.parametrize('data_gen', _no_overflow_ansi_gens, ids=idfn)
-def test_count_fallback_when_ansi_enabled(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_count_fallback_when_ansi_enabled(data_gen, kudo_enabled):
def do_it(spark):
df = gen_df(spark, [('a', data_gen), ('b', data_gen)], length=100)
return df.groupBy('a').agg(f.count("b"), f.count("*"))
assert_gpu_fallback_collect(do_it, 'Count',
- conf={'spark.sql.ansi.enabled': 'true'})
+ conf={'spark.sql.ansi.enabled': 'true', kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', _no_overflow_ansi_gens, ids=idfn)
-def test_no_fallback_when_ansi_enabled(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_no_fallback_when_ansi_enabled(data_gen, kudo_enabled):
def do_it(spark):
df = gen_df(spark, [('a', data_gen), ('b', data_gen)], length=100)
# coalescing because first/last are not deterministic
@@ -2062,7 +2249,7 @@ def do_it(spark):
return df.groupBy('a').agg(f.first("b"), f.last("b"), f.min("b"), f.max("b"))
assert_gpu_and_cpu_are_equal_collect(do_it,
- conf={'spark.sql.ansi.enabled': 'true'})
+ conf={'spark.sql.ansi.enabled': 'true', kudo_enabled_conf_key: kudo_enabled})
# Tests for standard deviation and variance aggregations.
@ignore_order(local=True)
@@ -2070,9 +2257,11 @@ def do_it(spark):
@incompat
@pytest.mark.parametrize('data_gen', _init_list_with_decimals, ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
-def test_std_variance(data_gen, conf):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_std_variance(data_gen, conf, kudo_enabled):
local_conf = copy_and_update(conf, {
- 'spark.rapids.sql.castDecimalToFloat.enabled': 'true'})
+ 'spark.rapids.sql.castDecimalToFloat.enabled': 'true',
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, data_gen, length=1000),
"data_table",
@@ -2101,8 +2290,10 @@ def test_std_variance(data_gen, conf):
@pytest.mark.parametrize('data_gen', [_grpkey_strings_with_extra_nulls], ids=idfn)
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
@pytest.mark.parametrize('ansi_enabled', ['true', 'false'])
-def test_std_variance_nulls(data_gen, conf, ansi_enabled):
- local_conf = copy_and_update(conf, {'spark.sql.ansi.enabled': ansi_enabled})
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_std_variance_nulls(data_gen, conf, ansi_enabled, kudo_enabled):
+ local_conf = copy_and_update(conf, {'spark.sql.ansi.enabled': ansi_enabled,
+ kudo_enabled_conf_key: kudo_enabled})
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, data_gen, length=1000),
"data_table",
@@ -2138,13 +2329,16 @@ def test_std_variance_nulls(data_gen, conf, ansi_enabled):
@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn)
@pytest.mark.parametrize('replace_mode', _replace_modes_non_distinct, ids=idfn)
@pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@pytest.mark.xfail(condition=is_databricks104_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/4963')
def test_std_variance_partial_replace_fallback(data_gen,
conf,
replace_mode,
- aqe_enabled):
+ aqe_enabled,
+ kudo_enabled):
local_conf = copy_and_update(conf, {'spark.rapids.sql.hashAgg.replaceMode': replace_mode,
- 'spark.sql.adaptive.enabled': aqe_enabled})
+ 'spark.sql.adaptive.enabled': aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled})
exist_clz = ['StddevPop', 'StddevSamp', 'VariancePop', 'VarianceSamp',
'GpuStddevPop', 'GpuStddevSamp', 'GpuVariancePop', 'GpuVarianceSamp']
@@ -2189,8 +2383,9 @@ def test_std_variance_partial_replace_fallback(data_gen,
null_gen] + array_gens_sample + struct_gens_sample
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', gens_for_max_min, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_min_max_in_groupby_and_reduction(data_gen):
+def test_min_max_in_groupby_and_reduction(data_gen, kudo_enabled):
df_gen = [('a', data_gen), ('b', RepeatSeqGen(IntegerGen(), length=20))]
# test max
@@ -2198,44 +2393,48 @@ def test_min_max_in_groupby_and_reduction(data_gen):
lambda spark : gen_df(spark, df_gen),
"hash_agg_table",
'select b, max(a) from hash_agg_table group by b',
- _float_conf)
+ copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}))
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, df_gen),
"hash_agg_table",
'select max(a) from hash_agg_table',
- _float_conf)
+ copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}))
# test min
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, df_gen, length=1024),
"hash_agg_table",
'select b, min(a) from hash_agg_table group by b',
- _float_conf)
+ copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}))
assert_gpu_and_cpu_are_equal_sql(
lambda spark : gen_df(spark, df_gen, length=1024),
"hash_agg_table",
'select min(a) from hash_agg_table',
- _float_conf)
+ copy_and_update(_float_conf, {kudo_enabled_conf_key: kudo_enabled}))
# Some Spark implementations will optimize this aggregation as a
# complete aggregation (i.e.: only one aggregation node in the plan)
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
-def test_hash_aggregate_complete_with_grouping_expressions():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_aggregate_complete_with_grouping_expressions(kudo_enabled):
assert_gpu_and_cpu_are_equal_sql(
lambda spark : spark.range(10).withColumn("id2", f.col("id")),
"hash_agg_complete_table",
- "select id, avg(id) from hash_agg_complete_table group by id, id2 + 1")
+ "select id, avg(id) from hash_agg_complete_table group by id, id2 + 1",
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
@pytest.mark.parametrize('cast_key_to', ["byte", "short", "int",
"long", "string", "DECIMAL(38,5)"], ids=idfn)
-def test_hash_agg_force_pre_sort(cast_key_to):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_agg_force_pre_sort(cast_key_to, kudo_enabled):
def do_it(spark):
gen = StructGen([("key", UniqueLongGen()), ("value", long_gen)], nullable=False)
df = gen_df(spark, gen)
return df.selectExpr("CAST((key div 10) as " + cast_key_to + ") as key", "value").groupBy("key").sum("value")
assert_gpu_and_cpu_are_equal_collect(do_it,
conf={'spark.rapids.sql.agg.forceSinglePassPartialSort': True,
- 'spark.rapids.sql.agg.singlePassPartialSortEnabled': True})
+ 'spark.rapids.sql.agg.singlePassPartialSortEnabled': True,
+ kudo_enabled_conf_key: kudo_enabled})
diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py
index 703fbe80230..936310bedeb 100644
--- a/integration_tests/src/main/python/join_test.py
+++ b/integration_tests/src/main/python/join_test.py
@@ -96,6 +96,8 @@
'spark.sql.shuffle.partitions': '2',
}
+kudo_enabled_conf_key = "spark.rapids.shuffle.kudo.serializer.enabled"
+
def create_df(spark, data_gen, left_length, right_length):
left = binary_op_df(spark, data_gen, length=left_length)
right = binary_op_df(spark, data_gen, length=right_length).withColumnRenamed("a", "r_a")\
@@ -125,53 +127,77 @@ def join_batch_size_test_params(*args):
@ignore_order(local=True)
@pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn)
@pytest.mark.parametrize("aqe_enabled", ["true", "false"], ids=idfn)
-def test_right_broadcast_nested_loop_join_without_condition_empty(join_type, aqe_enabled):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_right_broadcast_nested_loop_join_without_condition_empty(join_type, aqe_enabled, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, long_gen, 50, 0)
return left.join(broadcast(right), how=join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={ "spark.sql.adaptive.enabled": aqe_enabled })
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ "spark.sql.adaptive.enabled": aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled
+ })
@ignore_order(local=True)
@pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn)
@pytest.mark.parametrize("aqe_enabled", ["true", "false"], ids=idfn)
-def test_left_broadcast_nested_loop_join_without_condition_empty(join_type, aqe_enabled):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_left_broadcast_nested_loop_join_without_condition_empty(join_type, aqe_enabled, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, long_gen, 0, 50)
return left.join(broadcast(right), how=join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={ "spark.sql.adaptive.enabled": aqe_enabled })
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ "spark.sql.adaptive.enabled": aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled
+ })
@ignore_order(local=True)
@pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn)
@pytest.mark.parametrize("aqe_enabled", ["true", "false"], ids=idfn)
-def test_broadcast_nested_loop_join_without_condition_empty(join_type, aqe_enabled):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_loop_join_without_condition_empty(join_type, aqe_enabled, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, long_gen, 0, 0)
return left.join(broadcast(right), how=join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={ "spark.sql.adaptive.enabled": aqe_enabled })
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ "spark.sql.adaptive.enabled": aqe_enabled,
+ kudo_enabled_conf_key: kudo_enabled
+ })
@ignore_order(local=True)
@pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_right_broadcast_nested_loop_join_without_condition_empty_small_batch(join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_right_broadcast_nested_loop_join_without_condition_empty_small_batch(join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, long_gen, 50, 0)
return left.join(broadcast(right), how=join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.sql.adaptive.enabled': 'true'})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.sql.adaptive.enabled': 'true',
+ kudo_enabled_conf_key: kudo_enabled
+ })
@ignore_order(local=True)
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_empty_broadcast_hash_join(join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_empty_broadcast_hash_join(join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, long_gen, 50, 0)
return left.join(right.hint("broadcast"), left.a == right.r_a, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.sql.adaptive.enabled': 'true'})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.sql.adaptive.enabled': 'true',
+ kudo_enabled_conf_key: kudo_enabled
+ })
@pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_broadcast_hash_join_constant_keys(join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_hash_join_constant_keys(join_type, kudo_enabled):
def do_join(spark):
left = spark.range(10).withColumn("s", lit(1))
right = spark.range(10000).withColumn("r_s", lit(1))
return left.join(right.hint("broadcast"), left.s == right.r_s, join_type)
- assert_gpu_and_cpu_row_counts_equal(do_join, conf={'spark.sql.adaptive.enabled': 'true'})
+ assert_gpu_and_cpu_row_counts_equal(do_join, conf={
+ 'spark.sql.adaptive.enabled': 'true',
+ kudo_enabled_conf_key: kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
@@ -181,21 +207,29 @@ def do_join(spark):
(all_gen, '1g'),
(join_small_batch_gens, '1000')), ids=idfn)
@pytest.mark.parametrize('join_type', all_join_types, ids=idfn)
-def test_sortmerge_join(data_gen, join_type, batch_size):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sortmerge_join(data_gen, join_type, batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 500)
return left.join(right, left.a == right.r_a, join_type)
- conf = copy_and_update(_sortmerge_join_conf, {'spark.rapids.sql.batchSizeBytes': batch_size})
+ conf = copy_and_update(_sortmerge_join_conf, {
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', basic_nested_gens + [decimal_gen_128bit], ids=idfn)
@pytest.mark.parametrize('join_type', all_join_types, ids=idfn)
-def test_sortmerge_join_ridealong(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sortmerge_join_ridealong(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_ridealong_df(spark, short_gen, data_gen, 500, 500)
return left.join(right, left.key == right.r_key, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf, {
+ kudo_enabled_conf_key: kudo_enabled
+ })
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
# For floating point values the normalization is done using a higher order function. We could probably work around this
# for now it falls back to the CPU
@@ -205,11 +239,15 @@ def do_join(spark):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', single_level_array_gens + [binary_gen], ids=idfn)
@pytest.mark.parametrize('join_type', all_join_types, ids=idfn)
-def test_sortmerge_join_wrong_key_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sortmerge_join_wrong_key_fallback(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 500)
return left.join(right, left.a == right.r_a, join_type)
- assert_gpu_fallback_collect(do_join, 'SortMergeJoinExec', conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf, {
+ kudo_enabled_conf_key: kudo_enabled
+ })
+ assert_gpu_fallback_collect(do_join, 'SortMergeJoinExec', conf=conf)
# For spark to insert a shuffled hash join it has to be enabled with
# "spark.sql.join.preferSortMergeJoin" = "false" and both sides have to
@@ -231,10 +269,12 @@ def do_join(spark):
@pytest.mark.parametrize('data_gen', basic_nested_gens + [decimal_gen_128bit], ids=idfn)
@pytest.mark.parametrize('join_type', all_non_sized_join_types, ids=idfn)
@pytest.mark.parametrize('sub_part_enabled', ['false', 'true'], ids=['SubPartition_OFF', 'SubPartition_ON'])
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_join_ridealong_non_sized(data_gen, join_type, sub_part_enabled):
+def test_hash_join_ridealong_non_sized(data_gen, join_type, sub_part_enabled, kudo_enabled):
confs = {
- "spark.rapids.sql.test.subPartitioning.enabled": sub_part_enabled
+ "spark.rapids.sql.test.subPartitioning.enabled": sub_part_enabled,
+ kudo_enabled_conf_key: kudo_enabled
}
hash_join_ridealong(data_gen, join_type, confs)
@@ -242,10 +282,12 @@ def test_hash_join_ridealong_non_sized(data_gen, join_type, sub_part_enabled):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', basic_nested_gens + [decimal_gen_128bit], ids=idfn)
@pytest.mark.parametrize('join_type', all_symmetric_sized_join_types, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_join_ridealong_symmetric(data_gen, join_type):
+def test_hash_join_ridealong_symmetric(data_gen, join_type, kudo_enabled):
confs = {
"spark.rapids.sql.join.useShuffledSymmetricHashJoin": "true",
+ kudo_enabled_conf_key: kudo_enabled
}
hash_join_ridealong(data_gen, join_type, confs)
@@ -253,10 +295,12 @@ def test_hash_join_ridealong_symmetric(data_gen, join_type):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', basic_nested_gens + [decimal_gen_128bit], ids=idfn)
@pytest.mark.parametrize('join_type', all_asymmetric_sized_join_types, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_join_ridealong_asymmetric(data_gen, join_type):
+def test_hash_join_ridealong_asymmetric(data_gen, join_type, kudo_enabled):
confs = {
"spark.rapids.sql.join.useShuffledAsymmetricHashJoin": "true",
+ kudo_enabled_conf_key: kudo_enabled
}
hash_join_ridealong(data_gen, join_type, confs)
@@ -267,24 +311,29 @@ def test_hash_join_ridealong_asymmetric(data_gen, join_type):
# Not all join types can be translated to a broadcast join, but this tests them to be sure we
# can handle what spark is doing
@pytest.mark.parametrize('join_type', all_join_types, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_join_right_table(data_gen, join_type):
+def test_broadcast_join_right_table(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(broadcast(right), left.a == right.r_a, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ conf = {kudo_enabled_conf_key: kudo_enabled}
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = conf)
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', basic_nested_gens + [decimal_gen_128bit], ids=idfn)
# Not all join types can be translated to a broadcast join, but this tests them to be sure we
# can handle what spark is doing
@pytest.mark.parametrize('join_type', all_join_types, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_join_right_table_ridealong(data_gen, join_type):
+def test_broadcast_join_right_table_ridealong(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_ridealong_df(spark, short_gen, data_gen, 500, 500)
return left.join(broadcast(right), left.key == right.r_key, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+
+ conf = {kudo_enabled_conf_key: kudo_enabled}
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -293,13 +342,16 @@ def do_join(spark):
# Not all join types can be translated to a broadcast join, but this tests them to be sure we
# can handle what spark is doing
@pytest.mark.parametrize('join_type', all_join_types, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_join_right_table_with_job_group(data_gen, join_type):
+def test_broadcast_join_right_table_with_job_group(data_gen, join_type, kudo_enabled):
with_cpu_session(lambda spark : spark.sparkContext.setJobGroup("testjob1", "test", False))
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(broadcast(right), left.a == right.r_a, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+
+ conf = {kudo_enabled_conf_key: kudo_enabled}
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -308,12 +360,16 @@ def do_join(spark):
@pytest.mark.parametrize('data_gen,batch_size', join_batch_size_test_params(
(all_gen + basic_nested_gens, '1g'),
(join_small_batch_gens + [basic_struct_gen, ArrayGen(string_gen)], '100')), ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_cartesian_join(data_gen, batch_size):
+def test_cartesian_join(data_gen, batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
return left.crossJoin(right)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.rapids.sql.batchSizeBytes': batch_size})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -322,11 +378,15 @@ def do_join(spark):
@pytest.mark.xfail(condition=is_databricks_runtime(),
reason='https://github.com/NVIDIA/spark-rapids/issues/334')
@pytest.mark.parametrize('batch_size', ['100', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches
-def test_cartesian_join_special_case_count(batch_size):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_cartesian_join_special_case_count(batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, int_gen, 50, 25)
return left.crossJoin(right).selectExpr('COUNT(*)')
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.rapids.sql.batchSizeBytes': batch_size})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -335,11 +395,15 @@ def do_join(spark):
@pytest.mark.xfail(condition=is_databricks_runtime(),
reason='https://github.com/NVIDIA/spark-rapids/issues/334')
@pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches
-def test_cartesian_join_special_case_group_by_count(batch_size):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_cartesian_join_special_case_group_by_count(batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, int_gen, 50, 25)
return left.crossJoin(right).groupBy('a').count()
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.rapids.sql.batchSizeBytes': batch_size})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -348,8 +412,9 @@ def do_join(spark):
@pytest.mark.parametrize('data_gen,batch_size', join_batch_size_test_params(
(all_gen, '1g'),
(join_small_batch_gens, '100')), ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_cartesian_join_with_condition(data_gen, batch_size):
+def test_cartesian_join_with_condition(data_gen, batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# This test is impacted by https://github.com/NVIDIA/spark-rapids/issues/294
@@ -357,7 +422,10 @@ def do_join(spark):
# but these take a long time to verify so we run with smaller numbers by default
# that do not expose the error
return left.join(right, left.b >= right.r_b, "cross")
- conf = copy_and_update(_sortmerge_join_conf, {'spark.rapids.sql.batchSizeBytes': batch_size})
+ conf = copy_and_update(_sortmerge_join_conf, {
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
@@ -366,22 +434,30 @@ def do_join(spark):
@pytest.mark.parametrize('data_gen,batch_size', join_batch_size_test_params(
(all_gen + basic_nested_gens, '1g'),
(join_small_batch_gens, '100')), ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_nested_loop_join(data_gen, batch_size):
+def test_broadcast_nested_loop_join(data_gen, batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
return left.crossJoin(broadcast(right))
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.rapids.sql.batchSizeBytes': batch_size})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('batch_size', ['100', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches
-def test_broadcast_nested_loop_join_special_case_count(batch_size):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_loop_join_special_case_count(batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, int_gen, 50, 25)
return left.crossJoin(broadcast(right)).selectExpr('COUNT(*)')
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.rapids.sql.batchSizeBytes': batch_size})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -389,11 +465,15 @@ def do_join(spark):
@pytest.mark.xfail(condition=is_databricks_runtime(),
reason='https://github.com/NVIDIA/spark-rapids/issues/334')
@pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches
-def test_broadcast_nested_loop_join_special_case_group_by_count(batch_size):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_loop_join_special_case_group_by_count(batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, int_gen, 50, 25)
return left.crossJoin(broadcast(right)).groupBy('a').count()
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.rapids.sql.batchSizeBytes': batch_size})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -402,8 +482,9 @@ def do_join(spark):
(join_ast_gen, '1g'),
([int_gen], 100)), ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti', 'Cross'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_right_broadcast_nested_loop_join_with_ast_condition(data_gen, join_type, batch_size):
+def test_right_broadcast_nested_loop_join_with_ast_condition(data_gen, join_type, batch_size, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# This test is impacted by https://github.com/NVIDIA/spark-rapids/issues/294
@@ -411,14 +492,18 @@ def do_join(spark):
# but these take a long time to verify so we run with smaller numbers by default
# that do not expose the error
return left.join(broadcast(right), (left.b >= right.r_b), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.rapids.sql.batchSizeBytes': batch_size})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.rapids.sql.batchSizeBytes': batch_size,
+ kudo_enabled_conf_key: kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', join_ast_gen, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_left_broadcast_nested_loop_join_with_ast_condition(data_gen):
+def test_left_broadcast_nested_loop_join_with_ast_condition(data_gen, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# This test is impacted by https://github.com/NVIDIA/spark-rapids/issues/294
@@ -426,14 +511,15 @@ def do_join(spark):
# but these take a long time to verify so we run with smaller numbers by default
# that do not expose the error
return broadcast(left).join(right, (left.b >= right.r_b), 'Right')
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [IntegerGen(), LongGen(), pytest.param(FloatGen(), marks=[incompat]), pytest.param(DoubleGen(), marks=[incompat])], ids=idfn)
@pytest.mark.parametrize('join_type', ['Inner', 'Cross'], ids=idfn)
-def test_broadcast_nested_loop_join_with_condition_post_filter(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_loop_join_with_condition_post_filter(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# This test is impacted by https://github.com/NVIDIA/spark-rapids/issues/294
@@ -442,12 +528,13 @@ def do_join(spark):
# that do not expose the error
# AST does not support cast or logarithm yet, so this must be implemented as a post-filter
return left.join(broadcast(right), left.a > f.log(right.r_a), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [IntegerGen(), LongGen(), pytest.param(FloatGen(), marks=[incompat]), pytest.param(DoubleGen(), marks=[incompat])], ids=idfn)
@pytest.mark.parametrize('join_type', ['Cross', 'Left', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_broadcast_nested_loop_join_with_condition(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_loop_join_with_condition(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# AST does not support cast or logarithm yet which is supposed to be extracted into child
@@ -458,39 +545,46 @@ def do_join(spark):
# (1) adapt double to integer since AST current doesn't support it.
# (2) switch to right side build to pass checks of 'Left', 'LeftSemi', 'LeftAnti' join types
return left.join(broadcast(right), f.round(left.a).cast('integer') > f.round(f.log(right.r_a).cast('integer')), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={"spark.rapids.sql.castFloatToIntegralTypes.enabled": True})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ "spark.rapids.sql.castFloatToIntegralTypes.enabled": True,
+ kudo_enabled_conf_key: kudo_enabled
+ })
@allow_non_gpu('BroadcastExchangeExec', 'BroadcastNestedLoopJoinExec', 'Cast', 'GreaterThan', 'Log')
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [IntegerGen(), LongGen(), pytest.param(FloatGen(), marks=[incompat]), pytest.param(DoubleGen(), marks=[incompat])], ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_broadcast_nested_loop_join_with_condition_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_loop_join_with_condition_fallback(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# AST does not support double type which is not split-able into child nodes.
return broadcast(left).join(right, left.a > f.log(right.r_a), join_type)
- assert_gpu_fallback_collect(do_join, 'BroadcastNestedLoopJoinExec')
+ assert_gpu_fallback_collect(do_join, 'BroadcastNestedLoopJoinExec',
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [byte_gen, short_gen, int_gen, long_gen,
float_gen, double_gen,
string_gen, boolean_gen, date_gen, timestamp_gen], ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_nested_loop_join_with_array_contains(data_gen, join_type):
+def test_broadcast_nested_loop_join_with_array_contains(data_gen, join_type, kudo_enabled):
arr_gen = ArrayGen(data_gen)
literal = with_cpu_session(lambda spark: gen_scalar(data_gen))
def do_join(spark):
left, right = create_df(spark, arr_gen, 50, 25)
# Array_contains will be pushed down into project child nodes
return broadcast(left).join(right, array_contains(left.a, literal.cast(data_gen.data_type)) < array_contains(right.r_a, literal.cast(data_gen.data_type)))
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', all_gen, ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_right_broadcast_nested_loop_join_condition_missing(data_gen, join_type):
+def test_right_broadcast_nested_loop_join_condition_missing(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# This test is impacted by https://github.com/NVIDIA/spark-rapids/issues/294
@@ -500,13 +594,14 @@ def do_join(spark):
# Compute the distinct of the join result to verify the join produces a proper dataframe
# for downstream processing.
return left.join(broadcast(right), how=join_type).distinct()
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', all_gen, ids=idfn)
@pytest.mark.parametrize('join_type', ['Right'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_left_broadcast_nested_loop_join_condition_missing(data_gen, join_type):
+def test_left_broadcast_nested_loop_join_condition_missing(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# This test is impacted by https://github.com/NVIDIA/spark-rapids/issues/294
@@ -516,45 +611,52 @@ def do_join(spark):
# Compute the distinct of the join result to verify the join produces a proper dataframe
# for downstream processing.
return broadcast(left).join(right, how=join_type).distinct()
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', all_gen + single_level_array_gens + [binary_gen], ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_right_broadcast_nested_loop_join_condition_missing_count(data_gen, join_type):
+def test_right_broadcast_nested_loop_join_condition_missing_count(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
return left.join(broadcast(right), how=join_type).selectExpr('COUNT(*)')
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', all_gen + single_level_array_gens + [binary_gen], ids=idfn)
@pytest.mark.parametrize('join_type', ['Right'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_left_broadcast_nested_loop_join_condition_missing_count(data_gen, join_type):
+def test_left_broadcast_nested_loop_join_condition_missing_count(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
return broadcast(left).join(right, how=join_type).selectExpr('COUNT(*)')
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
@allow_non_gpu('BroadcastExchangeExec', 'BroadcastNestedLoopJoinExec', 'GreaterThanOrEqual', *non_utc_allow)
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', all_gen, ids=idfn)
@pytest.mark.parametrize('join_type', ['LeftOuter', 'LeftSemi', 'LeftAnti', 'FullOuter'], ids=idfn)
-def test_broadcast_nested_loop_join_with_conditionals_build_left_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_loop_join_with_conditionals_build_left_fallback(data_gen, join_type,
+ kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
return broadcast(left).join(right, (left.b >= right.r_b), join_type)
- assert_gpu_fallback_collect(do_join, 'BroadcastNestedLoopJoinExec')
+ assert_gpu_fallback_collect(do_join, 'BroadcastNestedLoopJoinExec',
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@allow_non_gpu('BroadcastExchangeExec', 'BroadcastNestedLoopJoinExec', 'GreaterThanOrEqual', *non_utc_allow)
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', all_gen, ids=idfn)
@pytest.mark.parametrize('join_type', ['RightOuter', 'FullOuter'], ids=idfn)
-def test_broadcast_nested_loop_with_conditionals_build_right_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_loop_with_conditionals_build_right_fallback(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
return left.join(broadcast(right), (left.b >= right.r_b), join_type)
- assert_gpu_fallback_collect(do_join, 'BroadcastNestedLoopJoinExec')
+ assert_gpu_fallback_collect(do_join, 'BroadcastNestedLoopJoinExec',
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -566,25 +668,28 @@ def do_join(spark):
# Specify 200 shuffle partitions to test cases where streaming side is empty
# as in https://github.com/NVIDIA/spark-rapids/issues/7516
@pytest.mark.parametrize('shuffle_conf', [{}, {'spark.sql.shuffle.partitions': 200}], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_join_left_table(data_gen, join_type, shuffle_conf):
+def test_broadcast_join_left_table(data_gen, join_type, shuffle_conf, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 250, 500)
return broadcast(left).join(right, left.a == right.r_a, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf=shuffle_conf)
+ conf = copy_and_update(shuffle_conf, {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', join_ast_gen, ids=idfn)
@pytest.mark.parametrize('join_type', all_join_types, ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_join_with_conditionals(data_gen, join_type):
+def test_broadcast_join_with_conditionals(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(broadcast(right),
(left.a == right.r_a) & (left.b >= right.r_b), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -592,14 +697,15 @@ def do_join(spark):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [long_gen], ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_broadcast_join_with_condition_ast_op_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_join_with_condition_ast_op_fallback(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# AST does not support cast or logarithm yet
return left.join(broadcast(right),
(left.a == right.r_a) & (left.b > f.log(right.r_b)), join_type)
exec = 'SortMergeJoinExec' if join_type in ['Right', 'FullOuter'] else 'BroadcastHashJoinExec'
- assert_gpu_fallback_collect(do_join, exec)
+ assert_gpu_fallback_collect(do_join, exec, conf = {kudo_enabled_conf_key: kudo_enabled})
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -607,38 +713,42 @@ def do_join(spark):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', join_no_ast_gen, ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_broadcast_join_with_condition_ast_type_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_join_with_condition_ast_type_fallback(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 50, 25)
# AST does not support cast or logarithm yet
return left.join(broadcast(right),
(left.a == right.r_a) & (left.b > right.r_b), join_type)
exec = 'SortMergeJoinExec' if join_type in ['Right', 'FullOuter'] else 'BroadcastHashJoinExec'
- assert_gpu_fallback_collect(do_join, exec)
+ assert_gpu_fallback_collect(do_join, exec, conf = {kudo_enabled_conf_key: kudo_enabled})
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', join_no_ast_gen, ids=idfn)
@pytest.mark.parametrize('join_type', ['Inner', 'Cross'], ids=idfn)
-def test_broadcast_join_with_condition_post_filter(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_join_with_condition_post_filter(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(broadcast(right),
(left.a == right.r_a) & (left.b > right.r_b), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', join_ast_gen, ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_sortmerge_join_with_condition_ast(data_gen, join_type):
+def test_sortmerge_join_with_condition_ast(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(right, (left.a == right.r_a) & (left.b >= right.r_b), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf, {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -646,12 +756,14 @@ def do_join(spark):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [long_gen], ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_sortmerge_join_with_condition_ast_op_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sortmerge_join_with_condition_ast_op_fallback(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
# AST does not support cast or logarithm yet
return left.join(right, (left.a == right.r_a) & (left.b > f.log(right.r_b)), join_type)
- assert_gpu_fallback_collect(do_join, 'SortMergeJoinExec', conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf, {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_fallback_collect(do_join, 'SortMergeJoinExec', conf=conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -659,11 +771,13 @@ def do_join(spark):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', join_no_ast_gen, ids=idfn)
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_sortmerge_join_with_condition_ast_type_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sortmerge_join_with_condition_ast_type_fallback(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(right, (left.a == right.r_a) & (left.b > right.r_b), join_type)
- assert_gpu_fallback_collect(do_join, 'SortMergeJoinExec', conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf, {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_fallback_collect(do_join, 'SortMergeJoinExec', conf=conf)
_mixed_df1_with_nulls = [('a', RepeatSeqGen(LongGen(nullable=(True, 20.0)), length= 10)),
@@ -674,20 +788,22 @@ def do_join(spark):
@ignore_order
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'LeftSemi', 'LeftAnti', 'FullOuter', 'Cross'], ids=idfn)
-def test_broadcast_join_mixed(join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_join_mixed(join_type, kudo_enabled):
def do_join(spark):
left = gen_df(spark, _mixed_df1_with_nulls, length=500)
right = gen_df(spark, _mixed_df2_with_nulls, length=500).withColumnRenamed("a", "r_a")\
.withColumnRenamed("b", "r_b").withColumnRenamed("c", "r_c")
return left.join(broadcast(right), left.a.eqNullSafe(right.r_a), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={kudo_enabled_conf_key: kudo_enabled})
@ignore_order
@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec')
@pytest.mark.xfail(condition=is_emr_runtime(),
reason='https://github.com/NVIDIA/spark-rapids/issues/821')
@pytest.mark.parametrize('repartition', ["true", "false"], ids=idfn)
-def test_join_bucketed_table(repartition, spark_tmp_table_factory):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_join_bucketed_table(repartition, spark_tmp_table_factory, kudo_enabled):
def do_join(spark):
table_name = spark_tmp_table_factory.get()
data = [("http://fooblog.com/blog-entry-116.html", "https://fooblog.com/blog-entry-116.html"),
@@ -702,7 +818,10 @@ def do_join(spark):
return testurls.repartition(20).join(resolved, "Url", "inner")
else:
return testurls.join(resolved, "Url", "inner")
- assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.sql.autoBroadcastJoinThreshold': '-1'})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf={
+ 'spark.sql.autoBroadcastJoinThreshold': '-1',
+ kudo_enabled_conf_key: kudo_enabled
+ })
# Because we disable ShuffleExchangeExec in some cases we need to allow it to not be on the GPU
# and we do the result sorting in python to avoid that shuffle also being off the GPU
@@ -711,7 +830,8 @@ def do_join(spark):
@pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn)
@pytest.mark.parametrize('cache_side', ['cache_left', 'cache_right'], ids=idfn)
@pytest.mark.parametrize('cpu_side', ['cache', 'not_cache'], ids=idfn)
-def test_half_cache_join(join_type, cache_side, cpu_side):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_half_cache_join(join_type, cache_side, cpu_side, kudo_enabled):
left_gen = [('a', SetValuesGen(LongType(), range(500))), ('b', IntegerGen())]
right_gen = [('r_a', SetValuesGen(LongType(), range(500))), ('c', LongGen())]
def do_join(spark):
@@ -743,46 +863,56 @@ def do_join(spark):
# Even though Spark does not know the size of an RDD input so it will not do a broadcast join unless
# we tell it to, this is just to be safe
- assert_gpu_and_cpu_are_equal_collect(do_join, {'spark.sql.autoBroadcastJoinThreshold': '1'})
+ assert_gpu_and_cpu_are_equal_collect(do_join, {
+ 'spark.sql.autoBroadcastJoinThreshold': '1',
+ 'spark.rapids.shuffle.kudo.serializer.enabled': kudo_enabled
+ })
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', struct_gens, ids=idfn)
@pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_sortmerge_join_struct_as_key(data_gen, join_type):
+def test_sortmerge_join_struct_as_key(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(right, left.a == right.r_a, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf, {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', struct_gens, ids=idfn)
@pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_sortmerge_join_struct_mixed_key(data_gen, join_type):
+def test_sortmerge_join_struct_mixed_key(data_gen, join_type, kudo_enabled):
def do_join(spark):
left = two_col_df(spark, data_gen, int_gen, length=500)
right = two_col_df(spark, data_gen, int_gen, length=500)
return left.join(right, (left.a == right.a) & (left.b == right.b), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf, {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', struct_gens, ids=idfn)
@pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_sortmerge_join_struct_mixed_key_with_null_filter(data_gen, join_type):
+def test_sortmerge_join_struct_mixed_key_with_null_filter(data_gen, join_type, kudo_enabled):
def do_join(spark):
left = two_col_df(spark, data_gen, int_gen, length=500)
right = two_col_df(spark, data_gen, int_gen, length=500)
return left.join(right, (left.a == right.a) & (left.b == right.b), join_type)
# Disable constraintPropagation to test null filter on built table with nullable structures.
- conf = {'spark.sql.constraintPropagation.enabled': 'false', **_sortmerge_join_conf}
+ conf = {'spark.sql.constraintPropagation.enabled': 'false',
+ 'spark.rapids.shuffle.kudo.serializer.enabled': kudo_enabled,
+ **_sortmerge_join_conf}
assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
@@ -790,25 +920,27 @@ def do_join(spark):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', struct_gens, ids=idfn)
@pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_join_right_struct_as_key(data_gen, join_type):
+def test_broadcast_join_right_struct_as_key(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(broadcast(right), left.a == right.r_a, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', struct_gens, ids=idfn)
@pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_broadcast_join_right_struct_mixed_key(data_gen, join_type):
+def test_broadcast_join_right_struct_mixed_key(data_gen, join_type, kudo_enabled):
def do_join(spark):
left = two_col_df(spark, data_gen, int_gen, length=500)
right = two_col_df(spark, data_gen, int_gen, length=250)
return left.join(broadcast(right), (left.a == right.a) & (left.b == right.b), join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
# local sort because of https://github.com/NVIDIA/spark-rapids/issues/84
# After 3.1.0 is the min spark version we can drop this
@@ -816,11 +948,14 @@ def do_join(spark):
@pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/2140')
@pytest.mark.parametrize('data_gen', [basic_struct_gen_with_floats], ids=idfn)
@pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn)
-def test_sortmerge_join_struct_with_floats_key(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sortmerge_join_struct_with_floats_key(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 250)
return left.join(right, left.a == right.r_a, join_type)
- assert_gpu_and_cpu_are_equal_collect(do_join, conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf,
+ {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
@allow_non_gpu('SortMergeJoinExec', 'SortExec', 'NormalizeNaNAndZero', 'CreateNamedStruct',
'GetStructField', 'Literal', 'If', 'IsNull', 'ShuffleExchangeExec', 'HashPartitioning',
@@ -828,15 +963,19 @@ def do_join(spark):
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', struct_gens, ids=idfn)
@pytest.mark.parametrize('join_type', ['FullOuter'], ids=idfn)
-def test_sortmerge_join_struct_as_key_fallback(data_gen, join_type):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sortmerge_join_struct_as_key_fallback(data_gen, join_type, kudo_enabled):
def do_join(spark):
left, right = create_df(spark, data_gen, 500, 500)
return left.join(right, left.a == right.r_a, join_type)
- assert_gpu_fallback_collect(do_join, 'SortMergeJoinExec', conf=_sortmerge_join_conf)
+ conf = copy_and_update(_sortmerge_join_conf,
+ {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_fallback_collect(do_join, 'SortMergeJoinExec', conf=conf)
# Regression test for https://github.com/NVIDIA/spark-rapids/issues/3775
@ignore_order(local=True)
-def test_struct_self_join(spark_tmp_table_factory):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_struct_self_join(spark_tmp_table_factory, kudo_enabled):
def do_join(spark):
data = [
(("Adam ", "", "Green"), "1", "M", 1000),
@@ -863,7 +1002,7 @@ def do_join(spark):
resultdf.createOrReplaceTempView(resultdf_name)
return spark.sql("select a.* from {} a, {} b where a.name=b.name".format(
resultdf_name, resultdf_name))
- assert_gpu_and_cpu_are_equal_collect(do_join)
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf = {kudo_enabled_conf_key: kudo_enabled})
# ExistenceJoin occurs in the context of existential subqueries (which is rewritten to SemiJoin) if
# there is an additional condition that may qualify left records even though they don't have
@@ -883,7 +1022,9 @@ def do_join(spark):
])
@pytest.mark.parametrize('conditionalJoin', [False, True], ids=['ast:off', 'ast:on'])
@pytest.mark.parametrize('forceBroadcastHashJoin', [False, True], ids=['broadcastHJ:off', 'broadcastHJ:on'])
-def test_existence_join(numComplementsToExists, aqeEnabled, conditionalJoin, forceBroadcastHashJoin, spark_tmp_table_factory):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_existence_join(numComplementsToExists, aqeEnabled, conditionalJoin,
+ forceBroadcastHashJoin, spark_tmp_table_factory, kudo_enabled):
leftTable = spark_tmp_table_factory.get()
rightTable = spark_tmp_table_factory.get()
def do_join(spark):
@@ -933,12 +1074,14 @@ def do_join(spark):
assert_cpu_and_gpu_are_equal_collect_with_capture(do_join, existenceJoinRegex,
conf={
"spark.sql.adaptive.enabled": aqeEnabled,
- "spark.sql.autoBroadcastJoinThreshold": bhjThreshold
+ "spark.sql.autoBroadcastJoinThreshold": bhjThreshold,
+ kudo_enabled_conf_key: kudo_enabled
})
@ignore_order
@pytest.mark.parametrize('aqeEnabled', [True, False], ids=['aqe:on', 'aqe:off'])
-def test_existence_join_in_broadcast_nested_loop_join(spark_tmp_table_factory, aqeEnabled):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_existence_join_in_broadcast_nested_loop_join(spark_tmp_table_factory, aqeEnabled, kudo_enabled):
left_table_name = spark_tmp_table_factory.get()
right_table_name = spark_tmp_table_factory.get()
@@ -958,11 +1101,13 @@ def do_join(spark):
capture_regexp = r"GpuBroadcastNestedLoopJoin ExistenceJoin\(exists#[0-9]+\),"
assert_cpu_and_gpu_are_equal_collect_with_capture(do_join, capture_regexp,
- conf={"spark.sql.adaptive.enabled": aqeEnabled})
+ conf={"spark.sql.adaptive.enabled": aqeEnabled,
+ kudo_enabled_conf_key: kudo_enabled})
@ignore_order
@pytest.mark.parametrize('aqeEnabled', [True, False], ids=['aqe:on', 'aqe:off'])
-def test_degenerate_broadcast_nested_loop_existence_join(spark_tmp_table_factory, aqeEnabled):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_degenerate_broadcast_nested_loop_existence_join(spark_tmp_table_factory, aqeEnabled, kudo_enabled):
left_table_name = spark_tmp_table_factory.get()
right_table_name = spark_tmp_table_factory.get()
@@ -982,13 +1127,15 @@ def do_join(spark):
capture_regexp = r"GpuBroadcastNestedLoopJoin ExistenceJoin\(exists#[0-9]+\),"
assert_cpu_and_gpu_are_equal_collect_with_capture(do_join, capture_regexp,
- conf={"spark.sql.adaptive.enabled": aqeEnabled})
+ conf={"spark.sql.adaptive.enabled": aqeEnabled,
+ kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize('data_gen', [StringGen(), IntegerGen()], ids=idfn)
@pytest.mark.parametrize("aqe_enabled", [True, False], ids=idfn)
@pytest.mark.parametrize("join_reorder_enabled", [True, False], ids=idfn)
-def test_multi_table_hash_join(data_gen, aqe_enabled, join_reorder_enabled):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_multi_table_hash_join(data_gen, aqe_enabled, join_reorder_enabled, kudo_enabled):
def do_join(spark):
t1 = binary_op_df(spark, data_gen, length=1000)
t2 = binary_op_df(spark, data_gen, length=800)
@@ -999,14 +1146,15 @@ def do_join(spark):
.join(t4, t3.a == t4.a, 'Inner')
conf = copy_and_update(_hash_join_conf, {
'spark.sql.adaptive.enabled': aqe_enabled,
- 'spark.rapids.sql.optimizer.joinReorder.enabled': join_reorder_enabled
+ 'spark.rapids.sql.optimizer.joinReorder.enabled': join_reorder_enabled,
+ kudo_enabled_conf_key: kudo_enabled
})
assert_gpu_and_cpu_are_equal_collect(do_join, conf=conf)
limited_integral_gens = [byte_gen, ShortGen(max_val=BYTE_MAX), IntegerGen(max_val=BYTE_MAX), LongGen(max_val=BYTE_MAX)]
-def hash_join_different_key_integral_types(left_gen, right_gen, join_type):
+def hash_join_different_key_integral_types(left_gen, right_gen, join_type, kudo_enabled):
def do_join(spark):
left = unary_op_df(spark, left_gen, length=50)
right = unary_op_df(spark, right_gen, length=500)
@@ -1014,7 +1162,8 @@ def do_join(spark):
_all_conf = copy_and_update(_hash_join_conf, {
"spark.rapids.sql.join.useShuffledSymmetricHashJoin": "true",
"spark.rapids.sql.join.useShuffledAsymmetricHashJoin": "true",
- "spark.rapids.sql.test.subPartitioning.enabled": True
+ "spark.rapids.sql.test.subPartitioning.enabled": True,
+ kudo_enabled_conf_key: kudo_enabled
})
assert_gpu_and_cpu_are_equal_collect(do_join, conf=_all_conf)
@@ -1023,24 +1172,27 @@ def do_join(spark):
@pytest.mark.parametrize('left_gen', limited_integral_gens, ids=idfn)
@pytest.mark.parametrize('right_gen', limited_integral_gens, ids=idfn)
@pytest.mark.parametrize('join_type', all_non_sized_join_types, ids=idfn)
-def test_hash_join_different_key_integral_types_non_sized(left_gen, right_gen, join_type):
- hash_join_different_key_integral_types(left_gen, right_gen, join_type)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_join_different_key_integral_types_non_sized(left_gen, right_gen, join_type, kudo_enabled):
+ hash_join_different_key_integral_types(left_gen, right_gen, join_type, kudo_enabled)
@validate_execs_in_gpu_plan('GpuShuffledSymmetricHashJoinExec')
@ignore_order(local=True)
@pytest.mark.parametrize('left_gen', limited_integral_gens, ids=idfn)
@pytest.mark.parametrize('right_gen', limited_integral_gens, ids=idfn)
@pytest.mark.parametrize('join_type', all_symmetric_sized_join_types, ids=idfn)
-def test_hash_join_different_key_integral_types_symmetric(left_gen, right_gen, join_type):
- hash_join_different_key_integral_types(left_gen, right_gen, join_type)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_join_different_key_integral_types_symmetric(left_gen, right_gen, join_type, kudo_enabled):
+ hash_join_different_key_integral_types(left_gen, right_gen, join_type, kudo_enabled)
@validate_execs_in_gpu_plan('GpuShuffledAsymmetricHashJoinExec')
@ignore_order(local=True)
@pytest.mark.parametrize('left_gen', limited_integral_gens, ids=idfn)
@pytest.mark.parametrize('right_gen', limited_integral_gens, ids=idfn)
@pytest.mark.parametrize('join_type', all_asymmetric_sized_join_types, ids=idfn)
-def test_hash_join_different_key_integral_types_asymmetric(left_gen, right_gen, join_type):
- hash_join_different_key_integral_types(left_gen, right_gen, join_type)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_join_different_key_integral_types_asymmetric(left_gen, right_gen, join_type, kudo_enabled):
+ hash_join_different_key_integral_types(left_gen, right_gen, join_type, kudo_enabled)
bloom_filter_confs = {
@@ -1068,8 +1220,10 @@ def do_join(spark):
@pytest.mark.parametrize("is_multi_column", [False, True], ids=idfn)
@pytest.mark.skipif(is_databricks_runtime(), reason="https://github.com/NVIDIA/spark-rapids/issues/8921")
@pytest.mark.skipif(is_before_spark_330(), reason="Bloom filter joins added in Spark 3.3.0")
-def test_bloom_filter_join(batch_size, is_multi_column):
- conf = {"spark.rapids.sql.batchSizeBytes": batch_size}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_bloom_filter_join(batch_size, is_multi_column, kudo_enabled):
+ conf = {"spark.rapids.sql.batchSizeBytes": batch_size,
+ kudo_enabled_conf_key: kudo_enabled}
check_bloom_filter_join(confs=conf,
expected_classes="GpuBloomFilterMightContain,GpuBloomFilterAggregate",
is_multi_column=is_multi_column)
@@ -1079,8 +1233,10 @@ def test_bloom_filter_join(batch_size, is_multi_column):
@pytest.mark.parametrize("is_multi_column", [False, True], ids=idfn)
@pytest.mark.skipif(is_databricks_runtime(), reason="https://github.com/NVIDIA/spark-rapids/issues/8921")
@pytest.mark.skipif(is_before_spark_330(), reason="Bloom filter joins added in Spark 3.3.0")
-def test_bloom_filter_join_cpu_probe(is_multi_column):
- conf = {"spark.rapids.sql.expression.BloomFilterMightContain": "false"}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_bloom_filter_join_cpu_probe(is_multi_column, kudo_enabled):
+ conf = {"spark.rapids.sql.expression.BloomFilterMightContain": "false",
+ kudo_enabled_conf_key: kudo_enabled}
check_bloom_filter_join(confs=conf,
expected_classes="BloomFilterMightContain,GpuBloomFilterAggregate",
is_multi_column=is_multi_column)
@@ -1090,8 +1246,10 @@ def test_bloom_filter_join_cpu_probe(is_multi_column):
@pytest.mark.parametrize("is_multi_column", [False, True], ids=idfn)
@pytest.mark.skipif(is_databricks_runtime(), reason="https://github.com/NVIDIA/spark-rapids/issues/8921")
@pytest.mark.skipif(is_before_spark_330(), reason="Bloom filter joins added in Spark 3.3.0")
-def test_bloom_filter_join_cpu_build(is_multi_column):
- conf = {"spark.rapids.sql.expression.BloomFilterAggregate": "false"}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_bloom_filter_join_cpu_build(is_multi_column, kudo_enabled):
+ conf = {"spark.rapids.sql.expression.BloomFilterAggregate": "false",
+ kudo_enabled_conf_key: kudo_enabled}
check_bloom_filter_join(confs=conf,
expected_classes="GpuBloomFilterMightContain,BloomFilterAggregate",
is_multi_column=is_multi_column)
@@ -1102,8 +1260,10 @@ def test_bloom_filter_join_cpu_build(is_multi_column):
@pytest.mark.parametrize("is_multi_column", [False, True], ids=idfn)
@pytest.mark.skipif(is_databricks_runtime(), reason="https://github.com/NVIDIA/spark-rapids/issues/8921")
@pytest.mark.skipif(is_before_spark_330(), reason="Bloom filter joins added in Spark 3.3.0")
-def test_bloom_filter_join_split_cpu_build(agg_replace_mode, is_multi_column):
- conf = {"spark.rapids.sql.hashAgg.replaceMode": agg_replace_mode}
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_bloom_filter_join_split_cpu_build(agg_replace_mode, is_multi_column, kudo_enabled):
+ conf = {"spark.rapids.sql.hashAgg.replaceMode": agg_replace_mode,
+ kudo_enabled_conf_key: kudo_enabled}
check_bloom_filter_join(confs=conf,
expected_classes="GpuBloomFilterMightContain,BloomFilterAggregate,GpuBloomFilterAggregate",
is_multi_column=is_multi_column)
@@ -1111,14 +1271,16 @@ def test_bloom_filter_join_split_cpu_build(agg_replace_mode, is_multi_column):
@ignore_order(local=True)
@pytest.mark.skipif(is_databricks_runtime(), reason="https://github.com/NVIDIA/spark-rapids/issues/8921")
@pytest.mark.skipif(is_before_spark_330(), reason="Bloom filter joins added in Spark 3.3.0")
-def test_bloom_filter_join_with_merge_some_null_filters(spark_tmp_path):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_bloom_filter_join_with_merge_some_null_filters(spark_tmp_path, kudo_enabled):
data_path1 = spark_tmp_path + "/BLOOM_JOIN_DATA1"
data_path2 = spark_tmp_path + "/BLOOM_JOIN_DATA2"
with_cpu_session(lambda spark: spark.range(100000).coalesce(1).write.parquet(data_path1))
with_cpu_session(lambda spark: spark.range(100000).withColumn("id2", col("id").cast("string"))\
.coalesce(1).write.parquet(data_path2))
confs = copy_and_update(bloom_filter_confs,
- {"spark.sql.files.maxPartitionBytes": "1000"})
+ {"spark.sql.files.maxPartitionBytes": "1000",
+ kudo_enabled_conf_key: kudo_enabled})
def do_join(spark):
left = spark.read.parquet(data_path1)
right = spark.read.parquet(data_path2)
@@ -1128,7 +1290,8 @@ def do_join(spark):
@ignore_order(local=True)
@pytest.mark.skipif(is_databricks_runtime(), reason="https://github.com/NVIDIA/spark-rapids/issues/8921")
@pytest.mark.skipif(is_before_spark_330(), reason="Bloom filter joins added in Spark 3.3.0")
-def test_bloom_filter_join_with_merge_all_null_filters(spark_tmp_path):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_bloom_filter_join_with_merge_all_null_filters(spark_tmp_path, kudo_enabled):
data_path1 = spark_tmp_path + "/BLOOM_JOIN_DATA1"
data_path2 = spark_tmp_path + "/BLOOM_JOIN_DATA2"
with_cpu_session(lambda spark: spark.range(100000).write.parquet(data_path1))
@@ -1138,13 +1301,15 @@ def do_join(spark):
left = spark.read.parquet(data_path1)
right = spark.read.parquet(data_path2)
return right.filter("cast(id2 as bigint) % 3 = 4").join(left, left.id == right.id, "inner")
- assert_gpu_and_cpu_are_equal_collect(do_join, bloom_filter_confs)
+ conf = copy_and_update(bloom_filter_confs, {kudo_enabled_conf_key: kudo_enabled})
+ assert_gpu_and_cpu_are_equal_collect(do_join, conf)
@ignore_order(local=True)
@allow_non_gpu("ProjectExec", "FilterExec", "BroadcastHashJoinExec", "ColumnarToRowExec", "BroadcastExchangeExec", "BatchScanExec")
@pytest.mark.parametrize("disable_build", [True, False])
-def test_broadcast_hash_join_fix_fallback_by_inputfile(spark_tmp_path, disable_build):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_hash_join_fix_fallback_by_inputfile(spark_tmp_path, disable_build, kudo_enabled):
data_path_parquet = spark_tmp_path + "/parquet"
data_path_orc = spark_tmp_path + "/orc"
# The smaller one (orc) will be the build side (a broadcast)
@@ -1174,13 +1339,15 @@ def do_join(spark):
do_join,
conf={"spark.sql.autoBroadcastJoinThreshold": "10M",
"spark.sql.sources.useV1SourceList": "",
- "spark.rapids.sql.input." + scan_name: False})
+ "spark.rapids.sql.input." + scan_name: False,
+ kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@allow_non_gpu("ProjectExec", "BroadcastNestedLoopJoinExec", "ColumnarToRowExec", "BroadcastExchangeExec", "BatchScanExec")
@pytest.mark.parametrize("disable_build", [True, False])
-def test_broadcast_nested_join_fix_fallback_by_inputfile(spark_tmp_path, disable_build):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_broadcast_nested_join_fix_fallback_by_inputfile(spark_tmp_path, disable_build, kudo_enabled):
data_path_parquet = spark_tmp_path + "/parquet"
data_path_orc = spark_tmp_path + "/orc"
# The smaller one (orc) will be the build side (a broadcast)
@@ -1209,14 +1376,17 @@ def do_join(spark):
do_join,
conf={"spark.sql.autoBroadcastJoinThreshold": "-1",
"spark.sql.sources.useV1SourceList": "",
- "spark.rapids.sql.input." + scan_name: False})
+ "spark.rapids.sql.input." + scan_name: False,
+ kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True)
@pytest.mark.parametrize("join_type", ["Inner", "LeftOuter", "RightOuter"], ids=idfn)
@pytest.mark.parametrize("batch_size", ["500", "1g"], ids=idfn)
-def test_distinct_join(join_type, batch_size):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_distinct_join(join_type, batch_size, kudo_enabled):
join_conf = {
- "spark.rapids.sql.batchSizeBytes": batch_size
+ "spark.rapids.sql.batchSizeBytes": batch_size,
+ kudo_enabled_conf_key: kudo_enabled
}
def do_join(spark):
left_df = spark.range(1024).withColumn("x", f.col("id") + 1)
@@ -1230,13 +1400,15 @@ def do_join(spark):
@pytest.mark.parametrize("is_right_host_shuffle", [False, True], ids=idfn)
@pytest.mark.parametrize("is_left_smaller", [False, True], ids=idfn)
@pytest.mark.parametrize("batch_size", ["1024", "1g"], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
def test_sized_join(join_type, is_left_host_shuffle, is_right_host_shuffle,
- is_left_smaller, batch_size):
+ is_left_smaller, batch_size, kudo_enabled):
join_conf = {
"spark.rapids.sql.join.useShuffledSymmetricHashJoin": "true",
"spark.rapids.sql.join.useShuffledAsymmetricHashJoin": "true",
"spark.sql.autoBroadcastJoinThreshold": "1",
- "spark.rapids.sql.batchSizeBytes": batch_size
+ "spark.rapids.sql.batchSizeBytes": batch_size,
+ kudo_enabled_conf_key: kudo_enabled
}
left_size, right_size = (2048, 1024) if is_left_smaller else (1024, 2048)
def do_join(spark):
@@ -1266,7 +1438,8 @@ def do_join(spark):
@pytest.mark.parametrize("is_left_smaller", [False, True], ids=idfn)
@pytest.mark.parametrize("is_ast_supported", [False, True], ids=idfn)
@pytest.mark.parametrize("batch_size", ["1024", "1g"], ids=idfn)
-def test_sized_join_conditional(join_type, is_ast_supported, is_left_smaller, batch_size):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_sized_join_conditional(join_type, is_ast_supported, is_left_smaller, batch_size, kudo_enabled):
if join_type != "Inner" and not is_ast_supported:
pytest.skip("Only inner joins support a non-AST condition")
join_conf = {
@@ -1274,7 +1447,8 @@ def test_sized_join_conditional(join_type, is_ast_supported, is_left_smaller, ba
"spark.rapids.sql.join.useShuffledAsymmetricHashJoin": "true",
"spark.rapids.sql.join.use"
"spark.sql.autoBroadcastJoinThreshold": "1",
- "spark.rapids.sql.batchSizeBytes": batch_size
+ "spark.rapids.sql.batchSizeBytes": batch_size,
+ kudo_enabled_conf_key: kudo_enabled
}
left_size, right_size = (2048, 1024) if is_left_smaller else (1024, 2048)
def do_join(spark):
@@ -1300,13 +1474,15 @@ def do_join(spark):
@pytest.mark.parametrize("is_left_replicated", [False, True], ids=idfn)
@pytest.mark.parametrize("is_conditional", [False, True], ids=idfn)
@pytest.mark.parametrize("is_outer_side_small", [False, True], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
def test_sized_join_high_key_replication(join_type, is_left_replicated, is_conditional,
- is_outer_side_small):
+ is_outer_side_small, kudo_enabled):
join_conf = {
"spark.rapids.sql.join.useShuffledSymmetricHashJoin": "true",
"spark.rapids.sql.join.useShuffledAsymmetricHashJoin": "true",
"spark.rapids.sql.join.use"
- "spark.sql.autoBroadcastJoinThreshold": "1"
+ "spark.sql.autoBroadcastJoinThreshold": "1",
+ kudo_enabled_conf_key: kudo_enabled
}
left_size, right_size = (30000, 40000)
left_key_gen, right_key_gen = (
diff --git a/integration_tests/src/main/python/repart_test.py b/integration_tests/src/main/python/repart_test.py
index 17991bc64de..19759b77f5d 100644
--- a/integration_tests/src/main/python/repart_test.py
+++ b/integration_tests/src/main/python/repart_test.py
@@ -57,6 +57,8 @@
struct_of_maps = StructGen([['child0', BooleanGen()]] + [
['child%d' % (i + 1), gen] for i, gen in enumerate(map_gens)])
+kudo_enabled_conf_key = "spark.rapids.shuffle.kudo.serializer.enabled"
+
@pytest.mark.parametrize('data_gen', [pytest.param((StructGen([['child0', DecimalGen(7, 2)]]),
StructGen([['child1', IntegerGen()]]))),
# left_struct(child0 = 4 level nested struct, child1 = Int)
@@ -78,11 +80,13 @@
StructGen([['child1', MapGen(BooleanGen(nullable=False), boolean_gen)]], nullable=False))], ids=idfn)
# This tests the union of DF of structs with different types of cols as long as the struct itself
# isn't null. This is a limitation in cudf because we don't support nested types as literals
-def test_union_struct_missing_children(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_union_struct_missing_children(data_gen, kudo_enabled):
left_gen, right_gen = data_gen
assert_gpu_and_cpu_are_equal_collect(
lambda spark : binary_op_df(spark, left_gen).unionByName(binary_op_df(
- spark, right_gen), True))
+ spark, right_gen), True),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', all_gen + map_gens + array_gens_sample +
[all_basic_struct_gen,
@@ -90,9 +94,11 @@ def test_union_struct_missing_children(data_gen):
nested_struct,
struct_of_maps], ids=idfn)
# This tests union of two DFs of two cols each. The types of the left col and right col is the same
-def test_union(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_union(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark : binary_op_df(spark, data_gen).union(binary_op_df(spark, data_gen)))
+ lambda spark : binary_op_df(spark, data_gen).union(binary_op_df(spark, data_gen)),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', all_gen + map_gens + array_gens_sample +
[all_basic_struct_gen,
@@ -100,9 +106,11 @@ def test_union(data_gen):
nested_struct,
struct_of_maps], ids=idfn)
# This tests union of two DFs of two cols each. The types of the left col and right col is the same
-def test_unionAll(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_unionAll(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark : binary_op_df(spark, data_gen).unionAll(binary_op_df(spark, data_gen)))
+ lambda spark : binary_op_df(spark, data_gen).unionAll(binary_op_df(spark, data_gen)),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', all_gen + map_gens + array_gens_sample +
[all_basic_struct_gen,
@@ -114,10 +122,13 @@ def test_unionAll(data_gen):
struct_of_maps], ids=idfn)
# This tests the union of two DFs of structs with missing child column names. The missing child
# column will be replaced by nulls in the output DF. This is a feature added in 3.1+
-def test_union_by_missing_col_name(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_union_by_missing_col_name(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
lambda spark : binary_op_df(spark, data_gen).withColumnRenamed("a", "x")
- .unionByName(binary_op_df(spark, data_gen).withColumnRenamed("a", "y"), True))
+ .unionByName(binary_op_df(spark, data_gen).withColumnRenamed("a",
+ "y"), True),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
# the first number ('1' and '2') is the nest level
@@ -133,7 +144,8 @@ def test_union_by_missing_col_name(data_gen):
nest_1_one, nest_1_two,
nest_2_one, nest_2_two])
@pytest.mark.skipif(is_before_spark_330(), reason="This is supported only in Spark 3.3.0+")
-def test_union_by_missing_field_name_in_arrays_structs(gen_pair):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_union_by_missing_field_name_in_arrays_structs(gen_pair, kudo_enabled):
"""
This tests the union of two DFs of arrays of structs with missing field names.
The missing field will be replaced be nulls in the output DF. This is a feature added in 3.3+
@@ -142,8 +154,8 @@ def test_union_by_missing_field_name_in_arrays_structs(gen_pair):
"""
def assert_union_equal(gen1, gen2):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark: unary_op_df(spark, gen1).unionByName(unary_op_df(spark, gen2), True)
- )
+ lambda spark: unary_op_df(spark, gen1).unionByName(unary_op_df(spark, gen2), True),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
assert_union_equal(gen_pair[0], gen_pair[1])
assert_union_equal(gen_pair[1], gen_pair[0])
@@ -155,9 +167,12 @@ def assert_union_equal(gen1, gen2):
StructGen([['child0', DecimalGen(7, 2)]]),
nested_struct,
struct_of_maps], ids=idfn)
-def test_union_by_name(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_union_by_name(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark : binary_op_df(spark, data_gen).unionByName(binary_op_df(spark, data_gen)))
+ lambda spark : binary_op_df(spark, data_gen).unionByName(binary_op_df(spark,
+ data_gen)),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', [
@@ -166,19 +181,23 @@ def test_union_by_name(data_gen):
pytest.param([('array' + str(i), gen) for i, gen in enumerate(array_gens_sample + [ArrayGen(BinaryGen(max_length=5), max_length=5)])]),
pytest.param([('map' + str(i), gen) for i, gen in enumerate(map_gens_sample)]),
], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_coalesce_types(data_gen):
+def test_coalesce_types(data_gen, kudo_enabled):
assert_gpu_and_cpu_are_equal_collect(
- lambda spark: gen_df(spark, data_gen).coalesce(2))
+ lambda spark: gen_df(spark, data_gen).coalesce(2),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('num_parts', [1, 10, 100, 1000, 2000], ids=idfn)
@pytest.mark.parametrize('length', [0, 2048, 4096], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_coalesce_df(num_parts, length):
+def test_coalesce_df(num_parts, length, kudo_enabled):
#This should change eventually to be more than just the basic gens
gen_list = [('_c' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens + [binary_gen])]
assert_gpu_and_cpu_are_equal_collect(
- lambda spark : gen_df(spark, gen_list, length=length).coalesce(num_parts))
+ lambda spark : gen_df(spark, gen_list, length=length).coalesce(num_parts),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', [
pytest.param([('_c' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens + [binary_gen])]),
@@ -188,15 +207,17 @@ def test_coalesce_df(num_parts, length):
], ids=idfn)
@pytest.mark.parametrize('num_parts', [1, 10, 2345], ids=idfn)
@pytest.mark.parametrize('length', [0, 2048, 4096], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test.
@allow_non_gpu(*non_utc_allow)
-def test_repartition_df(data_gen, num_parts, length):
+def test_repartition_df(data_gen, num_parts, length, kudo_enabled):
from pyspark.sql.functions import lit
assert_gpu_and_cpu_are_equal_collect(
# Add a computed column to avoid shuffle being optimized back to a CPU shuffle
lambda spark : gen_df(spark, data_gen, length=length).withColumn('x', lit(1)).repartition(num_parts),
# disable sort before shuffle so round robin works for maps
- conf = {'spark.sql.execution.sortBeforeRepartition': 'false'})
+ conf = {'spark.sql.execution.sortBeforeRepartition': 'false',
+ kudo_enabled_conf_key: kudo_enabled})
@pytest.mark.parametrize('data_gen', [
pytest.param([('_c' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens)]),
@@ -205,45 +226,53 @@ def test_repartition_df(data_gen, num_parts, length):
], ids=idfn)
@pytest.mark.parametrize('num_parts', [1, 10, 2345], ids=idfn)
@pytest.mark.parametrize('length', [0, 2048, 4096], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test.
@allow_non_gpu(*non_utc_allow)
-def test_repartition_df_for_round_robin(data_gen, num_parts, length):
+def test_repartition_df_for_round_robin(data_gen, num_parts, length, kudo_enabled):
from pyspark.sql.functions import lit
assert_gpu_and_cpu_are_equal_collect(
# Add a computed column to avoid shuffle being optimized back to a CPU shuffle
lambda spark : gen_df(spark, data_gen, length=length).withColumn('x', lit(1)).repartition(num_parts),
# Enable sort for round robin partition
- conf = {'spark.sql.execution.sortBeforeRepartition': 'true'}) # default is true
+ conf = {'spark.sql.execution.sortBeforeRepartition': 'true',
+ kudo_enabled_conf_key: kudo_enabled}) # default is true
@allow_non_gpu('ShuffleExchangeExec', 'RoundRobinPartitioning')
@pytest.mark.parametrize('data_gen', [[('a', simple_string_to_string_map_gen)]], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test.
-def test_round_robin_sort_fallback(data_gen):
+def test_round_robin_sort_fallback(data_gen, kudo_enabled):
from pyspark.sql.functions import lit
assert_gpu_fallback_collect(
# Add a computed column to avoid shuffle being optimized back to a CPU shuffle like in test_repartition_df
lambda spark : gen_df(spark, data_gen).withColumn('extra', lit(1)).repartition(13),
- 'ShuffleExchangeExec')
+ 'ShuffleExchangeExec',
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@allow_non_gpu("ProjectExec", "ShuffleExchangeExec")
@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test.
@pytest.mark.parametrize('num_parts', [2, 10, 17, 19, 32], ids=idfn)
@pytest.mark.parametrize('gen', [([('ag', ArrayGen(StructGen([('b1', long_gen)])))], ['ag'])], ids=idfn)
-def test_hash_repartition_exact_fallback(gen, num_parts):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_repartition_exact_fallback(gen, num_parts, kudo_enabled):
data_gen = gen[0]
part_on = gen[1]
assert_gpu_fallback_collect(
lambda spark : gen_df(spark, data_gen, length=1024) \
.repartition(num_parts, *part_on) \
.withColumn('id', f.spark_partition_id()) \
- .selectExpr('*'), "ShuffleExchangeExec")
+ .selectExpr('*'), "ShuffleExchangeExec",
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@allow_non_gpu("ProjectExec")
@pytest.mark.parametrize('data_gen', [ArrayGen(StructGen([('b1', long_gen)]))], ids=idfn)
-def test_hash_fallback(data_gen):
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_hash_fallback(data_gen, kudo_enabled):
assert_gpu_fallback_collect(
lambda spark : unary_op_df(spark, data_gen, length=1024) \
- .selectExpr('*', 'hash(a) as h'), "ProjectExec")
+ .selectExpr('*', 'hash(a) as h'), "ProjectExec",
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test.
@pytest.mark.parametrize('num_parts', [1, 2, 10, 17, 19, 32], ids=idfn)
@@ -279,8 +308,9 @@ def test_hash_fallback(data_gen):
([('a', decimal_gen_64bit), ('b', decimal_gen_64bit), ('c', decimal_gen_64bit)], ['a', 'b', 'c']),
([('a', decimal_gen_128bit), ('b', decimal_gen_128bit), ('c', decimal_gen_128bit)], ['a', 'b', 'c']),
], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_repartition_exact(gen, num_parts):
+def test_hash_repartition_exact(gen, num_parts, kudo_enabled):
data_gen = gen[0]
part_on = gen[1]
assert_gpu_and_cpu_are_equal_collect(
@@ -288,7 +318,8 @@ def test_hash_repartition_exact(gen, num_parts):
.repartition(num_parts, *part_on)\
.withColumn('id', f.spark_partition_id())\
.withColumn('hashed', f.hash(*part_on))\
- .selectExpr('*', 'pmod(hashed, {})'.format(num_parts)))
+ .selectExpr('*', 'pmod(hashed, {})'.format(num_parts)),
+ conf = {kudo_enabled_conf_key: kudo_enabled})
@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test.
@@ -311,9 +342,10 @@ def test_hash_repartition_exact_longs_no_overflow(num_parts, is_ansi_mode):
@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test.
@pytest.mark.parametrize('num_parts', [17], ids=idfn)
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
@allow_non_gpu(*non_utc_allow)
-def test_hash_repartition_long_overflow_ansi_exception(num_parts):
- conf = ansi_enabled_conf
+def test_hash_repartition_long_overflow_ansi_exception(num_parts, kudo_enabled):
+ conf = copy_and_update(ansi_enabled_conf, {kudo_enabled_conf_key: kudo_enabled})
def test_function(spark):
df = gen_df(spark, [('a', long_gen)], length=1024)
@@ -332,11 +364,13 @@ def test_function(spark):
# Test a query that should cause Spark to leverage getShuffleRDD
@ignore_order(local=True)
-def test_union_with_filter():
+@pytest.mark.parametrize("kudo_enabled", ["true", "false"], ids=idfn)
+def test_union_with_filter(kudo_enabled):
def doit(spark):
dfa = spark.range(1, 100).withColumn("id2", f.col("id"))
dfb = dfa.groupBy("id").agg(f.size(f.collect_set("id2")).alias("idc"))
dfc = dfb.filter(f.col("idc") == 1).select("id")
return dfc.union(dfc)
- conf = { "spark.sql.adaptive.enabled": "true" }
+ conf = { "spark.sql.adaptive.enabled": "true",
+ kudo_enabled_conf_key: kudo_enabled}
assert_gpu_and_cpu_are_equal_collect(doit, conf)
diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java
index 30b24fab11d..f2be4264162 100644
--- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java
+++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java
@@ -528,6 +528,55 @@ public static Schema from(StructType input) {
return builder.build();
}
+ /**
+ * Converts a list of Spark data types to a cudf schema.
+ *
+ *
+ * This method correctly handles nested types, but will generate random field names.
+ *
+ * @param dataTypes the list of data types to convert
+ * @return the cudf schema
+ */
+ public static Schema from(DataType[] dataTypes) {
+ Schema.Builder builder = Schema.builder();
+ visit(dataTypes, builder, 0);
+ return builder.build();
+ }
+
+ private static void visit(DataType[] dataTypes, Schema.Builder builder, int level) {
+ for (int idx = 0; idx < dataTypes.length; idx ++) {
+ DataType dt = dataTypes[idx];
+ String name = "_col_" + level + "_" + idx;
+ if (dt instanceof MapType) {
+ // MapType is list of struct in cudf, so need to handle it specially.
+ Schema.Builder listBuilder = builder.addColumn(DType.LIST, name);
+ Schema.Builder structBuilder = listBuilder.addColumn(DType.STRUCT, name + "_map");
+ MapType mt = (MapType) dt;
+ DataType[] structChildren = {mt.keyType(), mt.valueType()};
+ visit(structChildren, structBuilder, level + 1);
+ } else if (dt instanceof BinaryType) {
+ Schema.Builder listBuilder = builder.addColumn(DType.LIST, name);
+ listBuilder.addColumn(DType.UINT8, name + "_bytes");
+ } else {
+ Schema.Builder childBuilder = builder.addColumn(GpuColumnVector.getRapidsType(dt), name);
+ if (dt instanceof ArrayType) {
+ // Array (aka List)
+ DataType[] childType = {((ArrayType) dt).elementType()};
+ visit(childType, childBuilder, level + 1);
+ } else if (dt instanceof StructType) {
+ // Struct
+ StructType st = (StructType) dt;
+ DataType[] childrenTypes = new DataType[st.length()];
+ for (int i = 0; i < childrenTypes.length; i ++) {
+ childrenTypes[i] = st.apply(i).dataType();
+ }
+ visit(childrenTypes, childBuilder, level + 1);
+ }
+ }
+ }
+ }
+
+
/**
* Convert a ColumnarBatch to a table. The table will increment the reference count for all of
* the columns in the batch, so you will need to close both the batch passed in and the table
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
index 049f3f21bcf..116b8b97504 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2019-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2019-2024, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -22,19 +22,31 @@ import java.nio.ByteBuffer
import scala.collection.mutable.ArrayBuffer
import scala.reflect.ClassTag
-import ai.rapids.cudf.{HostColumnVector, HostMemoryBuffer, JCudfSerialization, NvtxColor, NvtxRange}
+import ai.rapids.cudf.{Cuda, HostColumnVector, HostMemoryBuffer, JCudfSerialization, NvtxColor, NvtxRange}
import ai.rapids.cudf.JCudfSerialization.SerializedTableHeader
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion
+import com.nvidia.spark.rapids.jni.kudo.{KudoSerializer, KudoTable}
import org.apache.spark.TaskContext
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer, SerializerInstance}
-import org.apache.spark.sql.types.NullType
+import org.apache.spark.sql.types.{DataType, NullType}
import org.apache.spark.sql.vectorized.ColumnarBatch
+/**
+ * Iterator that reads serialized tables from a stream.
+ */
+trait BaseSerializedTableIterator extends Iterator[(Int, ColumnarBatch)] {
+ /**
+ * Attempt to read the next batch size from the stream.
+ * @return the length of the data to read, or None if the stream is closed or ended
+ */
+ def peekNextBatchSize(): Option[Long]
+}
+
class SerializedBatchIterator(dIn: DataInputStream)
- extends Iterator[(Int, ColumnarBatch)] {
+ extends BaseSerializedTableIterator {
private[this] var nextHeader: Option[SerializedTableHeader] = None
private[this] var toBeReturned: Option[ColumnarBatch] = None
private[this] var streamClosed: Boolean = false
@@ -48,8 +60,8 @@ class SerializedBatchIterator(dIn: DataInputStream)
}
}
- def tryReadNextHeader(): Option[Long] = {
- if (streamClosed){
+ override def peekNextBatchSize(): Option[Long] = {
+ if (streamClosed) {
None
} else {
if (nextHeader.isEmpty) {
@@ -68,7 +80,7 @@ class SerializedBatchIterator(dIn: DataInputStream)
}
}
- def tryReadNext(): Option[ColumnarBatch] = {
+ private def tryReadNext(): Option[ColumnarBatch] = {
if (nextHeader.isEmpty) {
None
} else {
@@ -90,13 +102,13 @@ class SerializedBatchIterator(dIn: DataInputStream)
}
override def hasNext: Boolean = {
- tryReadNextHeader()
+ peekNextBatchSize()
nextHeader.isDefined
}
override def next(): (Int, ColumnarBatch) = {
if (toBeReturned.isEmpty) {
- tryReadNextHeader()
+ peekNextBatchSize()
toBeReturned = tryReadNext()
if (nextHeader.isEmpty || toBeReturned.isEmpty) {
throw new NoSuchElementException("Walked off of the end...")
@@ -108,6 +120,7 @@ class SerializedBatchIterator(dIn: DataInputStream)
(0, ret)
}
}
+
/**
* Serializer for serializing `ColumnarBatch`s for use during normal shuffle.
*
@@ -124,10 +137,16 @@ class SerializedBatchIterator(dIn: DataInputStream)
*
* @note The RAPIDS shuffle does not use this code.
*/
-class GpuColumnarBatchSerializer(dataSize: GpuMetric)
- extends Serializer with Serializable {
- override def newInstance(): SerializerInstance =
- new GpuColumnarBatchSerializerInstance(dataSize)
+class GpuColumnarBatchSerializer(dataSize: GpuMetric, dataTypes: Array[DataType], useKudo: Boolean)
+ extends Serializer with Serializable {
+ override def newInstance(): SerializerInstance = {
+ if (useKudo) {
+ new KudoSerializerInstance(dataSize, dataTypes)
+ } else {
+ new GpuColumnarBatchSerializerInstance(dataSize)
+ }
+ }
+
override def supportsRelocationOfSerializedObjects: Boolean = true
}
@@ -252,8 +271,10 @@ private class GpuColumnarBatchSerializerInstance(dataSize: GpuMetric) extends Se
// These methods are never called by shuffle code.
override def serialize[T: ClassTag](t: T): ByteBuffer = throw new UnsupportedOperationException
+
override def deserialize[T: ClassTag](bytes: ByteBuffer): T =
throw new UnsupportedOperationException
+
override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T =
throw new UnsupportedOperationException
}
@@ -282,7 +303,7 @@ object SerializedTableColumn {
* Build a `ColumnarBatch` consisting of a single [[SerializedTableColumn]] describing
* the specified serialized table.
*
- * @param header header for the serialized table
+ * @param header header for the serialized table
* @param hostBuffer host buffer containing the table data
* @return columnar batch to be passed to [[GpuShuffleCoalesceExec]]
*/
@@ -299,7 +320,7 @@ object SerializedTableColumn {
val cv = batch.column(0)
cv match {
case serializedTableColumn: SerializedTableColumn
- if serializedTableColumn.hostBuffer != null =>
+ if serializedTableColumn.hostBuffer != null =>
sum += serializedTableColumn.hostBuffer.getLength
case _ =>
}
@@ -307,3 +328,226 @@ object SerializedTableColumn {
sum
}
}
+
+/**
+ * Serializer instance for serializing `ColumnarBatch`s for use during shuffle with
+ * [[KudoSerializer]]
+ *
+ * @param dataSize metric to track the size of the serialized data
+ * @param dataTypes data types of the columns in the batch
+ */
+private class KudoSerializerInstance(
+ val dataSize: GpuMetric,
+ val dataTypes: Array[DataType]) extends SerializerInstance {
+
+ private lazy val kudo = new KudoSerializer(GpuColumnVector.from(dataTypes))
+
+ override def serializeStream(out: OutputStream): SerializationStream = new SerializationStream {
+ private[this] val dOut: DataOutputStream =
+ new DataOutputStream(new BufferedOutputStream(out))
+
+ override def writeValue[T: ClassTag](value: T): SerializationStream = {
+ val batch = value.asInstanceOf[ColumnarBatch]
+ val numColumns = batch.numCols()
+ val columns: Array[HostColumnVector] = new Array(numColumns)
+ withResource(new ArrayBuffer[AutoCloseable]()) { toClose =>
+ var startRow = 0
+ val numRows = batch.numRows()
+ if (batch.numCols() > 0) {
+ val firstCol = batch.column(0)
+ if (firstCol.isInstanceOf[SlicedGpuColumnVector]) {
+ // We don't have control over ColumnarBatch to put in the slice, so we have to do it
+ // for each column. In this case we are using the first column.
+ startRow = firstCol.asInstanceOf[SlicedGpuColumnVector].getStart
+ for (i <- 0 until numColumns) {
+ columns(i) = batch.column(i).asInstanceOf[SlicedGpuColumnVector].getBase
+ }
+ } else {
+ for (i <- 0 until numColumns) {
+ batch.column(i) match {
+ case gpu: GpuColumnVector =>
+ val cpu = gpu.copyToHostAsync(Cuda.DEFAULT_STREAM)
+ toClose += cpu
+ columns(i) = cpu.getBase
+ case cpu: RapidsHostColumnVector =>
+ columns(i) = cpu.getBase
+ }
+ }
+
+ Cuda.DEFAULT_STREAM.sync()
+ }
+
+ withResource(new NvtxRange("Serialize Batch", NvtxColor.YELLOW)) { _ =>
+ dataSize += kudo.writeToStream(columns, dOut, startRow, numRows)
+ }
+ } else {
+ withResource(new NvtxRange("Serialize Row Only Batch", NvtxColor.YELLOW)) { _ =>
+ dataSize += KudoSerializer.writeRowCountToStream(dOut, numRows)
+ }
+ }
+ this
+ }
+ }
+
+ override def writeKey[T: ClassTag](key: T): SerializationStream = {
+ // The key is only needed on the map side when computing partition ids. It does not need to
+ // be shuffled.
+ assert(null == key || key.isInstanceOf[Int])
+ this
+ }
+
+ override def writeAll[T: ClassTag](iter: Iterator[T]): SerializationStream = {
+ // This method is never called by shuffle code.
+ throw new UnsupportedOperationException
+ }
+
+ override def writeObject[T: ClassTag](t: T): SerializationStream = {
+ // This method is never called by shuffle code.
+ throw new UnsupportedOperationException
+ }
+
+ override def flush(): Unit = {
+ dOut.flush()
+ }
+
+ override def close(): Unit = {
+ dOut.close()
+ }
+ }
+
+ override def deserializeStream(in: InputStream): DeserializationStream = {
+ new DeserializationStream {
+ private[this] val dIn: DataInputStream = new DataInputStream(new BufferedInputStream(in))
+
+ override def asKeyValueIterator: Iterator[(Int, ColumnarBatch)] = {
+ new KudoSerializedBatchIterator(dIn)
+ }
+
+ override def asIterator: Iterator[Any] = {
+ // This method is never called by shuffle code.
+ throw new UnsupportedOperationException
+ }
+
+ override def readKey[T]()(implicit classType: ClassTag[T]): T = {
+ // We skipped serialization of the key in writeKey(), so just return a dummy value since
+ // this is going to be discarded anyways.
+ null.asInstanceOf[T]
+ }
+
+ override def readValue[T]()(implicit classType: ClassTag[T]): T = {
+ // This method should never be called by shuffle code.
+ throw new UnsupportedOperationException
+ }
+
+ override def readObject[T]()(implicit classType: ClassTag[T]): T = {
+ // This method is never called by shuffle code.
+ throw new UnsupportedOperationException
+ }
+
+ override def close(): Unit = {
+ dIn.close()
+ }
+ }
+ }
+
+ // These methods are never called by shuffle code.
+ override def serialize[T: ClassTag](t: T): ByteBuffer = throw new UnsupportedOperationException
+
+ override def deserialize[T: ClassTag](bytes: ByteBuffer): T =
+ throw new UnsupportedOperationException
+
+ override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T =
+ throw new UnsupportedOperationException
+}
+
+/**
+ * A special `ColumnVector` that describes a serialized table read from shuffle using
+ * [[KudoSerializer]].
+ *
+ * This appears in a `ColumnarBatch` to pass serialized tables to [[GpuShuffleCoalesceExec]]
+ * which should always appear in the query plan immediately after a shuffle.
+ */
+case class KudoSerializedTableColumn(kudoTable: KudoTable) extends GpuColumnVectorBase(NullType) {
+ override def close(): Unit = {
+ if (kudoTable != null) {
+ kudoTable.close()
+ }
+ }
+
+ override def hasNull: Boolean = throw new IllegalStateException("should not be called")
+
+ override def numNulls(): Int = throw new IllegalStateException("should not be called")
+}
+
+object KudoSerializedTableColumn {
+ /**
+ * Build a `ColumnarBatch` consisting of a single [[KudoSerializedTableColumn]] describing
+ * the specified serialized table.
+ *
+ * @param kudoTable Serialized kudo table.
+ * @return columnar batch to be passed to [[GpuShuffleCoalesceExec]]
+ */
+ def from(kudoTable: KudoTable): ColumnarBatch = {
+ val column = new KudoSerializedTableColumn(kudoTable)
+ new ColumnarBatch(Array(column), kudoTable.getHeader.getNumRows)
+ }
+}
+
+class KudoSerializedBatchIterator(dIn: DataInputStream)
+ extends BaseSerializedTableIterator {
+ private[this] var nextTable: Option[KudoTable] = None
+ private[this] var streamClosed: Boolean = false
+
+ // Don't install the callback if in a unit test
+ Option(TaskContext.get()).foreach { tc =>
+ onTaskCompletion(tc) {
+ nextTable.foreach(_.close())
+ nextTable = None
+ dIn.close()
+ }
+ }
+
+ private def tryReadNext(): Unit = {
+ if (!streamClosed) {
+ withResource(new NvtxRange("Read Kudo Table", NvtxColor.YELLOW)) { _ =>
+ val kudoTable = KudoTable.from(dIn)
+ if (kudoTable.isPresent) {
+ nextTable = Some(kudoTable.get())
+ } else {
+ dIn.close()
+ streamClosed = true
+ nextTable = None
+ }
+ }
+ }
+ }
+
+ override def hasNext: Boolean = {
+ nextTable match {
+ case Some(_) => true
+ case None =>
+ tryReadNext()
+ nextTable.isDefined
+ }
+ }
+
+ override def next(): (Int, ColumnarBatch) = {
+ if (hasNext) {
+ val ret = KudoSerializedTableColumn.from(nextTable.get)
+ nextTable = None
+ (0, ret)
+ } else {
+ throw new NoSuchElementException("Walked off of the end...")
+ }
+ }
+
+ /**
+ * Attempt to read the next header from the stream.
+ *
+ * @return the length of the data to read, or None if the stream is closed or ended
+ */
+ override def peekNextBatchSize(): Option[Long] = {
+ tryReadNext()
+ nextTable.flatMap(t => Option(t.getBuffer)).map(_.getLength)
+ }
+}
\ No newline at end of file
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala
index 17f2c35a8eb..0ffead09de6 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala
@@ -87,6 +87,8 @@ object GpuMetric extends Logging {
val FILECACHE_DATA_RANGE_READ_TIME = "filecacheDataRangeReadTime"
val DELETION_VECTOR_SCATTER_TIME = "deletionVectorScatterTime"
val DELETION_VECTOR_SIZE = "deletionVectorSize"
+ val CONCAT_HEADER_TIME = "concatHeaderTime"
+ val CONCAT_BUFFER_TIME = "concatBufferTime"
// Metric Descriptions.
val DESCRIPTION_BUFFER_TIME = "buffer time"
@@ -123,6 +125,8 @@ object GpuMetric extends Logging {
val DESCRIPTION_FILECACHE_DATA_RANGE_READ_TIME = "cached data read time"
val DESCRIPTION_DELETION_VECTOR_SCATTER_TIME = "deletion vector scatter time"
val DESCRIPTION_DELETION_VECTOR_SIZE = "deletion vector size"
+ val DESCRIPTION_CONCAT_HEADER_TIME = "concat header time"
+ val DESCRIPTION_CONCAT_BUFFER_TIME = "concat buffer time"
def unwrap(input: GpuMetric): SQLMetric = input match {
case w :WrappedGpuMetric => w.sqlMetric
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleCoalesceExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleCoalesceExec.scala
index d57f6430a0f..c33c19cdd8a 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleCoalesceExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleCoalesceExec.scala
@@ -23,7 +23,9 @@ import scala.reflect.ClassTag
import ai.rapids.cudf.{JCudfSerialization, NvtxColor, NvtxRange}
import ai.rapids.cudf.JCudfSerialization.HostConcatResult
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
+import com.nvidia.spark.rapids.GpuMetric.{CONCAT_BUFFER_TIME, CONCAT_HEADER_TIME}
import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion
+import com.nvidia.spark.rapids.jni.kudo.{KudoHostMergeResult, KudoSerializer, KudoTable}
import com.nvidia.spark.rapids.shims.ShimUnaryExecNode
import org.apache.spark.TaskContext
@@ -32,6 +34,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -39,11 +42,12 @@ import org.apache.spark.sql.vectorized.ColumnarBatch
* Coalesces serialized tables on the host up to the target batch size before transferring
* the coalesced result to the GPU. This reduces the overhead of copying data to the GPU
* and also helps avoid holding onto the GPU semaphore while shuffle I/O is being performed.
+ *
* @note This should ALWAYS appear in the plan after a GPU shuffle when RAPIDS shuffle is
* not being used.
*/
case class GpuShuffleCoalesceExec(child: SparkPlan, targetBatchByteSize: Long)
- extends ShimUnaryExecNode with GpuExec {
+ extends ShimUnaryExecNode with GpuExec {
import GpuMetric._
@@ -51,7 +55,9 @@ case class GpuShuffleCoalesceExec(child: SparkPlan, targetBatchByteSize: Long)
OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME),
NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS),
NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES),
- CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME)
+ CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME),
+ CONCAT_HEADER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_HEADER_TIME),
+ CONCAT_BUFFER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_BUFFER_TIME),
)
override protected val outputBatchesLevel = MODERATE_LEVEL
@@ -68,7 +74,7 @@ case class GpuShuffleCoalesceExec(child: SparkPlan, targetBatchByteSize: Long)
val metricsMap = allMetrics
val targetSize = targetBatchByteSize
val dataTypes = GpuColumnVector.extractTypes(schema)
- val readOption = CoalesceReadOption(new RapidsConf(conf))
+ val readOption = CoalesceReadOption(conf)
child.executeColumnar().mapPartitions { iter =>
GpuShuffleCoalesceUtils.getGpuShuffleCoalesceIterator(iter, targetSize, dataTypes,
@@ -81,9 +87,12 @@ case class GpuShuffleCoalesceExec(child: SparkPlan, targetBatchByteSize: Long)
case class CoalesceReadOption private(kudoEnabled: Boolean)
object CoalesceReadOption {
+ def apply(conf: SQLConf): CoalesceReadOption = {
+ CoalesceReadOption(RapidsConf.SHUFFLE_KUDO_SERIALIZER_ENABLED.get(conf))
+ }
+
def apply(conf: RapidsConf): CoalesceReadOption = {
- // TODO get the value from conf
- CoalesceReadOption(false)
+ CoalesceReadOption(conf.shuffleKudoSerializerEnabled)
}
}
@@ -95,11 +104,11 @@ object GpuShuffleCoalesceUtils {
* The input iterator is expected to contain only serialized host batches just
* returned from the Shuffle deserializer. Otherwise, it will blow up.
*
- * @param iter the input iterator containing only serialized host batches
- * @param targetSize the target batch size for coalescing
- * @param dataTypes the schema of the input batches
- * @param readOption the coalesce read option
- * @param metricsMap metrics map
+ * @param iter the input iterator containing only serialized host batches
+ * @param targetSize the target batch size for coalescing
+ * @param dataTypes the schema of the input batches
+ * @param readOption the coalesce read option
+ * @param metricsMap metrics map
* @param prefetchFirstBatch whether prefetching the first bundle of serialized
* batches with the total size up to the "targetSize". The
* prefetched batches will be cached on host until the "next()"
@@ -113,8 +122,7 @@ object GpuShuffleCoalesceUtils {
metricsMap: Map[String, GpuMetric],
prefetchFirstBatch: Boolean = false): Iterator[ColumnarBatch] = {
val hostIter = if (readOption.kudoEnabled) {
- // TODO replace with the actual Kudo host iterator
- throw new UnsupportedOperationException("Kudo is not supported yet")
+ new KudoHostShuffleCoalesceIterator(iter, targetSize, metricsMap, dataTypes)
} else {
new HostShuffleCoalesceIterator(iter, targetSize, metricsMap)
}
@@ -135,7 +143,7 @@ object GpuShuffleCoalesceUtils {
def getSerializedBufferSize(cb: ColumnarBatch): Long = {
assert(cb.numCols() == 1)
val hmb = cb.column(0) match {
- // TODO add the Kudo case
+ case col: KudoSerializedTableColumn => col.kudoTable.getBuffer
case serCol: SerializedTableColumn => serCol.hostBuffer
case o => throw new IllegalStateException(s"unsupported type: ${o.getClass}")
}
@@ -161,7 +169,9 @@ sealed trait CoalescedHostResult extends AutoCloseable {
*/
sealed trait SerializedTableOperator[T <: AutoCloseable] {
def getDataLen(table: T): Long
+
def getNumRows(table: T): Int
+
def concatOnHost(tables: Array[T]): CoalescedHostResult
}
@@ -178,6 +188,7 @@ class JCudfCoalescedHostResult(hostConcatResult: HostConcatResult) extends Coale
class JCudfTableOperator extends SerializedTableOperator[SerializedTableColumn] {
override def getDataLen(table: SerializedTableColumn): Long = table.header.getDataLen
+
override def getNumRows(table: SerializedTableColumn): Int = table.header.getNumRows
override def concatOnHost(tables: Array[SerializedTableColumn]): CoalescedHostResult = {
@@ -194,13 +205,73 @@ class JCudfTableOperator extends SerializedTableOperator[SerializedTableColumn]
}
}
+case class KudoHostMergeResultWrapper(inner: KudoHostMergeResult) extends CoalescedHostResult {
+
+ /** Convert itself to a GPU batch */
+ override def toGpuBatch(dataTypes: Array[DataType]): ColumnarBatch = {
+ RmmRapidsRetryIterator.withRetryNoSplit {
+ withResource(inner.toTable) { cudfTable =>
+ GpuColumnVector.from(cudfTable, dataTypes)
+ }
+ }
+ }
+
+ /** Get the data size */
+ override def getDataSize: Long = inner.getDataLength
+
+ override def close(): Unit = inner.close()
+}
+
+case class RowCountOnlyMergeResult(rowCount: Int) extends CoalescedHostResult {
+ override def toGpuBatch(dataTypes: Array[DataType]): ColumnarBatch = {
+ new ColumnarBatch(Array.empty, rowCount)
+ }
+
+ override def getDataSize: Long = 0
+
+ override def close(): Unit = {}
+}
+
+class KudoTableOperator(
+ kudo: Option[KudoSerializer] ,
+ kudoMergeHeaderTime: GpuMetric,
+ kudoMergeBufferTime: GpuMetric) extends SerializedTableOperator[KudoSerializedTableColumn] {
+ require(kudo != null, "kudo serializer should not be null")
+
+ override def getDataLen(column: KudoSerializedTableColumn): Long = column.kudoTable.getHeader
+ .getTotalDataLen
+
+ override def getNumRows(column: KudoSerializedTableColumn): Int = column.kudoTable.getHeader
+ .getNumRows
+
+ override def concatOnHost(columns: Array[KudoSerializedTableColumn]): CoalescedHostResult = {
+ require(columns.nonEmpty, "no tables to be concatenated")
+ val numCols = columns.head.kudoTable.getHeader.getNumColumns
+ if (numCols == 0) {
+ val totalRowsNum = columns.map(getNumRows).sum
+ RowCountOnlyMergeResult(totalRowsNum)
+ } else {
+ val kudoTables = new util.ArrayList[KudoTable](columns.length)
+ columns.foreach { column =>
+ kudoTables.add(column.kudoTable)
+ }
+
+ val result = kudo.get.mergeOnHost(kudoTables)
+ kudoMergeHeaderTime += result.getRight.getCalcHeaderTime
+ kudoMergeBufferTime += result.getRight.getMergeIntoHostBufferTime
+
+ KudoHostMergeResultWrapper(result.getLeft)
+ }
+ }
+}
+
/**
* Iterator that coalesces columnar batches that are expected to only contain
* serialized tables from shuffle. The serialized tables within are collected up
* to the target batch size and then concatenated on the host before handing
* them to the caller on `.next()`
*/
-abstract class HostCoalesceIteratorBase[T <: AutoCloseable: ClassTag](
+abstract class HostCoalesceIteratorBase[T <: AutoCloseable : ClassTag](
iter: Iterator[ColumnarBatch],
targetBatchByteSize: Long,
metricsMap: Map[String, GpuMetric])
@@ -305,6 +376,23 @@ class HostShuffleCoalesceIterator(
override protected def tableOperator = new JCudfTableOperator
}
+class KudoHostShuffleCoalesceIterator(
+ iter: Iterator[ColumnarBatch],
+ targetBatchSize: Long,
+ metricsMap: Map[String, GpuMetric],
+ dataTypes: Array[DataType])
+ extends HostCoalesceIteratorBase[KudoSerializedTableColumn](iter, targetBatchSize, metricsMap) {
+ override protected def tableOperator = {
+ val kudoSer = if (dataTypes.nonEmpty) {
+ Some(new KudoSerializer(GpuColumnVector.from(dataTypes)))
+ } else {
+ None
+ }
+ new KudoTableOperator(kudoSer, metricsMap(CONCAT_HEADER_TIME), metricsMap(CONCAT_BUFFER_TIME))
+ }
+}
+
+
/**
* Iterator that expects only "CoalescedHostResult"s as the input, and transfers
* them to GPU.
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala
index b9525c73966..a95ff45a973 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala
@@ -144,6 +144,8 @@ case class GpuShuffledHashJoinExec(
override lazy val additionalMetrics: Map[String, GpuMetric] = Map(
OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME),
CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME),
+ CONCAT_HEADER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_HEADER_TIME),
+ CONCAT_BUFFER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_BUFFER_TIME),
BUILD_DATA_SIZE -> createSizeMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_DATA_SIZE),
BUILD_TIME -> createNanoTimingMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_TIME),
STREAM_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_STREAM_TIME),
@@ -284,13 +286,13 @@ object GpuShuffledHashJoinExec extends Logging {
coalesceMetrics: Map[String, GpuMetric]):
(Either[ColumnarBatch, Iterator[ColumnarBatch]], Iterator[ColumnarBatch]) = {
val buildTime = coalesceMetrics(GpuMetric.BUILD_TIME)
- val buildTypes = buildOutput.map(_.dataType).toArray
+ val buildDataType = buildOutput.map(_.dataType).toArray
closeOnExcept(new CloseableBufferedIterator(buildIter)) { bufBuildIter =>
val startTime = System.nanoTime()
var isBuildSerialized = false
// Batches type detection
val coalesceBuiltIter = getHostShuffleCoalesceIterator(
- bufBuildIter, targetSize, coalesceMetrics).map { iter =>
+ bufBuildIter, buildDataType, targetSize, coalesceMetrics).map { iter =>
isBuildSerialized = true
iter
}.getOrElse(bufBuildIter)
@@ -308,7 +310,7 @@ object GpuShuffledHashJoinExec extends Logging {
// It can be optimized for grabbing the GPU semaphore when there is only a single
// serialized host batch and the sub-partitioning is not activated.
val (singleBuildCb, bufferedStreamIter) = getBuildBatchOptimizedAndClose(
- firstBuildBatch.asInstanceOf[CoalescedHostResult], streamIter, buildTypes,
+ firstBuildBatch.asInstanceOf[CoalescedHostResult], streamIter, buildDataType,
buildGoal, buildTime)
logDebug("In the optimized case for grabbing the GPU semaphore, return " +
s"a single batch (size: ${getBatchSize(singleBuildCb)}) for the build side " +
@@ -321,7 +323,7 @@ object GpuShuffledHashJoinExec extends Logging {
val gpuBuildIter = if (isBuildSerialized) {
// batches on host, move them to GPU
new GpuShuffleCoalesceIterator(safeIter.asInstanceOf[Iterator[CoalescedHostResult]],
- buildTypes, coalesceMetrics)
+ buildDataType, coalesceMetrics)
} else { // batches already on GPU
safeIter.asInstanceOf[Iterator[ColumnarBatch]]
}
@@ -347,7 +349,7 @@ object GpuShuffledHashJoinExec extends Logging {
}
} else {
// build is empty
- (Left(GpuColumnVector.emptyBatchFromTypes(buildTypes)), streamIter)
+ (Left(GpuColumnVector.emptyBatchFromTypes(buildDataType)), streamIter)
}
}
}
@@ -463,12 +465,15 @@ object GpuShuffledHashJoinExec extends Logging {
private def getHostShuffleCoalesceIterator(
iter: BufferedIterator[ColumnarBatch],
+ dataTypes: Array[DataType],
targetSize: Long,
coalesceMetrics: Map[String, GpuMetric]): Option[Iterator[CoalescedHostResult]] = {
var retIter: Option[Iterator[CoalescedHostResult]] = None
if (iter.hasNext && iter.head.numCols() == 1) {
iter.head.column(0) match {
- // TODO add the Kudo case
+ case _: KudoSerializedTableColumn =>
+ retIter = Some(new KudoHostShuffleCoalesceIterator(iter, targetSize, coalesceMetrics,
+ dataTypes))
case _: SerializedTableColumn =>
retIter = Some(new HostShuffleCoalesceIterator(iter, targetSize, coalesceMetrics))
case _ => // should be gpu batches
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledSizedHashJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledSizedHashJoinExec.scala
index 252c31da125..177710fea81 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledSizedHashJoinExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledSizedHashJoinExec.scala
@@ -26,6 +26,7 @@ import com.nvidia.spark.rapids.GpuShuffledSizedHashJoinExec.JoinInfo
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.RmmRapidsRetryIterator.withRetryNoSplit
import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion
+import com.nvidia.spark.rapids.jni.kudo.{KudoTable, KudoTableHeader}
import com.nvidia.spark.rapids.shims.GpuHashPartitioning
import org.apache.spark.rdd.RDD
@@ -319,7 +320,10 @@ object GpuShuffledSizedHashJoinExec {
// Use a filtered metrics map to avoid output batch counts and other unrelated metric updates
Map(
OP_TIME -> metrics(OP_TIME),
- CONCAT_TIME -> metrics(CONCAT_TIME)).withDefaultValue(NoopMetric)
+ CONCAT_TIME -> metrics(CONCAT_TIME),
+ CONCAT_HEADER_TIME -> metrics(CONCAT_HEADER_TIME),
+ CONCAT_BUFFER_TIME -> metrics(CONCAT_BUFFER_TIME)
+ ).withDefaultValue(NoopMetric)
}
def createJoinIterator(
@@ -385,6 +389,8 @@ abstract class GpuShuffledSizedHashJoinExec[HOST_BATCH_TYPE <: AutoCloseable] ex
override lazy val additionalMetrics: Map[String, GpuMetric] = Map(
OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME),
CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME),
+ CONCAT_HEADER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_HEADER_TIME),
+ CONCAT_BUFFER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_BUFFER_TIME),
BUILD_DATA_SIZE -> createSizeMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_DATA_SIZE),
BUILD_TIME -> createNanoTimingMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_TIME),
STREAM_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_STREAM_TIME),
@@ -1120,11 +1126,34 @@ class CudfSpillableHostConcatResult(
override def getDataLen: Long = header.getDataLen
}
+class KudoSpillableHostConcatResult(kudoTableHeader: KudoTableHeader,
+ val hmb: HostMemoryBuffer
+) extends SpillableHostConcatResult {
+ require(kudoTableHeader != null, "KudoTableHeader cannot be null")
+ require(hmb != null, "HostMemoryBuffer cannot be null")
+
+ override def toBatch: ColumnarBatch = closeOnExcept(buffer.getHostBuffer()) { hostBuf =>
+ KudoSerializedTableColumn.from(new KudoTable(kudoTableHeader, hostBuf))
+ }
+
+ override def getNumRows: Long = kudoTableHeader.getNumRows
+
+ override def getDataLen: Long = hmb.getLength
+}
+
object SpillableHostConcatResult {
def from(batch: ColumnarBatch): SpillableHostConcatResult = {
- require(batch.numCols() > 0, "Batch must have at least 1 column")
+ require(batch.numCols() == 1, "Batch must have exactly 1 column")
batch.column(0) match {
- // TODO add the Kudo case
+ case col: KudoSerializedTableColumn => {
+ // This will be closed
+ val oldKudoTable = col.kudoTable
+ val buffer = col.kudoTable.getBuffer
+ if (buffer != null) {
+ buffer.incRefCount()
+ }
+ new KudoSpillableHostConcatResult(oldKudoTable.getHeader, buffer)
+ }
case col: SerializedTableColumn =>
val buffer = col.hostBuffer
buffer.incRefCount()
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
index 49aa5f3aff4..398e5c9fc92 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
@@ -1938,6 +1938,13 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression.
.integerConf
.createWithDefault(20)
+ val SHUFFLE_KUDO_SERIALIZER_ENABLED = conf("spark.rapids.shuffle.kudo.serializer.enabled")
+ .doc("Enable or disable the Kudo serializer for the shuffle.")
+ .internal()
+ .startupOnly()
+ .booleanConf
+ .createWithDefault(false)
+
// ALLUXIO CONFIGS
val ALLUXIO_MASTER = conf("spark.rapids.alluxio.master")
.doc("The Alluxio master hostname. If not set, read Alluxio master URL from " +
@@ -3075,6 +3082,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging {
lazy val shuffleMultiThreadedReaderThreads: Int = get(SHUFFLE_MULTITHREADED_READER_THREADS)
+ lazy val shuffleKudoSerializerEnabled: Boolean = get(SHUFFLE_KUDO_SERIALIZER_ENABLED)
+
def isUCXShuffleManagerMode: Boolean =
RapidsShuffleManagerMode
.withName(get(SHUFFLE_MANAGER_MODE)) == RapidsShuffleManagerMode.UCX
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala
index afc05128ba9..a44580c3bf5 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala
@@ -764,14 +764,14 @@ abstract class RapidsShuffleThreadedReaderBase[K, C](
case class BlockState(
blockId: BlockId,
- batchIter: SerializedBatchIterator,
+ batchIter: BaseSerializedTableIterator,
origStream: AutoCloseable)
extends Iterator[(Any, Any)] with AutoCloseable {
private var nextBatchSize = {
var success = false
try {
- val res = batchIter.tryReadNextHeader().getOrElse(0L)
+ val res = batchIter.peekNextBatchSize().getOrElse(0L)
success = true
res
} finally {
@@ -791,7 +791,7 @@ abstract class RapidsShuffleThreadedReaderBase[K, C](
val nextBatch = batchIter.next()
var success = false
try {
- nextBatchSize = batchIter.tryReadNextHeader().getOrElse(0L)
+ nextBatchSize = batchIter.peekNextBatchSize().getOrElse(0L)
success = true
nextBatch
} finally {
@@ -942,7 +942,8 @@ abstract class RapidsShuffleThreadedReaderBase[K, C](
readBlockedTime += System.nanoTime() - readBlockedStart
val deserStream = serializerInstance.deserializeStream(inputStream)
- val batchIter = deserStream.asKeyValueIterator.asInstanceOf[SerializedBatchIterator]
+ val batchIter = deserStream.asKeyValueIterator
+ .asInstanceOf[BaseSerializedTableIterator]
val blockState = BlockState(blockId, batchIter, inputStream)
// get the next known batch size (there could be multiple batches)
if (limiter.acquire(blockState.getNextBatchSize)) {
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala
index 5323fc89019..f17cfbac13f 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2019-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2019-2024, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -169,6 +169,8 @@ abstract class GpuShuffleExchangeExecBase(
child: SparkPlan) extends Exchange with ShimUnaryExecNode with GpuExec {
import GpuMetric._
+ private lazy val useKudo = RapidsConf.SHUFFLE_KUDO_SERIALIZER_ENABLED.get(child.conf)
+
private lazy val useGPUShuffle = {
gpuOutputPartitioning match {
case gpuPartitioning: GpuPartitioning => gpuPartitioning.usesGPUShuffle
@@ -231,7 +233,7 @@ abstract class GpuShuffleExchangeExecBase(
// This value must be lazy because the child's output may not have been resolved
// yet in all cases.
private lazy val serializer: Serializer = new GpuColumnarBatchSerializer(
- gpuLongMetric("dataSize"))
+ gpuLongMetric("dataSize"), sparkTypes, useKudo)
@transient lazy val inputBatchRDD: RDD[ColumnarBatch] = child.executeColumnar()
diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala
index 3ae1ed2ced6..5244db1b642 100644
--- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala
+++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala
@@ -93,7 +93,9 @@ case class GpuBroadcastHashJoinExec(
JOIN_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_JOIN_TIME),
NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS),
NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES),
- CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME)
+ CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME),
+ CONCAT_HEADER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_HEADER_TIME),
+ CONCAT_BUFFER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_BUFFER_TIME)
)
override def requiredChildDistribution: Seq[Distribution] = {
diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala
index cd4864f648a..507026c6a34 100644
--- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala
+++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala
@@ -149,7 +149,9 @@ case class GpuBroadcastNestedLoopJoinExec(
JOIN_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_JOIN_TIME),
NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS),
NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES),
- CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME)
+ CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME),
+ CONCAT_HEADER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_HEADER_TIME),
+ CONCAT_BUFFER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_BUFFER_TIME)
)
def isExecutorBroadcast(): Boolean = {
diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala
index 5e89e49472d..487e63d4a12 100644
--- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala
+++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala
@@ -22,12 +22,15 @@
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.rapids.execution
-import com.nvidia.spark.rapids.{ConcatAndConsumeAll, GpuCoalesceIterator, GpuColumnVector, GpuMetric, GpuShuffleCoalesceIterator, HostShuffleCoalesceIterator, NoopMetric, RequireSingleBatch}
+import com.nvidia.spark.rapids.{ConcatAndConsumeAll, GpuCoalesceIterator, GpuColumnVector, GpuMetric, NoopMetric, RequireSingleBatch}
import com.nvidia.spark.rapids.Arm.withResource
+import com.nvidia.spark.rapids.CoalesceReadOption
+import com.nvidia.spark.rapids.GpuShuffleCoalesceUtils
import org.apache.spark.TaskContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -71,14 +74,17 @@ object GpuExecutorBroadcastHelper {
// to ensure this always a single batch for the following step.
val shuffleMetrics = Map(
CONCAT_TIME -> metricsMap(CONCAT_TIME),
- OP_TIME -> metricsMap(OP_TIME)
+ CONCAT_HEADER_TIME -> metricsMap(CONCAT_HEADER_TIME),
+ CONCAT_BUFFER_TIME -> metricsMap(CONCAT_BUFFER_TIME),
+ OP_TIME -> metricsMap(OP_TIME),
).withDefaultValue(NoopMetric)
val iter = shuffleDataIterator(shuffleData)
new GpuCoalesceIterator(
- new GpuShuffleCoalesceIterator(
- new HostShuffleCoalesceIterator(iter, targetSize, shuffleMetrics),
- dataTypes, shuffleMetrics).asInstanceOf[Iterator[ColumnarBatch]],
+ GpuShuffleCoalesceUtils.getGpuShuffleCoalesceIterator(iter, targetSize,
+ dataTypes,
+ CoalesceReadOption(SQLConf.get),
+ shuffleMetrics),
dataTypes,
RequireSingleBatch,
NoopMetric, // numInputRows
diff --git a/tests/src/test/spark321/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedReaderSuite.scala b/tests/src/test/spark321/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedReaderSuite.scala
index e092d1199ea..3958dce6fdb 100644
--- a/tests/src/test/spark321/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedReaderSuite.scala
+++ b/tests/src/test/spark321/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedReaderSuite.scala
@@ -113,7 +113,7 @@ class RapidsShuffleThreadedReaderSuite
val shuffleId = 22
val numMaps = 6
val keyValuePairsPerMap = 10
- val serializer = new GpuColumnarBatchSerializer(NoopMetric)
+ val serializer = new GpuColumnarBatchSerializer(NoopMetric, Array.empty, false)
// Make a mock BlockManager that will return RecordingManagedByteBuffers of data, so that we
// can ensure retain() and release() are properly called.
From 7110cf421efa83e382c6df4566fbc1fbb0775d31 Mon Sep 17 00:00:00 2001
From: Tim Liu
Date: Fri, 22 Nov 2024 10:15:41 +0800
Subject: [PATCH 073/103] Do not package the Databricks 14.3 shim into the dist
jar [skip ci] (#11744)
* Do not package the Databricks 14.3 shim into the dist jar
The 350db143 shim will be packaged into the dist jar in branch-25.02
Signed-off-by: timl
* Add a follow issue
Signed-off-by: timl
---------
Signed-off-by: timl
---
build/get_buildvers.py | 2 +-
pom.xml | 3 ++-
scala2.13/pom.xml | 3 ++-
3 files changed, 5 insertions(+), 3 deletions(-)
diff --git a/build/get_buildvers.py b/build/get_buildvers.py
index 174fd7c3ff8..5fe864670b5 100644
--- a/build/get_buildvers.py
+++ b/build/get_buildvers.py
@@ -34,7 +34,7 @@ def _get_buildvers(buildvers, pom_file, logger=None):
else:
no_snapshots.append(release)
excluded_shims = pom.find(".//pom:dyn.shim.excluded.releases", ns)
- if excluded_shims:
+ if excluded_shims is not None:
for removed_shim in [x.strip() for x in excluded_shims.text.split(",")]:
if removed_shim in snapshots:
snapshots.remove(removed_shim)
diff --git a/pom.xml b/pom.xml
index f8b397ede2c..12828404031 100644
--- a/pom.xml
+++ b/pom.xml
@@ -813,7 +813,8 @@
-
+
+ 350db143
.
diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml
index 6d2371456d1..d52c8658423 100644
--- a/scala2.13/pom.xml
+++ b/scala2.13/pom.xml
@@ -813,7 +813,8 @@
-
+
+ 350db143
.
From 6e6ce33ea220f000d005d21a93c3809028488edc Mon Sep 17 00:00:00 2001
From: "Robert (Bobby) Evans"
Date: Fri, 22 Nov 2024 09:03:38 -0600
Subject: [PATCH 074/103] Add a few more JSON tests for MAP
(#11721)
Signed-off-by: Nghia Truong
Signed-off-by: Robert (Bobby) Evans
Co-authored-by: Nghia Truong
Co-authored-by: Nghia Truong <7416935+ttnghia@users.noreply.github.com>
---
.../src/main/python/json_matrix_test.py | 184 +++++++++++++++++-
1 file changed, 183 insertions(+), 1 deletion(-)
diff --git a/integration_tests/src/main/python/json_matrix_test.py b/integration_tests/src/main/python/json_matrix_test.py
index 872d6802d04..50fbe9745dc 100644
--- a/integration_tests/src/main/python/json_matrix_test.py
+++ b/integration_tests/src/main/python/json_matrix_test.py
@@ -66,6 +66,7 @@ def read_json_as_text(spark, data_path, column_name):
WITH_COMMENTS_FILE = "withComments.json"
WITH_COMMENTS_SCHEMA = StructType([StructField("str", StringType())])
+WITH_COMMENTS_MAP_SCHEMA = MapType(StringType(), StringType())
@allow_non_gpu('FileSourceScanExec')
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@@ -86,6 +87,14 @@ def test_from_json_allow_comments_on(std_input_path):
'JsonToStructs',
conf =_enable_json_to_structs_conf)
+@allow_non_gpu(TEXT_INPUT_EXEC, 'ProjectExec')
+def test_from_json_allow_comments_on_map(std_input_path):
+ schema = WITH_COMMENTS_MAP_SCHEMA
+ assert_gpu_fallback_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_COMMENTS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowComments': "true"})),
+ 'JsonToStructs',
+ conf =_enable_json_to_structs_conf)
+
# Off is the default so it really needs to work
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
def test_scan_json_allow_comments_off(std_input_path, read_func, spark_tmp_table_factory):
@@ -104,6 +113,14 @@ def test_from_json_allow_comments_off(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_COMMENTS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowComments': "false"})),
conf =_enable_json_to_structs_conf)
+# Off is the default so it really needs to work
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_allow_comments_off_map(std_input_path):
+ schema = WITH_COMMENTS_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_COMMENTS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowComments': "false"})),
+ conf =_enable_json_to_structs_conf)
+
# Off is the default so it really needs to work
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_allow_comments_off(std_input_path):
@@ -119,6 +136,7 @@ def test_json_tuple_allow_comments_off(std_input_path):
WITH_SQ_FILE = "withSingleQuotes.json"
WITH_SQ_SCHEMA = StructType([StructField("str", StringType())])
+WITH_SQ_MAP_SCHEMA = MapType(StringType(), StringType())
@allow_non_gpu('FileSourceScanExec')
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@@ -137,6 +155,13 @@ def test_from_json_allow_single_quotes_off(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_SQ_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowSingleQuotes': "false"})),
conf =_enable_json_to_structs_conf)
+@allow_non_gpu('ProjectExec', TEXT_INPUT_EXEC)
+def test_from_json_allow_single_quotes_off_map(std_input_path):
+ schema = WITH_SQ_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_SQ_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowSingleQuotes': "false"})),
+ conf =_enable_json_to_structs_conf)
+
# On is the default so it really needs to work
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
def test_scan_json_allow_single_quotes_on(std_input_path, read_func, spark_tmp_table_factory):
@@ -155,6 +180,14 @@ def test_from_json_allow_single_quotes_on(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_SQ_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowSingleQuotes': "true"})),
conf =_enable_json_to_structs_conf)
+# On is the default so it really needs to work
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_allow_single_quotes_on_map(std_input_path):
+ schema = WITH_SQ_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_SQ_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowSingleQuotes': "true"})),
+ conf =_enable_json_to_structs_conf)
+
# On is the default so it really needs to work
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_allow_single_quotes_on(std_input_path):
@@ -170,6 +203,7 @@ def test_json_tuple_allow_single_quotes_on(std_input_path):
WITH_UNQUOTE_FIELD_NAMES_FILE = "withUnquotedFieldNames.json"
WITH_UNQUOTE_FIELD_NAMES_SCHEMA = StructType([StructField("str", StringType())])
+WITH_UNQUOTE_FIELD_NAMES_MAP_SCHEMA = MapType(StringType(), StringType())
@allow_non_gpu('FileSourceScanExec')
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@@ -190,6 +224,14 @@ def test_from_json_allow_unquoted_field_names_on(std_input_path):
'JsonToStructs',
conf =_enable_json_to_structs_conf)
+@allow_non_gpu('ProjectExec', TEXT_INPUT_EXEC)
+def test_from_json_allow_unquoted_field_names_on_map(std_input_path):
+ schema = WITH_UNQUOTE_FIELD_NAMES_MAP_SCHEMA
+ assert_gpu_fallback_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTE_FIELD_NAMES_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowUnquotedFieldNames': "true"})),
+ 'JsonToStructs',
+ conf =_enable_json_to_structs_conf)
+
# Off is the default so it really needs to work
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
def test_scan_json_allow_unquoted_field_names_off(std_input_path, read_func, spark_tmp_table_factory):
@@ -202,12 +244,20 @@ def test_scan_json_allow_unquoted_field_names_off(std_input_path, read_func, spa
# Off is the default so it really needs to work
@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
-def test_from_json_allow_unquoted_field_names_on(std_input_path):
+def test_from_json_allow_unquoted_field_names_off(std_input_path):
schema = WITH_UNQUOTE_FIELD_NAMES_SCHEMA
assert_gpu_and_cpu_are_equal_collect(
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTE_FIELD_NAMES_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowUnquotedFieldNames': "false"})),
conf =_enable_json_to_structs_conf)
+# Off is the default so it really needs to work
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_allow_unquoted_field_names_off_map(std_input_path):
+ schema = WITH_UNQUOTE_FIELD_NAMES_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTE_FIELD_NAMES_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {'allowUnquotedFieldNames': "false"})),
+ conf =_enable_json_to_structs_conf)
+
# Off is the default so it really needs to work
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_allow_unquoted_field_names_off(std_input_path):
@@ -226,6 +276,7 @@ def test_json_tuple_allow_unquoted_field_names_off(std_input_path):
StructField("int", IntegerType()),
StructField("float", FloatType()),
StructField("decimal", DecimalType(10, 3))])
+WITH_NUMERIC_LEAD_ZEROS_MAP_SCHEMA = MapType(StringType(), StringType())
@approximate_float()
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@@ -245,6 +296,13 @@ def test_from_json_allow_numeric_leading_zeros_on(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NUMERIC_LEAD_ZEROS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNumericLeadingZeros": "true"})),
conf =_enable_json_to_structs_conf)
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_allow_numeric_leading_zeros_on_map(std_input_path):
+ schema = WITH_NUMERIC_LEAD_ZEROS_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NUMERIC_LEAD_ZEROS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNumericLeadingZeros": "true"})),
+ conf =_enable_json_to_structs_conf)
+
# Off is the default so it really needs to work
@approximate_float()
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@@ -266,6 +324,14 @@ def test_from_json_allow_numeric_leading_zeros_off(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NUMERIC_LEAD_ZEROS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNumericLeadingZeros": "false"})),
conf =_enable_json_to_structs_conf)
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_allow_numeric_leading_zeros_off_map(std_input_path):
+ schema = WITH_NUMERIC_LEAD_ZEROS_MAP_SCHEMA
+
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NUMERIC_LEAD_ZEROS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNumericLeadingZeros": "false"})),
+ conf =_enable_json_to_structs_conf)
+
# Off is the default so it really needs to work
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_allow_numeric_leading_zeros_off(std_input_path):
@@ -284,6 +350,7 @@ def test_json_tuple_allow_numeric_leading_zeros_off(std_input_path):
WITH_NONNUMERIC_NUMBERS_SCHEMA = StructType([
StructField("float", FloatType()),
StructField("double", DoubleType())])
+WITH_NONNUMERIC_NUMBERS_MAP_SCHEMA = MapType(StringType(), StringType())
@approximate_float()
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@@ -305,6 +372,14 @@ def test_from_json_allow_nonnumeric_numbers_off(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NONNUMERIC_NUMBERS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNonNumericNumbers": "false"})),
conf =_enable_json_to_structs_conf)
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+@pytest.mark.xfail(condition = is_before_spark_330(), reason = 'https://github.com/NVIDIA/spark-rapids/issues/10493')
+def test_from_json_allow_nonnumeric_numbers_off_map(std_input_path):
+ schema = WITH_NONNUMERIC_NUMBERS_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NONNUMERIC_NUMBERS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNonNumericNumbers": "false"})),
+ conf =_enable_json_to_structs_conf)
+
# On is the default for scan so it really needs to work
@approximate_float()
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@@ -327,6 +402,14 @@ def test_from_json_allow_nonnumeric_numbers_on(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NONNUMERIC_NUMBERS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNonNumericNumbers": "true"})),
conf =_enable_json_to_structs_conf)
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+@pytest.mark.xfail(condition = is_before_spark_330(), reason = 'https://github.com/NVIDIA/spark-rapids/issues/10493')
+def test_from_json_allow_nonnumeric_numbers_on_map(std_input_path):
+ schema = WITH_NONNUMERIC_NUMBERS_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_NONNUMERIC_NUMBERS_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowNonNumericNumbers": "true"})),
+ conf =_enable_json_to_structs_conf)
+
# Off is the default for get_json_object so we want this to work
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_allow_nonnumeric_numbers_off(std_input_path):
@@ -344,6 +427,7 @@ def test_json_tuple_allow_nonnumeric_numbers_off(std_input_path):
WITH_BS_ESC_FILE = "withBackslashEscapingAnyCharacter.json"
WITH_BS_ESC_SCHEMA = StructType([
StructField("str", StringType())])
+WITH_BS_ESC_MAP_SCHEMA = MapType(StringType(), StringType())
# Off is the default for scan so it really needs to work
@pytest.mark.parametrize('read_func', [read_json_df]) # we have done so many tests already that we don't need both read func. They are the same
@@ -363,6 +447,14 @@ def test_from_json_allow_backslash_escape_any_off(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_BS_ESC_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowBackslashEscapingAnyCharacter": "false"})),
conf =_enable_json_to_structs_conf)
+# Off is the default for from_json so it really needs to work
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_allow_backslash_escape_any_off_map(std_input_path):
+ schema = WITH_BS_ESC_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_BS_ESC_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowBackslashEscapingAnyCharacter": "false"})),
+ conf =_enable_json_to_structs_conf)
+
@allow_non_gpu('FileSourceScanExec')
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
def test_scan_json_allow_backslash_escape_any_on(std_input_path, read_func, spark_tmp_table_factory):
@@ -382,6 +474,14 @@ def test_from_json_allow_backslash_escape_any_on(std_input_path):
'JsonToStructs',
conf =_enable_json_to_structs_conf)
+@allow_non_gpu(TEXT_INPUT_EXEC, 'ProjectExec')
+def test_from_json_allow_backslash_escape_any_on_map(std_input_path):
+ schema = WITH_BS_ESC_MAP_SCHEMA
+ assert_gpu_fallback_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_BS_ESC_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowBackslashEscapingAnyCharacter": "true"})),
+ 'JsonToStructs',
+ conf =_enable_json_to_structs_conf)
+
# Off is the default for get_json_object so we want this to work
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_allow_backslash_escape_any_off(std_input_path):
@@ -398,6 +498,7 @@ def test_json_tuple_allow_backslash_escape_any_off(std_input_path):
WITH_UNQUOTED_CONTROL_FILE = "withUnquotedControlChars.json"
WITH_UNQUOTED_CONTROL_SCHEMA = StructType([
StructField("str", StringType())])
+WITH_UNQUOTED_CONTROL_MAP_SCHEMA = MapType(StringType(), StringType())
# Off is the default for scan so it really needs to work
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@@ -417,6 +518,14 @@ def test_from_json_allow_unquoted_control_chars_off(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTED_CONTROL_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowUnquotedControlChars": "false"})),
conf =_enable_json_to_structs_conf)
+# Off is the default for from_json so it really needs to work
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_allow_unquoted_control_chars_off_map(std_input_path):
+ schema = WITH_UNQUOTED_CONTROL_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTED_CONTROL_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowUnquotedControlChars": "false"})),
+ conf =_enable_json_to_structs_conf)
+
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
def test_scan_json_allow_unquoted_control_chars_on(std_input_path, read_func, spark_tmp_table_factory):
assert_gpu_and_cpu_are_equal_collect(
@@ -433,6 +542,13 @@ def test_from_json_allow_unquoted_control_chars_on(std_input_path):
lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTED_CONTROL_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowUnquotedControlChars": "true"})),
conf =_enable_json_to_structs_conf)
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_allow_unquoted_control_chars_on_map(std_input_path):
+ schema = WITH_UNQUOTED_CONTROL_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_UNQUOTED_CONTROL_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"allowUnquotedControlChars": "true"})),
+ conf =_enable_json_to_structs_conf)
+
# On is the default for get_json_object so we want this to work
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_allow_unquoted_control_chars_on(std_input_path):
@@ -451,6 +567,7 @@ def test_json_tuple_allow_unquoted_control_chars_on(std_input_path):
WITH_DEC_LOCALE_NON_ARIBIC_FILE = "decimal_locale_formatted_strings_non_aribic.json"
WITH_DEC_LOCALE_SCHEMA = StructType([
StructField("data", DecimalType(10, 5))])
+WITH_DEC_LOCALE_MAP_SCHEMA = MapType(StringType(), StringType())
NON_US_DEC_LOCALES=["it-CH","ko-KR","h-TH-x-lvariant-TH","ru-RU","de-DE","iw-IL","hi-IN","ar-QA","zh-CN","ko-KR"]
# US is the default locale so we kind of what it to work
@@ -491,6 +608,23 @@ def test_from_json_dec_locale(std_input_path, locale):
'JsonToStructs',
conf =_enable_json_to_structs_conf)
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_dec_locale_US_map(std_input_path):
+ schema = WITH_DEC_LOCALE_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema)),
+ conf =_enable_json_to_structs_conf)
+
+# This will not fall back because we only support map
+# and locals impact decimal parsing, not strings.
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+@pytest.mark.parametrize('locale', NON_US_DEC_LOCALES)
+def test_from_json_dec_locale_map(std_input_path, locale):
+ schema = WITH_DEC_LOCALE_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"locale": locale})),
+ conf =_enable_json_to_structs_conf)
+
#There is no way to set a locale for these, and it really should not matter
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_dec_locale(std_input_path):
@@ -549,6 +683,25 @@ def test_from_json_dec_locale_non_aribic(std_input_path, locale):
'JsonToStructs',
conf =_enable_json_to_structs_conf)
+# This will not fail because we only support map
+# and decimal is needed to trigger the translation issue
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_dec_locale_US_non_aribic_map(std_input_path):
+ schema = WITH_DEC_LOCALE_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_NON_ARIBIC_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema)),
+ conf =_enable_json_to_structs_conf)
+
+# This will not fall back because we only support map
+# and locals impact decimal parsing, not strings.
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+@pytest.mark.parametrize('locale', NON_US_DEC_LOCALES)
+def test_from_json_dec_locale_non_aribic_map(std_input_path, locale):
+ schema = WITH_DEC_LOCALE_MAP_SCHEMA
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + WITH_DEC_LOCALE_NON_ARIBIC_FILE, "json").select(f.col('json'), f.from_json(f.col('json'), schema, {"locale": locale})),
+ conf =_enable_json_to_structs_conf)
+
#There is no way to set a locale for these, and it really should not matter
@allow_non_gpu(TEXT_INPUT_EXEC)
def test_get_json_object_dec_locale_non_aribic(std_input_path):
@@ -588,6 +741,34 @@ def test_json_tuple_dec_locale_non_aribic(std_input_path):
COMMON_SCAN_TEST_FILES = COMMON_TEST_FILES + [
"scan_emtpy_lines.json"]
+
+@pytest.mark.parametrize('input_file', [
+ "int_formatted.json",
+ "float_formatted.json",
+ "sci_formatted.json",
+ "int_formatted_strings.json",
+ "float_formatted_strings.json",
+ "sci_formatted_strings.json",
+ "decimal_locale_formatted_strings.json",
+ "single_quoted_strings.json",
+ "boolean_formatted.json",
+ "int_array_formatted.json",
+ "int_struct_formatted.json",
+ "int_mixed_array_struct_formatted.json",
+ "bad_whitespace.json",
+ "escaped_strings.json",
+ "nested_escaped_strings.json",
+ "repeated_columns.json", # This works for maps, but not others.
+ "mixed_objects.json",
+ "timestamp_formatted_strings.json",
+ "timestamp_tz_formatted_strings.json"])
+@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
+def test_from_json_map_string_string(std_input_path, input_file):
+ schema = MapType(StringType(), StringType())
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : read_json_as_text(spark, std_input_path + '/' + input_file, "json").select(f.col('json'), f.from_json(f.col('json'), schema)),
+ conf =_enable_json_to_structs_conf)
+
@pytest.mark.parametrize('input_file', COMMON_SCAN_TEST_FILES)
@pytest.mark.parametrize('read_func', [read_json_df]) # we have done so many tests already that we don't need both read func. They are the same
def test_scan_json_bytes(std_input_path, read_func, spark_tmp_table_factory, input_file):
@@ -1484,6 +1665,7 @@ def test_scan_json_mixed_struct(std_input_path, read_func, spark_tmp_table_facto
pytest.param("mixed_objects.json", "data STRUCT>>",
marks=pytest.mark.xfail(condition=is_before_spark_330(), reason='https://github.com/NVIDIA/spark-rapids/issues/11390')),
("mixed_objects.json", "company STRUCT>>>>"),
+ ("mixed_objects.json", "MAP")
])
@allow_non_gpu(TEXT_INPUT_EXEC, *non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453
def test_from_json_mixed_corrected(std_input_path, input_file, schema):
From a847575004f0b335d061671fe38d4e3bc8022461 Mon Sep 17 00:00:00 2001
From: Gera Shegalov
Date: Fri, 22 Nov 2024 10:18:03 -0800
Subject: [PATCH 075/103] Add NVIDIA Copyright (#11723)
Files without a Copyright header found with
```
git grep -L 'Copyright (c) .*NVIDIA' '*.scala' '*.java' | grep
-v com/nvidia/spark/rapids/format
```
Signed-off-by: Gera Shegalov
---
.../sql/rapids/execution/ShuffledBatchRDD.scala | 2 ++
.../spark/sql/hive/rapids/shims/HiveFileUtil.scala | 13 ++++++-------
2 files changed, 8 insertions(+), 7 deletions(-)
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/ShuffledBatchRDD.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/ShuffledBatchRDD.scala
index 17fcebc3fc7..34c99f40dd9 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/ShuffledBatchRDD.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/ShuffledBatchRDD.scala
@@ -1,4 +1,6 @@
/*
+ * Copyright (c) 2020-2022, NVIDIA CORPORATION.
+ *
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala
index 46ee32a8424..41a239ce0ab 100644
--- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala
+++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala
@@ -1,12 +1,11 @@
/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
+ * Copyright (c) 2023-2024, NVIDIA CORPORATION.
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
From e5547a1db358504af414e916003c6d0061bfebaa Mon Sep 17 00:00:00 2001
From: Zach Puller
Date: Fri, 22 Nov 2024 11:06:56 -0800
Subject: [PATCH 076/103] Remove batch size bytes limits (#11746)
* batch limit removed
Signed-off-by: Zach Puller
---------
Signed-off-by: Zach Puller
---
docs/configs.md | 2 +-
.../scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala | 2 --
.../src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala | 6 ++----
3 files changed, 3 insertions(+), 7 deletions(-)
diff --git a/docs/configs.md b/docs/configs.md
index 2d757a40779..7f9544496c4 100644
--- a/docs/configs.md
+++ b/docs/configs.md
@@ -37,7 +37,7 @@ Name | Description | Default Value | Applicable at
spark.rapids.memory.gpu.minAllocFraction|The fraction of total GPU memory that limits the minimum size of the RMM pool. The value must be less than or equal to the setting for spark.rapids.memory.gpu.allocFraction.|0.25|Startup
spark.rapids.memory.host.spillStorageSize|Amount of off-heap host memory to use for buffering spilled GPU data before spilling to local disk. Use -1 to set the amount to the combined size of pinned and pageable memory pools.|-1|Startup
spark.rapids.memory.pinnedPool.size|The size of the pinned memory pool in bytes unless otherwise specified. Use 0 to disable the pool.|0|Startup
-spark.rapids.sql.batchSizeBytes|Set the target number of bytes for a GPU batch. Splits sizes for input data is covered by separate configs. The maximum setting is 2 GB to avoid exceeding the cudf row count limit of a column.|1073741824|Runtime
+spark.rapids.sql.batchSizeBytes|Set the target number of bytes for a GPU batch. Splits sizes for input data is covered by separate configs.|1073741824|Runtime
spark.rapids.sql.concurrentGpuTasks|Set the number of tasks that can execute concurrently per GPU. Tasks may temporarily block when the number of concurrent tasks in the executor exceeds this amount. Allowing too many concurrent tasks on the same GPU may lead to GPU out of memory errors.|2|Runtime
spark.rapids.sql.enabled|Enable (true) or disable (false) sql operations on the GPU|true|Runtime
spark.rapids.sql.explain|Explain why some parts of a query were not placed on a GPU or not. Possible values are ALL: print everything, NONE: print nothing, NOT_ON_GPU: print only parts of a query that did not go on the GPU|NOT_ON_GPU|Runtime
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala
index 1afc03b177b..b7fea71d3ef 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala
@@ -223,8 +223,6 @@ case class RequireSingleBatchWithFilter(filterExpression: GpuExpression)
case class TargetSize(override val targetSizeBytes: Long)
extends CoalesceSizeGoal
with SplittableGoal {
- require(targetSizeBytes <= Integer.MAX_VALUE,
- "Target cannot exceed 2GB without checks for cudf row count limit")
}
/**
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
index 398e5c9fc92..e279385be82 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
@@ -553,12 +553,10 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern")
val GPU_BATCH_SIZE_BYTES = conf("spark.rapids.sql.batchSizeBytes")
.doc("Set the target number of bytes for a GPU batch. Splits sizes for input data " +
- "is covered by separate configs. The maximum setting is 2 GB to avoid exceeding the " +
- "cudf row count limit of a column.")
+ "is covered by separate configs.")
.commonlyUsed()
.bytesConf(ByteUnit.BYTE)
- .checkValue(v => v >= 0 && v <= Integer.MAX_VALUE,
- s"Batch size must be positive and not exceed ${Integer.MAX_VALUE} bytes.")
+ .checkValue(v => v > 0, "Batch size must be positive")
.createWithDefault(1 * 1024 * 1024 * 1024) // 1 GiB is the default
val CHUNKED_READER = conf("spark.rapids.sql.reader.chunked")
From cacc3ae95bb0d07de42ad06f6d14ad63a89e2daf Mon Sep 17 00:00:00 2001
From: Zach Puller
Date: Fri, 22 Nov 2024 11:07:31 -0800
Subject: [PATCH 077/103] host watermark metric (#11725)
* host watermark metric
Signed-off-by: Zach Puller
* make disk and host trackers global
Signed-off-by: Zach Puller
---------
Signed-off-by: Zach Puller
---
.../com/nvidia/spark/rapids/HostAlloc.scala | 20 +++++++
.../spark/sql/rapids/GpuTaskMetrics.scala | 52 ++++++++++++++++---
2 files changed, 64 insertions(+), 8 deletions(-)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala
index 7223463b8b7..6a34d15dc6e 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala
@@ -19,7 +19,9 @@ package com.nvidia.spark.rapids
import ai.rapids.cudf.{DefaultHostMemoryAllocator, HostMemoryAllocator, HostMemoryBuffer, MemoryBuffer, PinnedMemoryPool}
import com.nvidia.spark.rapids.jni.{CpuRetryOOM, RmmSpark}
+import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
+import org.apache.spark.sql.rapids.GpuTaskMetrics
private class HostAlloc(nonPinnedLimit: Long) extends HostMemoryAllocator with Logging {
private var currentNonPinnedAllocated: Long = 0L
@@ -52,10 +54,22 @@ private class HostAlloc(nonPinnedLimit: Long) extends HostMemoryAllocator with L
}
}
+ private def getHostAllocMetricsLogStr(metrics: GpuTaskMetrics): String = {
+ Option(TaskContext.get()).map { context =>
+ val taskId = context.taskAttemptId()
+ val totalSize = metrics.getHostBytesAllocated
+ val maxSize = metrics.getMaxHostBytesAllocated
+ s"total size for task $taskId is $totalSize, max size is $maxSize"
+ }.getOrElse("allocated memory outside of a task context")
+ }
+
private def releasePinned(ptr: Long, amount: Long): Unit = {
synchronized {
currentPinnedAllocated -= amount
}
+ val metrics = GpuTaskMetrics.get
+ metrics.decHostBytesAllocated(amount)
+ logTrace(getHostAllocMetricsLogStr(metrics))
RmmSpark.cpuDeallocate(ptr, amount)
}
@@ -63,6 +77,9 @@ private class HostAlloc(nonPinnedLimit: Long) extends HostMemoryAllocator with L
synchronized {
currentNonPinnedAllocated -= amount
}
+ val metrics = GpuTaskMetrics.get
+ metrics.decHostBytesAllocated(amount)
+ logTrace(getHostAllocMetricsLogStr(metrics))
RmmSpark.cpuDeallocate(ptr, amount)
}
@@ -186,6 +203,9 @@ private class HostAlloc(nonPinnedLimit: Long) extends HostMemoryAllocator with L
allocAttemptFinishedWithoutException = true
} finally {
if (ret.isDefined) {
+ val metrics = GpuTaskMetrics.get
+ metrics.incHostBytesAllocated(amount)
+ logTrace(getHostAllocMetricsLogStr(metrics))
RmmSpark.postCpuAllocSuccess(ret.get.getAddress, amount, blocking, isRecursive)
} else {
// shouldRetry should indicate if spill did anything for us and we should try again.
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala
index 5f1052f0e59..84ca5e2ac51 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala
@@ -121,25 +121,38 @@ class GpuTaskMetrics extends Serializable {
private val readSpillFromDiskTimeNs = new NanoSecondAccumulator
private val maxDeviceMemoryBytes = new HighWatermarkAccumulator
+ private val maxHostMemoryBytes = new HighWatermarkAccumulator
private val maxDiskMemoryBytes = new HighWatermarkAccumulator
- private var diskBytesAllocated: Long = 0
+ private var maxHostBytesAllocated: Long = 0
+
private var maxDiskBytesAllocated: Long = 0
- def getDiskBytesAllocated: Long = diskBytesAllocated
+ def getDiskBytesAllocated: Long = GpuTaskMetrics.diskBytesAllocated
def getMaxDiskBytesAllocated: Long = maxDiskBytesAllocated
+ def getHostBytesAllocated: Long = GpuTaskMetrics.hostBytesAllocated
+
+ def getMaxHostBytesAllocated: Long = maxHostBytesAllocated
+
+ def incHostBytesAllocated(bytes: Long): Unit = {
+ GpuTaskMetrics.incHostBytesAllocated(bytes)
+ maxHostBytesAllocated = maxHostBytesAllocated.max(GpuTaskMetrics.hostBytesAllocated)
+ }
+
+ def decHostBytesAllocated(bytes: Long): Unit = {
+ GpuTaskMetrics.decHostBytesAllocated(bytes)
+ }
+
+
def incDiskBytesAllocated(bytes: Long): Unit = {
- diskBytesAllocated += bytes
- maxDiskBytesAllocated = maxDiskBytesAllocated.max(diskBytesAllocated)
+ GpuTaskMetrics.incDiskBytesAllocated(bytes)
+ maxDiskBytesAllocated = maxDiskBytesAllocated.max(GpuTaskMetrics.diskBytesAllocated)
}
def decDiskBytesAllocated(bytes: Long): Unit = {
- diskBytesAllocated -= bytes
- // For some reason it's possible for the task to start out by releasing resources,
- // possibly from a previous task, in such case we probably should just ignore it.
- diskBytesAllocated = diskBytesAllocated.max(0)
+ GpuTaskMetrics.decHostBytesAllocated(bytes)
}
private val metrics = Map[String, AccumulatorV2[_, _]](
@@ -153,6 +166,7 @@ class GpuTaskMetrics extends Serializable {
"gpuReadSpillFromHostTime" -> readSpillFromHostTimeNs,
"gpuReadSpillFromDiskTime" -> readSpillFromDiskTimeNs,
"gpuMaxDeviceMemoryBytes" -> maxDeviceMemoryBytes,
+ "gpuMaxHostMemoryBytes" -> maxHostMemoryBytes,
"gpuMaxDiskMemoryBytes" -> maxDiskMemoryBytes
)
@@ -242,6 +256,9 @@ class GpuTaskMetrics extends Serializable {
// add method instead of adding a dedicated max method to the accumulator.
maxDeviceMemoryBytes.add(maxMem)
}
+ if (maxHostBytesAllocated > 0) {
+ maxHostMemoryBytes.add(maxHostBytesAllocated)
+ }
if (maxDiskBytesAllocated > 0) {
maxDiskMemoryBytes.add(maxDiskBytesAllocated)
}
@@ -254,6 +271,25 @@ class GpuTaskMetrics extends Serializable {
object GpuTaskMetrics extends Logging {
private val taskLevelMetrics = mutable.Map[Long, GpuTaskMetrics]()
+ private var hostBytesAllocated: Long = 0
+ private var diskBytesAllocated: Long = 0
+
+ private def incHostBytesAllocated(bytes: Long): Unit = synchronized {
+ hostBytesAllocated += bytes
+ }
+
+ private def decHostBytesAllocated(bytes: Long): Unit = synchronized {
+ hostBytesAllocated -= bytes
+ }
+
+ def incDiskBytesAllocated(bytes: Long): Unit = synchronized {
+ diskBytesAllocated += bytes
+ }
+
+ def decDiskBytesAllocated(bytes: Long): Unit = synchronized {
+ diskBytesAllocated -= bytes
+ }
+
def registerOnTask(metrics: GpuTaskMetrics): Unit = synchronized {
val tc = TaskContext.get()
if (tc != null) {
From daaaf24a1dfbc1e643cc9246f6d2ec0e91823f24 Mon Sep 17 00:00:00 2001
From: Nghia Truong <7416935+ttnghia@users.noreply.github.com>
Date: Sat, 23 Nov 2024 11:13:03 -0800
Subject: [PATCH 078/103] Execute `from_json` with struct schema using
`JSONUtils.fromJSONToStructs` (#11618)
* Migrate `castJsonStringToBool` to `JSONUtils.castStringsToBooleans`
Signed-off-by: Nghia Truong
* Migrate undoKeepQuotes` to use `JSONUtils.removeQuote`
Signed-off-by: Nghia Truong
* Migrate `fixupQuotedStrings` to `JSONUtils.removeQuotes`
Signed-off-by: Nghia Truong
* Use `castStringsToDecimals`
Signed-off-by: Nghia Truong
* Use `removeQuotesForFloats` for implementing `castStringToFloat`
Signed-off-by: Nghia Truong
* Use `JSONUtils.castStringsToIntegers`
Signed-off-by: Nghia Truong
* Throw if not supported type
Signed-off-by: Nghia Truong
* Use `JSONUtils.castStringsToDates` for non-legacy conversion
Signed-off-by: Nghia Truong
* Revert "Use `JSONUtils.castStringsToDates` for non-legacy conversion"
This reverts commit b3dcffc155efb12a53ae08fe9035af2c45c65ded.
* Use `JSONUtils.castStringsToFloats`
Signed-off-by: Nghia Truong
* Fix compile error
Signed-off-by: Nghia Truong
* Adopting `fromJSONToStructs`
Signed-off-by: Nghia Truong
* Fix style
Signed-off-by: Nghia Truong
* Adopt `JSONUtils.convertDataType`
Signed-off-by: Nghia Truong
* Cleanup
Signed-off-by: Nghia Truong
* Fix import
Signed-off-by: Nghia Truong
* Revert unrelated change
Signed-off-by: Nghia Truong
* Remove empty lines
Signed-off-by: Nghia Truong
* Change function name
Signed-off-by: Nghia Truong
* Add more data to test
Signed-off-by: Nghia Truong
* Fix test pattern
Signed-off-by: Nghia Truong
* Add test
Signed-off-by: Nghia Truong
---------
Signed-off-by: Nghia Truong
---
.../src/main/python/json_test.py | 30 ++
.../spark/sql/rapids/GpuJsonReadCommon.scala | 263 +++++-------------
.../spark/sql/rapids/GpuJsonToStructs.scala | 105 ++-----
3 files changed, 111 insertions(+), 287 deletions(-)
diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py
index d21e7c46e63..6e8165846e7 100644
--- a/integration_tests/src/main/python/json_test.py
+++ b/integration_tests/src/main/python/json_test.py
@@ -994,6 +994,36 @@ def test_from_json_struct_of_list(schema):
.select(f.from_json('a', schema)),
conf=_enable_all_types_conf)
+@allow_non_gpu(*non_utc_allow)
+def test_from_json_struct_of_list_with_mismatched_schema():
+ json_string_gen = StringGen(r'{"teacher": "[A-Z]{1}[a-z]{2,5}",' \
+ r'"student": \["[A-Z]{1}[a-z]{2,5}"\]}') \
+ .with_special_pattern('', weight=50) \
+ .with_special_pattern('null', weight=50)
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : unary_op_df(spark, json_string_gen) \
+ .select(f.from_json('a', 'struct>>')),
+ conf=_enable_all_types_conf)
+
+@pytest.mark.parametrize('schema', ['struct',
+ 'struct>>',
+ 'struct>>'])
+@allow_non_gpu(*non_utc_allow)
+@pytest.mark.xfail(reason='https://github.com/rapidsai/cudf/issues/17349')
+def test_from_json_struct_of_list_with_mixed_nested_types_input(schema):
+ json_string_gen = StringGen(r'{"teacher": "[A-Z]{1}[a-z]{2,5}",' \
+ r'"student": \[{"name": "[A-Z]{1}[a-z]{2,5}", "class": "junior"},' \
+ r'{"name": "[A-Z]{1}[a-z]{2,5}", "class": "freshman"}\]}') \
+ .with_special_pattern('', weight=50) \
+ .with_special_pattern('null', weight=50) \
+ .with_special_pattern('{"student": \["[A-Z]{1}[a-z]{2,5}"\]}', weight=100) \
+ .with_special_pattern('{"student": \[[1-9]{1,5}\]}', weight=100) \
+ .with_special_pattern('{"student": {"[A-Z]{1}[a-z]{2,5}": "[A-Z]{1}[a-z]{2,5}"}}', weight=100)
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark : unary_op_df(spark, json_string_gen) \
+ .select(f.from_json('a', schema)),
+ conf=_enable_all_types_conf)
+
@pytest.mark.parametrize('schema', [
'struct'
])
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala
index b614a9b170f..017d9722257 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala
@@ -14,17 +14,16 @@
* limitations under the License.
*/
-
package org.apache.spark.sql.rapids
import java.util.Locale
-import ai.rapids.cudf.{BinaryOp, ColumnVector, ColumnView, DType, NvtxColor, NvtxRange, Scalar, Schema, Table}
+import ai.rapids.cudf.{ColumnVector, ColumnView, DType, NvtxColor, NvtxRange, Schema, Table}
import com.fasterxml.jackson.core.JsonParser
-import com.nvidia.spark.rapids.{ColumnCastUtil, GpuCast, GpuColumnVector, GpuScalar, GpuTextBasedPartitionReader}
+import com.nvidia.spark.rapids.{ColumnCastUtil, GpuColumnVector, GpuScalar, GpuTextBasedPartitionReader}
import com.nvidia.spark.rapids.Arm.withResource
import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingArray
-import com.nvidia.spark.rapids.jni.CastStrings
+import com.nvidia.spark.rapids.jni.JSONUtils
import org.apache.spark.sql.catalyst.json.{GpuJsonUtils, JSONOptions}
import org.apache.spark.sql.rapids.shims.GpuJsonToStructsShim
@@ -47,8 +46,10 @@ object GpuJsonReadCommon {
}
case _: MapType =>
throw new IllegalArgumentException("MapType is not supported yet for schema conversion")
+ case dt: DecimalType =>
+ builder.addColumn(GpuColumnVector.getNonNestedRapidsType(dt), name, dt.precision)
case _ =>
- builder.addColumn(DType.STRING, name)
+ builder.addColumn(GpuColumnVector.getNonNestedRapidsType(dt), name)
}
/**
@@ -62,160 +63,6 @@ object GpuJsonReadCommon {
builder.build
}
- private def isQuotedString(input: ColumnView): ColumnVector = {
- withResource(Scalar.fromString("\"")) { quote =>
- withResource(input.startsWith(quote)) { sw =>
- withResource(input.endsWith(quote)) { ew =>
- sw.binaryOp(BinaryOp.LOGICAL_AND, ew, DType.BOOL8)
- }
- }
- }
- }
-
- private def stripFirstAndLastChar(input: ColumnView): ColumnVector = {
- withResource(Scalar.fromInt(1)) { one =>
- val end = withResource(input.getCharLengths) { cc =>
- withResource(cc.sub(one)) { endWithNulls =>
- withResource(endWithNulls.isNull) { eIsNull =>
- eIsNull.ifElse(one, endWithNulls)
- }
- }
- }
- withResource(end) { _ =>
- withResource(ColumnVector.fromScalar(one, end.getRowCount.toInt)) { start =>
- input.substring(start, end)
- }
- }
- }
- }
-
- private def undoKeepQuotes(input: ColumnView): ColumnVector = {
- withResource(isQuotedString(input)) { iq =>
- withResource(stripFirstAndLastChar(input)) { stripped =>
- iq.ifElse(stripped, input)
- }
- }
- }
-
- private def fixupQuotedStrings(input: ColumnView): ColumnVector = {
- withResource(isQuotedString(input)) { iq =>
- withResource(stripFirstAndLastChar(input)) { stripped =>
- withResource(Scalar.fromString(null)) { ns =>
- iq.ifElse(stripped, ns)
- }
- }
- }
- }
-
- private lazy val specialUnquotedFloats =
- Seq("NaN", "+INF", "-INF", "+Infinity", "Infinity", "-Infinity")
- private lazy val specialQuotedFloats = specialUnquotedFloats.map(s => '"'+s+'"')
-
- /**
- * JSON has strict rules about valid numeric formats. See https://www.json.org/ for specification.
- *
- * Spark then has its own rules for supporting NaN and Infinity, which are not
- * valid numbers in JSON.
- */
- private def sanitizeFloats(input: ColumnView, options: JSONOptions): ColumnVector = {
- // Note that this is not 100% consistent with Spark versions prior to Spark 3.3.0
- // due to https://issues.apache.org/jira/browse/SPARK-38060
- if (options.allowNonNumericNumbers) {
- // Need to normalize the quotes to non-quoted to parse properly
- withResource(ColumnVector.fromStrings(specialQuotedFloats: _*)) { quoted =>
- withResource(ColumnVector.fromStrings(specialUnquotedFloats: _*)) { unquoted =>
- input.findAndReplaceAll(quoted, unquoted)
- }
- }
- } else {
- input.copyToColumnVector()
- }
- }
-
- private def sanitizeInts(input: ColumnView): ColumnVector = {
- // Integer numbers cannot look like a float, so no `.` or e The rest of the parsing should
- // handle this correctly. The rest of the validation is in CUDF itself
-
- val tmp = withResource(Scalar.fromString(".")) { dot =>
- withResource(input.stringContains(dot)) { hasDot =>
- withResource(Scalar.fromString("e")) { e =>
- withResource(input.stringContains(e)) { hase =>
- hasDot.or(hase)
- }
- }
- }
- }
- val invalid = withResource(tmp) { _ =>
- withResource(Scalar.fromString("E")) { E =>
- withResource(input.stringContains(E)) { hasE =>
- tmp.or(hasE)
- }
- }
- }
- withResource(invalid) { _ =>
- withResource(Scalar.fromNull(DType.STRING)) { nullString =>
- invalid.ifElse(nullString, input)
- }
- }
- }
-
- private def sanitizeQuotedDecimalInUSLocale(input: ColumnView): ColumnVector = {
- // The US locale is kind of special in that it will remove the , and then parse the
- // input normally
- withResource(stripFirstAndLastChar(input)) { stripped =>
- withResource(Scalar.fromString(",")) { comma =>
- withResource(Scalar.fromString("")) { empty =>
- stripped.stringReplace(comma, empty)
- }
- }
- }
- }
-
- private def sanitizeDecimal(input: ColumnView, options: JSONOptions): ColumnVector = {
- assert(options.locale == Locale.US)
- withResource(isQuotedString(input)) { isQuoted =>
- withResource(sanitizeQuotedDecimalInUSLocale(input)) { quoted =>
- isQuoted.ifElse(quoted, input)
- }
- }
- }
-
- private def castStringToFloat(input: ColumnView, dt: DType,
- options: JSONOptions): ColumnVector = {
- withResource(sanitizeFloats(input, options)) { sanitizedInput =>
- CastStrings.toFloat(sanitizedInput, false, dt)
- }
- }
-
- private def castStringToDecimal(input: ColumnVector, dt: DecimalType): ColumnVector = {
- // TODO there is a bug here around 0 https://github.com/NVIDIA/spark-rapids/issues/10898
- CastStrings.toDecimal(input, false, false, dt.precision, -dt.scale)
- }
-
- private def castJsonStringToBool(input: ColumnView): ColumnVector = {
- // Sadly there is no good kernel right now to do just this check/conversion
- val isTrue = withResource(Scalar.fromString("true")) { trueStr =>
- input.equalTo(trueStr)
- }
- withResource(isTrue) { _ =>
- val isFalse = withResource(Scalar.fromString("false")) { falseStr =>
- input.equalTo(falseStr)
- }
- val falseOrNull = withResource(isFalse) { _ =>
- withResource(Scalar.fromBool(false)) { falseLit =>
- withResource(Scalar.fromNull(DType.BOOL8)) { nul =>
- isFalse.ifElse(falseLit, nul)
- }
- }
- }
- withResource(falseOrNull) { _ =>
- withResource(Scalar.fromBool(true)) { trueLit =>
- isTrue.ifElse(trueLit, falseOrNull)
- }
- }
- }
- }
-
private def dateFormat(options: JSONOptions): Option[String] =
GpuJsonUtils.optionalDateFormatInRead(options)
@@ -228,7 +75,7 @@ object GpuJsonReadCommon {
}
private def nestedColumnViewMismatchTransform(cv: ColumnView,
- dt: DataType): (Option[ColumnView], Seq[AutoCloseable]) = {
+ dt: DataType): (Option[ColumnView], Seq[AutoCloseable]) = {
// In the future we should be able to convert strings to maps/etc, but for
// now we are working around issues where CUDF is not returning a STRING for nested
// types when asked for it.
@@ -264,43 +111,40 @@ object GpuJsonReadCommon {
}
}
+ private def convertStringToDate(input: ColumnView, options: JSONOptions): ColumnVector = {
+ withResource(JSONUtils.removeQuotes(input, /*nullifyIfNotQuoted*/ true)) { removedQuotes =>
+ GpuJsonToStructsShim.castJsonStringToDateFromScan(removedQuotes, DType.TIMESTAMP_DAYS,
+ dateFormat(options))
+ }
+ }
+
+ private def convertStringToTimestamp(input: ColumnView, options: JSONOptions): ColumnVector = {
+ withResource(JSONUtils.removeQuotes(input, /*nullifyIfNotQuoted*/ true)) { removedQuotes =>
+ GpuTextBasedPartitionReader.castStringToTimestamp(removedQuotes, timestampFormat(options),
+ DType.TIMESTAMP_MICROSECONDS)
+ }
+ }
+
private def convertToDesiredType(inputCv: ColumnVector,
topLevelType: DataType,
options: JSONOptions): ColumnVector = {
ColumnCastUtil.deepTransform(inputCv, Some(topLevelType),
Some(nestedColumnViewMismatchTransform)) {
- case (cv, Some(BooleanType)) if cv.getType == DType.STRING =>
- castJsonStringToBool(cv)
case (cv, Some(DateType)) if cv.getType == DType.STRING =>
- withResource(fixupQuotedStrings(cv)) { fixed =>
- GpuJsonToStructsShim.castJsonStringToDateFromScan(fixed, DType.TIMESTAMP_DAYS,
- dateFormat(options))
- }
+ convertStringToDate(cv, options)
case (cv, Some(TimestampType)) if cv.getType == DType.STRING =>
- withResource(fixupQuotedStrings(cv)) { fixed =>
- GpuTextBasedPartitionReader.castStringToTimestamp(fixed, timestampFormat(options),
- DType.TIMESTAMP_MICROSECONDS)
- }
- case (cv, Some(StringType)) if cv.getType == DType.STRING =>
- undoKeepQuotes(cv)
- case (cv, Some(dt: DecimalType)) if cv.getType == DType.STRING =>
- withResource(sanitizeDecimal(cv, options)) { tmp =>
- castStringToDecimal(tmp, dt)
- }
- case (cv, Some(dt)) if (dt == DoubleType || dt == FloatType) && cv.getType == DType.STRING =>
- castStringToFloat(cv, GpuColumnVector.getNonNestedRapidsType(dt), options)
- case (cv, Some(dt))
- if (dt == ByteType || dt == ShortType || dt == IntegerType || dt == LongType ) &&
- cv.getType == DType.STRING =>
- withResource(sanitizeInts(cv)) { tmp =>
- CastStrings.toInteger(tmp, false, GpuColumnVector.getNonNestedRapidsType(dt))
- }
+ convertStringToTimestamp(cv, options)
case (cv, Some(dt)) if cv.getType == DType.STRING =>
- GpuCast.doCast(cv, StringType, dt)
+ // There is an issue with the Schema implementation such that the schema's top level
+ // is never used when passing down data schema from Java to C++.
+ // As such, we have to wrap the current column schema `dt` in a struct schema.
+ val builder = Schema.builder // This is created as a struct schema
+ populateSchema(dt, "", builder)
+ JSONUtils.convertFromStrings(cv, builder.build, options.allowNonNumericNumbers,
+ options.locale == Locale.US)
}
}
-
/**
* Convert the parsed input table to the desired output types
* @param table the table to start with
@@ -320,10 +164,28 @@ object GpuJsonReadCommon {
}
}
- def cudfJsonOptions(options: JSONOptions): ai.rapids.cudf.JSONOptions =
- cudfJsonOptionBuilder(options).build()
+ /**
+ * Convert a strings column into date/time types.
+ * @param inputCv The input column vector
+ * @param topLevelType The desired output data type
+ * @param options JSON options for the conversion
+ * @return The converted column vector
+ */
+ def convertDateTimeType(inputCv: ColumnVector,
+ topLevelType: DataType,
+ options: JSONOptions): ColumnVector = {
+ withResource(new NvtxRange("convertDateTimeType", NvtxColor.RED)) { _ =>
+ ColumnCastUtil.deepTransform(inputCv, Some(topLevelType),
+ Some(nestedColumnViewMismatchTransform)) {
+ case (cv, Some(DateType)) if cv.getType == DType.STRING =>
+ convertStringToDate(cv, options)
+ case (cv, Some(TimestampType)) if cv.getType == DType.STRING =>
+ convertStringToTimestamp(cv, options)
+ }
+ }
+ }
- def cudfJsonOptionBuilder(options: JSONOptions): ai.rapids.cudf.JSONOptions.Builder = {
+ def cudfJsonOptions(options: JSONOptions): ai.rapids.cudf.JSONOptions = {
// This is really ugly, but options.allowUnquotedControlChars is marked as private
// and this is the only way I know to get it without even uglier tricks
@scala.annotation.nowarn("msg=Java enum ALLOW_UNQUOTED_CONTROL_CHARS in " +
@@ -332,16 +194,17 @@ object GpuJsonReadCommon {
.isEnabled(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS)
ai.rapids.cudf.JSONOptions.builder()
- .withRecoverWithNull(true)
- .withMixedTypesAsStrings(true)
- .withNormalizeWhitespace(true)
- .withKeepQuotes(true)
- .withNormalizeSingleQuotes(options.allowSingleQuotes)
- .withStrictValidation(true)
- .withLeadingZeros(options.allowNumericLeadingZeros)
- .withNonNumericNumbers(options.allowNonNumericNumbers)
- .withUnquotedControlChars(allowUnquotedControlChars)
- .withCudfPruneSchema(true)
- .withExperimental(true)
+ .withRecoverWithNull(true)
+ .withMixedTypesAsStrings(true)
+ .withNormalizeWhitespace(true)
+ .withKeepQuotes(true)
+ .withNormalizeSingleQuotes(options.allowSingleQuotes)
+ .withStrictValidation(true)
+ .withLeadingZeros(options.allowNumericLeadingZeros)
+ .withNonNumericNumbers(options.allowNonNumericNumbers)
+ .withUnquotedControlChars(allowUnquotedControlChars)
+ .withCudfPruneSchema(true)
+ .withExperimental(true)
+ .build()
}
}
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala
index 9bcf95847bb..5a08f561a19 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala
@@ -16,55 +16,21 @@
package org.apache.spark.sql.rapids
+import java.util.Locale
+
import ai.rapids.cudf
-import ai.rapids.cudf.{ColumnView, Cuda, DataSource, DeviceMemoryBuffer, HostMemoryBuffer, NvtxColor, NvtxRange}
-import com.nvidia.spark.rapids.{GpuColumnVector, GpuUnaryExpression, HostAlloc}
-import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
+import ai.rapids.cudf.{NvtxColor, NvtxRange}
+import com.nvidia.spark.rapids.{GpuColumnVector, GpuUnaryExpression}
+import com.nvidia.spark.rapids.Arm.withResource
import com.nvidia.spark.rapids.jni.JSONUtils
import com.nvidia.spark.rapids.shims.NullIntolerantShim
import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, TimeZoneAwareExpression}
import org.apache.spark.sql.catalyst.json.JSONOptions
import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.rapids.execution.TrampolineUtil
import org.apache.spark.sql.types._
-/**
- * Exception thrown when cudf cannot parse the JSON data because some Json to Struct cases are not
- * currently supported.
- */
-class JsonParsingException(s: String, cause: Throwable) extends RuntimeException(s, cause) {}
-
-class JsonDeviceDataSource(data: DeviceMemoryBuffer) extends DataSource {
- lazy val totalSize: Long = data.getLength
- override def size(): Long = totalSize
-
- override def hostRead(offset: Long, length: Long): HostMemoryBuffer = {
- val realLength = math.min(totalSize - offset, length)
- withResource(data.slice(offset, realLength)) { sliced =>
- closeOnExcept(HostAlloc.alloc(realLength)) { hostMemoryBuffer =>
- hostMemoryBuffer.copyFromDeviceBuffer(sliced.asInstanceOf[DeviceMemoryBuffer])
- hostMemoryBuffer
- }
- }
- }
-
- override def hostRead(offset: Long, hostMemoryBuffer: HostMemoryBuffer): Long = {
- val length = math.min(totalSize - offset, hostMemoryBuffer.getLength)
- withResource(data.slice(offset, length)) { sliced =>
- hostMemoryBuffer.copyFromDeviceBuffer(sliced.asInstanceOf[DeviceMemoryBuffer])
- }
- length
- }
-
- override def supportsDeviceRead = true
-
- override def deviceRead(offset: Long, dest: DeviceMemoryBuffer, stream: Cuda.Stream): Long = {
- val length = math.min(totalSize - offset, dest.getLength)
- dest.copyFromDeviceBufferAsync(0, data, offset, length, stream)
- length
- }
-}
-
case class GpuJsonToStructs(
schema: DataType,
options: Map[String, String],
@@ -79,59 +45,24 @@ case class GpuJsonToStructs(
timeZoneId.get,
SQLConf.get.columnNameOfCorruptRecord)
- private lazy val jsonOptionBuilder =
- GpuJsonReadCommon.cudfJsonOptionBuilder(parsedOptions)
+ private lazy val cudfOptions = GpuJsonReadCommon.cudfJsonOptions(parsedOptions)
override protected def doColumnar(input: GpuColumnVector): cudf.ColumnVector = {
withResource(new NvtxRange("GpuJsonToStructs", NvtxColor.YELLOW)) { _ =>
schema match {
- case _: MapType =>
- JSONUtils.extractRawMapFromJsonString(input.getBase, jsonOptionBuilder.build())
+ case _: MapType => JSONUtils.extractRawMapFromJsonString(input.getBase, cudfOptions)
case struct: StructType =>
- // if we ever need to support duplicate keys we need to keep track of the duplicates
- // and make the first one null, but I don't think this will ever happen in practice
- val cudfSchema = makeSchema(struct)
-
- // We cannot handle all corner cases with this right now. The parser just isn't
- // good enough, but we will try to handle a few common ones.
- val numRows = input.getRowCount.toInt
-
- // Step 1: Concat the data into a single buffer, with verifying nulls/empty strings
- val concatenated = JSONUtils.concatenateJsonStrings(input.getBase)
- withResource(concatenated) { _ =>
- // Step 2: Setup a datasource from the concatenated JSON strings
- val table = withResource(new JsonDeviceDataSource(concatenated.data)) { ds =>
- withResource(new NvtxRange("Table.readJSON", NvtxColor.RED)) { _ =>
- // Step 3: Have cudf parse the JSON data
- try {
- cudf.Table.readJSON(cudfSchema,
- jsonOptionBuilder.withLineDelimiter(concatenated.delimiter).build(),
- ds,
- numRows)
- } catch {
- case e: RuntimeException =>
- throw new JsonParsingException("Currently some JsonToStructs cases " +
- "are not supported. " +
- "Consider to set spark.rapids.sql.expression.JsonToStructs=false", e)
- }
- }
- }
-
- withResource(table) { _ =>
- // Step 4: Verify that the data looks correct
- if (table.getRowCount != numRows) {
- throw new IllegalStateException("The input data didn't parse correctly and " +
- s"we read a different number of rows than was expected. Expected $numRows, " +
- s"but got ${table.getRowCount}")
- }
-
- // Step 5: Convert the read table into columns of desired types.
- withResource(convertTableToDesiredType(table, struct, parsedOptions)) { columns =>
- // Step 6: Turn the data into structs.
- JSONUtils.makeStructs(columns.asInstanceOf[Array[ColumnView]],
- concatenated.isNullOrEmpty)
- }
+ val parsedStructs = JSONUtils.fromJSONToStructs(input.getBase, makeSchema(struct),
+ cudfOptions, parsedOptions.locale == Locale.US)
+ val hasDateTime = TrampolineUtil.dataTypeExistsRecursively(struct, t =>
+ t.isInstanceOf[DateType] || t.isInstanceOf[TimestampType]
+ )
+ if (hasDateTime) {
+ withResource(parsedStructs) { _ =>
+ convertDateTimeType(parsedStructs, struct, parsedOptions)
}
+ } else {
+ parsedStructs
}
case _ => throw new IllegalArgumentException(
s"GpuJsonToStructs currently does not support schema of type $schema.")
From 6cba00d057a0ce3b487e1455db875e3c8f12a301 Mon Sep 17 00:00:00 2001
From: Liangcai Li
Date: Mon, 25 Nov 2024 14:38:21 +0800
Subject: [PATCH 079/103] Print out the current attempt object when OOM inside
a retry block (#11733)
closes #11732
This PR adds the support to print out the current attempt object being processed
when OOM happens in the retry block.
This is designed for the better OOM issues triage.
---------
Signed-off-by: Firestarman
---
.../spark/rapids/GpuCoalesceBatches.scala | 7 ++-
.../nvidia/spark/rapids/GpuGenerateExec.scala | 3 ++
.../spark/rapids/RmmRapidsRetryIterator.scala | 46 +++++++++++++++++--
.../spark/rapids/SpillableColumnarBatch.scala | 24 +++++++++-
4 files changed, 73 insertions(+), 7 deletions(-)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala
index b7fea71d3ef..cc1196d44e4 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2019-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2019-2024, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -690,6 +690,11 @@ case class BatchesToCoalesce(batches: Array[SpillableColumnarBatch])
override def close(): Unit = {
batches.safeClose()
}
+
+ override def toString: String = {
+ val totalSize = batches.map(_.sizeInBytes).sum
+ s"BatchesToCoalesce totalSize:$totalSize, batches:[${batches.mkString(";")}]"
+ }
}
class GpuCoalesceIterator(iter: Iterator[ColumnarBatch],
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala
index cf83c5b1264..239b7a3d4c0 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala
@@ -953,6 +953,9 @@ class BatchToGenerate(val fixUpOffset: Long, val spillable: SpillableColumnarBat
override def close(): Unit = {
spillable.close()
}
+
+ override def toString: String =
+ s"BatchToGenerate fixUpOffset:$fixUpOffset, spillable:$spillable"
}
class GpuGenerateIterator(
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala
index d86aa596325..04bc56af0c4 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2023, NVIDIA CORPORATION.
+ * Copyright (c) 2023-2024, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -311,6 +311,14 @@ object RmmRapidsRetryIterator extends Logging {
override def iterator: Iterator[T] = ts.iterator
override def apply(idx: Int): T = ts.apply(idx)
+
+ override def toString(): String = {
+ val totalSize = ts.map {
+ case scb: SpillableColumnarBatch => scb.sizeInBytes
+ case _ => 0L
+ }.sum
+ s"AutoCloseableSeqInternal totalSize:$totalSize, inner:[${ts.mkString(";")}]"
+ }
}
/**
@@ -454,14 +462,42 @@ object RmmRapidsRetryIterator extends Logging {
// there is likely not much we can do, and for now we don't handle
// this OOM
if (splitPolicy == null) {
+ val message = s"could not split inputs and retry. The current attempt: " +
+ s"{${attemptStack.head}}"
if (isFromGpuOom) {
- throw new GpuSplitAndRetryOOM("GPU OutOfMemory: could not split inputs and retry")
+ throw new GpuSplitAndRetryOOM(s"GPU OutOfMemory: $message")
} else {
- throw new CpuSplitAndRetryOOM("CPU OutOfMemory: could not split inputs and retry")
+ throw new CpuSplitAndRetryOOM(s"CPU OutOfMemory: $message")
}
}
- // splitPolicy must take ownership of the argument
- val splitted = splitPolicy(attemptStack.pop())
+ val curAttempt = attemptStack.pop()
+ // Get the info before running the split, since the attempt may be closed after splitting.
+ val attemptAsString = closeOnExcept(curAttempt)(_.toString)
+ val splitted = try {
+ // splitPolicy must take ownership of the argument
+ splitPolicy(curAttempt)
+ } catch {
+ // We only care about OOM exceptions and wrap it by a new exception with the
+ // same type to provide more context for the OOM.
+ // This looks a little odd, because we can not change the type of root exception.
+ // Otherwise, some unit tests will fail due to the wrong exception type returned.
+ case go: GpuRetryOOM =>
+ throw new GpuRetryOOM(
+ s"GPU OutOfMemory: Could not split the current attempt: {$attemptAsString}"
+ ).initCause(go)
+ case go: GpuSplitAndRetryOOM =>
+ throw new GpuSplitAndRetryOOM(
+ s"GPU OutOfMemory: Could not split the current attempt: {$attemptAsString}"
+ ).initCause(go)
+ case co: CpuRetryOOM =>
+ throw new CpuRetryOOM(
+ s"CPU OutOfMemory: Could not split the current attempt: {$attemptAsString}"
+ ).initCause(co)
+ case co: CpuSplitAndRetryOOM =>
+ throw new CpuSplitAndRetryOOM(
+ s"CPU OutOfMemory: Could not split the current attempt: {$attemptAsString}"
+ ).initCause(co)
+ }
// the splitted sequence needs to be inserted in reverse order
// so we try the first item first.
splitted.reverse.foreach(attemptStack.push)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala
index d5216cbda9f..e1f45c34180 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala
@@ -55,6 +55,9 @@ trait SpillableColumnarBatch extends AutoCloseable {
def sizeInBytes: Long
def dataTypes: Array[DataType]
+
+ override def toString: String =
+ s"SCB size:$sizeInBytes, types:${dataTypes.toList}, rows:${numRows()}"
}
/**
@@ -79,6 +82,8 @@ class JustRowsColumnarBatch(numRows: Int)
// There is no off heap data and close is a noop so just return this
override def incRefCount(): SpillableColumnarBatch = this
+
+ override def toString: String = s"JustRowsSCB size:$sizeInBytes, rows:$numRows"
}
/**
@@ -148,7 +153,8 @@ class SpillableColumnarBatchImpl (
}
override def toString: String =
- s"SCB $handle $rowCount ${sparkTypes.toList} $refCount"
+ s"GpuSCB size:$sizeInBytes, handle:$handle, rows:$rowCount, types:${sparkTypes.toList}," +
+ s" refCount:$refCount"
}
class JustRowsHostColumnarBatch(numRows: Int)
@@ -167,6 +173,8 @@ class JustRowsHostColumnarBatch(numRows: Int)
// There is no off heap data and close is a noop so just return this
override def incRefCount(): SpillableColumnarBatch = this
+
+ override def toString: String = s"JustRowsHostSCB size:$sizeInBytes, rows:$numRows"
}
/**
@@ -233,6 +241,10 @@ class SpillableHostColumnarBatchImpl (
throw new IllegalStateException("Double free on SpillableHostColumnarBatchImpl")
}
}
+
+ override def toString: String =
+ s"HostSCB size:$sizeInBytes, handle:$handle, rows:$rowCount, types:${sparkTypes.toList}," +
+ s" refCount:$refCount"
}
object SpillableColumnarBatch {
@@ -388,6 +400,13 @@ class SpillableBuffer(
override def close(): Unit = {
handle.close()
}
+
+ override def toString: String = {
+ val size = withResource(RapidsBufferCatalog.acquireBuffer(handle)) { rapidsBuffer =>
+ rapidsBuffer.memoryUsedBytes
+ }
+ s"SpillableBuffer size:$size, handle:$handle"
+ }
}
/**
@@ -422,6 +441,9 @@ class SpillableHostBuffer(handle: RapidsBufferHandle,
rapidsBuffer.getHostMemoryBuffer
}
}
+
+ override def toString: String =
+ s"SpillableHostBuffer length:$length, handle:$handle"
}
object SpillableBuffer {
From 65394412f54f003c5be7b1a572a8e38164a5f025 Mon Sep 17 00:00:00 2001
From: "Robert (Bobby) Evans"
Date: Mon, 25 Nov 2024 12:57:40 -0600
Subject: [PATCH 080/103] Enable JSON Scan and from_json by default (#11753)
Signed-off-by: Robert (Bobby) Evans
Co-authored-by: Nghia Truong <7416935+ttnghia@users.noreply.github.com>
---
.../advanced_configs.md | 6 +-
docs/compatibility.md | 161 ++++++++----------
docs/supported_ops.md | 4 +-
.../nvidia/spark/rapids/GpuOverrides.scala | 8 +-
.../com/nvidia/spark/rapids/RapidsConf.scala | 4 +-
.../320/supportedDataSource.csv | 2 +-
tools/generated_files/320/supportedExprs.csv | 4 +-
.../321/supportedDataSource.csv | 2 +-
tools/generated_files/321/supportedExprs.csv | 4 +-
.../321cdh/supportedDataSource.csv | 2 +-
.../generated_files/321cdh/supportedExprs.csv | 4 +-
.../322/supportedDataSource.csv | 2 +-
tools/generated_files/322/supportedExprs.csv | 4 +-
.../323/supportedDataSource.csv | 2 +-
tools/generated_files/323/supportedExprs.csv | 4 +-
.../324/supportedDataSource.csv | 2 +-
tools/generated_files/324/supportedExprs.csv | 4 +-
.../330/supportedDataSource.csv | 2 +-
tools/generated_files/330/supportedExprs.csv | 4 +-
.../330cdh/supportedDataSource.csv | 2 +-
.../generated_files/330cdh/supportedExprs.csv | 4 +-
.../331/supportedDataSource.csv | 2 +-
tools/generated_files/331/supportedExprs.csv | 4 +-
.../332/supportedDataSource.csv | 2 +-
tools/generated_files/332/supportedExprs.csv | 4 +-
.../332cdh/supportedDataSource.csv | 2 +-
.../generated_files/332cdh/supportedExprs.csv | 4 +-
.../333/supportedDataSource.csv | 2 +-
tools/generated_files/333/supportedExprs.csv | 4 +-
.../334/supportedDataSource.csv | 2 +-
tools/generated_files/334/supportedExprs.csv | 4 +-
.../340/supportedDataSource.csv | 2 +-
tools/generated_files/340/supportedExprs.csv | 4 +-
.../341/supportedDataSource.csv | 2 +-
tools/generated_files/341/supportedExprs.csv | 4 +-
.../342/supportedDataSource.csv | 2 +-
tools/generated_files/342/supportedExprs.csv | 4 +-
.../343/supportedDataSource.csv | 2 +-
tools/generated_files/343/supportedExprs.csv | 4 +-
.../344/supportedDataSource.csv | 2 +-
tools/generated_files/344/supportedExprs.csv | 4 +-
.../350/supportedDataSource.csv | 2 +-
tools/generated_files/350/supportedExprs.csv | 4 +-
.../351/supportedDataSource.csv | 2 +-
tools/generated_files/351/supportedExprs.csv | 4 +-
.../352/supportedDataSource.csv | 2 +-
tools/generated_files/352/supportedExprs.csv | 4 +-
.../353/supportedDataSource.csv | 2 +-
tools/generated_files/353/supportedExprs.csv | 4 +-
.../400/supportedDataSource.csv | 2 +-
tools/generated_files/400/supportedExprs.csv | 4 +-
tools/generated_files/supportedDataSource.csv | 2 +-
tools/generated_files/supportedExprs.csv | 4 +-
53 files changed, 151 insertions(+), 176 deletions(-)
diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md
index f3157b46099..07346a5b850 100644
--- a/docs/additional-functionality/advanced_configs.md
+++ b/docs/additional-functionality/advanced_configs.md
@@ -95,8 +95,8 @@ Name | Description | Default Value | Applicable at
spark.rapids.sql.format.hive.text.write.enabled|When set to false disables Hive text table write acceleration|false|Runtime
spark.rapids.sql.format.iceberg.enabled|When set to false disables all Iceberg acceleration|true|Runtime
spark.rapids.sql.format.iceberg.read.enabled|When set to false disables Iceberg input acceleration|true|Runtime
-spark.rapids.sql.format.json.enabled|When set to true enables all json input and output acceleration. (only input is currently supported anyways)|false|Runtime
-spark.rapids.sql.format.json.read.enabled|When set to true enables json input acceleration|false|Runtime
+spark.rapids.sql.format.json.enabled|When set to true enables all json input and output acceleration. (only input is currently supported anyways)|true|Runtime
+spark.rapids.sql.format.json.read.enabled|When set to true enables json input acceleration|true|Runtime
spark.rapids.sql.format.orc.enabled|When set to false disables all orc input and output acceleration|true|Runtime
spark.rapids.sql.format.orc.floatTypesToString.enable|When reading an ORC file, the source data schemas(schemas of ORC file) may differ from the target schemas (schemas of the reader), we need to handle the castings from source type to target type. Since float/double numbers in GPU have different precision with CPU, when casting float/double to string, the result of GPU is different from result of CPU spark. Its default value is `true` (this means the strings result will differ from result of CPU). If it's set `false` explicitly and there exists casting from float/double to string in the job, then such behavior will cause an exception, and the job will fail.|true|Runtime
spark.rapids.sql.format.orc.multiThreadedRead.maxNumFilesParallel|A limit on the maximum number of files per task processed in parallel on the CPU side before the file is sent to the GPU. This affects the amount of host memory used when reading the files in parallel. Used with MULTITHREADED reader, see spark.rapids.sql.format.orc.reader.type.|2147483647|Runtime
@@ -278,7 +278,7 @@ Name | SQL Function(s) | Description | Default Value | Notes
spark.rapids.sql.expression.IsNaN|`isnan`|Checks if a value is NaN|true|None|
spark.rapids.sql.expression.IsNotNull|`isnotnull`|Checks if a value is not null|true|None|
spark.rapids.sql.expression.IsNull|`isnull`|Checks if a value is null|true|None|
-spark.rapids.sql.expression.JsonToStructs|`from_json`|Returns a struct value with the given `jsonStr` and `schema`|false|This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case|
+spark.rapids.sql.expression.JsonToStructs|`from_json`|Returns a struct value with the given `jsonStr` and `schema`|true|None|
spark.rapids.sql.expression.JsonTuple|`json_tuple`|Returns a tuple like the function get_json_object, but it takes multiple names. All the input parameters and output column types are string.|false|This is disabled by default because Experimental feature that could be unstable or have performance issues.|
spark.rapids.sql.expression.KnownFloatingPointNormalized| |Tag to prevent redundant normalization|true|None|
spark.rapids.sql.expression.KnownNotNull| |Tag an expression as known to not be null|true|None|
diff --git a/docs/compatibility.md b/docs/compatibility.md
index 1382b1a9a1f..0c745069032 100644
--- a/docs/compatibility.md
+++ b/docs/compatibility.md
@@ -316,125 +316,102 @@ case.
## JSON
-The JSON format read is an experimental feature which is expected to have some issues, so we disable
-it by default. If you would like to test it, you need to enable `spark.rapids.sql.format.json.enabled` and
-`spark.rapids.sql.format.json.read.enabled`.
+JSON, despite being a standard format, has some ambiguity in it. Spark also offers the ability to allow
+some invalid JSON to be parsed. We have tried to provide JSON parsing that is compatible with
+what Apache Spark does support. Note that Spark itself has changed through different releases, and we will
+try to call out which releases we offer different results for. JSON parsing is enabled by default
+except for date and timestamp types where we still have work to complete. If you wish to disable
+JSON Scan you can set `spark.rapids.sql.format.json.enabled` or
+`spark.rapids.sql.format.json.read.enabled` to false. To disable `from_json` you can set
+`spark.rapids.sql.expression.JsonToStructs` to false.
-### Invalid JSON
+### Limits
-In Apache Spark on the CPU if a line in the JSON file is invalid the entire row is considered
-invalid and will result in nulls being returned for all columns. It is considered invalid if it
-violates the JSON specification, but with a few extensions.
+In versions of Spark before 3.5.0 there is no maximum to how deeply nested JSON can be. After
+3.5.0 this was updated to be 1,000 by default. The current GPU implementation of JSON Scan and
+`from_json` limits this to 254 no matter what version of Spark is used. If the nesting level is
+over this the JSON is considered invalid and all values will be returned as nulls.
+`get_json_object` and `json_tuple` have a maximum nesting depth of 64. An exception is thrown if
+the nesting depth goes over the maximum.
- * Single quotes are allowed to quote strings and keys
- * Unquoted values like NaN and Infinity can be parsed as floating point values
- * Control characters do not need to be replaced with the corresponding escape sequences in a
- quoted string.
- * Garbage at the end of a row, if there is valid JSON at the beginning of the row, is ignored.
+Spark 3.5.0 and above have limits on maximum string length 20,000,000 and maximum number length of
+1,000. We do not have any of these limits on the GPU.
-The GPU implementation does the same kinds of validations, but many of them are done on a per-column
-basis, which, for example, means if a number is formatted incorrectly, it is likely only that value
-will be considered invalid and return a null instead of nulls for the entire row.
+We, like Spark, cannot support an JSON string that is larger than 2 GiB is size.
-There are options that can be used to enable and disable many of these features which are mostly
-listed below.
+### JSON Validation
-### JSON options
+Spark supports the option `allowNonNumericNumbers`. Versions of Spark prior to 3.3.0 where inconsistent between
+quoted and non-quoted values ([SPARK-38060](https://issues.apache.org/jira/browse/SPARK-38060)). The
+GPU implementation is consistent with 3.3.0 and above.
-Spark supports passing options to the JSON parser when reading a dataset. In most cases if the RAPIDS Accelerator
-sees one of these options that it does not support it will fall back to the CPU. In some cases we do not. The
-following options are documented below.
+### JSON Floating Point Types
-- `allowNumericLeadingZeros` - Allows leading zeros in numbers (e.g. 00012). By default this is set to false.
- When it is false Spark considers the JSON invalid if it encounters this type of number. The RAPIDS
- Accelerator supports validating columns that are returned to the user with this option on or off.
-
-- `allowUnquotedControlChars` - Allows JSON Strings to contain unquoted control characters (ASCII characters with
- value less than 32, including tab and line feed characters) or not. By default this is set to false. If the schema
- is provided while reading JSON file, then this flag has no impact on the RAPIDS Accelerator as it always allows
- unquoted control characters but Spark sees these are invalid are returns nulls. However, if the schema is not provided
- and this option is false, then RAPIDS Accelerator's behavior is same as Spark where an exception is thrown
- as discussed in `JSON Schema discovery` section.
-
-- `allowNonNumericNumbers` - Allows `NaN` and `Infinity` values to be parsed (note that these are not valid numeric
- values in the [JSON specification](https://json.org)). Spark versions prior to 3.3.0 have inconsistent behavior and will
- parse some variants of `NaN` and `Infinity` even when this option is disabled
- ([SPARK-38060](https://issues.apache.org/jira/browse/SPARK-38060)). The RAPIDS Accelerator behavior is consistent with
- Spark version 3.3.0 and later.
-
-### Nesting
-In versions of Spark before 3.5.0 there is no maximum to how deeply nested JSON can be. After
-3.5.0 this was updated to be 1000 by default. The current GPU implementation limits this to 254
-no matter what version of Spark is used. If the nesting level is over this the JSON is considered
-invalid and all values will be returned as nulls.
-
-Mixed types can have some problems. If an item being read could have some lines that are arrays
-and others that are structs/dictionaries it is possible an error will be thrown.
-
-Dates and Timestamps have some issues and may return values for technically invalid inputs.
-
-Floating point numbers have issues generally like with the rest of Spark, and we can parse them into
-a valid floating point number, but it might not match 100% with the way Spark does it.
-
-Strings are supported, but the data returned might not be normalized in the same way as the CPU
-implementation. Generally this comes down to the GPU not modifying the input, whereas Spark will
-do things like remove extra white space and parse numbers before turning them back into a string.
+Parsing floating-point values has the same limitations as [casting from string to float](#string-to-float).
-### JSON Floating Point
+### JSON Integral Types
-Parsing floating-point values has the same limitations as [casting from string to float](#string-to-float).
+Versions of Spark prior to 3.3.0 would parse quoted integer values, like "1". But 3.3.0 and above consider
+these to be invalid and will return `null` when parsed as an Integral types. The GPU implementation
+follows 3.3.0 and above.
-Prior to Spark 3.3.0, reading JSON strings such as `"+Infinity"` when specifying that the data type is `FloatType`
-or `DoubleType` caused these values to be parsed even when `allowNonNumericNumbers` is set to false. Also, Spark
-versions prior to 3.3.0 only supported the `"Infinity"` and `"-Infinity"` representations of infinity and did not
-support `"+INF"`, `"-INF"`, or `"+Infinity"`, which Spark considers valid when unquoted. The GPU JSON reader is
-consistent with the behavior in Spark 3.3.0 and later.
+### JSON Decimal Types
-Another limitation of the GPU JSON reader is that it will parse strings containing non-string boolean or numeric values where
-Spark will treat them as invalid inputs and will just return `null`.
+Spark supports parsing decimal types either formatted as floating point number or integral numbers, even if it is
+in a quoted string. If it is in a quoted string the local of the JVM is used to determine the number format.
+If the local is not for the `US`, which is the default we will fall back to the CPU because we do not currently
+parse those numbers correctly. The `US` format removes all commas ',' from the quoted string.
+As a part of this, though, non-arabic numbers are also supported. We do not support parsing these numbers
+see (issue 10532)[https://github.com/NVIDIA/spark-rapids/issues/10532].
-### JSON Dates/Timestamps
+### JSON Date/Timestamp Types
Dates and timestamps are not supported by default in JSON parser, since the GPU implementation is not 100%
compatible with Apache Spark.
If needed, they can be turned on through the config `spark.rapids.sql.json.read.datetime.enabled`.
-Once enabled, the JSON parser still does not support the `TimestampNTZ` type and will fall back to CPU
-if `spark.sql.timestampType` is set to `TIMESTAMP_NTZ` or if an explicit schema is provided that
-contains the `TimestampNTZ` type.
+This config works for both JSON scan and `from_json`. Once enabled, the JSON parser still does
+not support the `TimestampNTZ` type and will fall back to CPU if `spark.sql.timestampType` is set
+to `TIMESTAMP_NTZ` or if an explicit schema is provided that contains the `TimestampNTZ` type.
There is currently no support for reading numeric values as timestamps and null values are returned instead
-([#4940](https://github.com/NVIDIA/spark-rapids/issues/4940)). A workaround would be to read as longs and then cast
-to timestamp.
+([#4940](https://github.com/NVIDIA/spark-rapids/issues/4940)). A workaround would be to read as longs and then cast to timestamp.
-### JSON Schema discovery
+### JSON Arrays and Structs with Overflowing Numbers
-Spark SQL can automatically infer the schema of a JSON dataset if schema is not provided explicitly. The CPU
-handles schema discovery and there is no GPU acceleration of this. By default Spark will read/parse the entire
-dataset to determine the schema. This means that some options/errors which are ignored by the GPU may still
-result in an exception if used with schema discovery.
+Spark is inconsistent between versions in how it handles numbers that overflow that are nested in either an array
+or a non-top-level struct. In some versions only the value that overflowed is marked as null. In other versions the
+wrapping array or struct is marked as null. We currently only mark the individual value as null. This matches
+versions 3.4.2 and above of Spark for structs. Arrays on most versions of spark invalidate the entire array if there
+is a single value that overflows within it.
-### `from_json` function
+### Duplicate Struct Names
-`JsonToStructs` of `from_json` is based on the same code as reading a JSON lines file. There are
-a few differences with it.
+The JSON specification technically allows for duplicate keys in a struct, but does not explain what to
+do with them. In the case of Spark it is inconsistent between operators which value wins. `get_json_object`
+depends on the query being performed. We do not always match what Spark does. We do match it in many cases,
+but we consider this enough of a corner case that we have not tried to make it work in all cases.
-The `from_json` function is disabled by default because it is experimental and has some known
-incompatibilities with Spark, and can be enabled by setting
-`spark.rapids.sql.expression.JsonToStructs=true`. You don't need to set
-`spark.rapids.sql.format.json.enabled` and`spark.rapids.sql.format.json.read.enabled` to true.
-In addition, if the input schema contains date and/or timestamp types, an additional config
-`spark.rapids.sql.json.read.datetime.enabled` also needs to be set to `true` in order
-to enable this function on the GPU.
+We also do not support schemas where there are duplicate column names. We just fall back to the CPU for those cases.
-There is no schema discovery as a schema is required as input to `from_json`
+### JSON Normalization (String Types)
-In addition to `structs`, a top level `map` type is supported, but only if the key and value are
-strings.
+In versions of Spark prior to 4.0.0 input JSON Strings were parsed to JSON tokens and then converted back to
+strings. This effectively normalizes the output string. So things like single quotes are transformed into double
+quotes, floating point numbers are parsed and converted back to strings possibly changing the format, and
+escaped characters are converted back to their simplest form. We try to support this on the GPU as well. Single quotes
+will be converted to double quotes. Only `get_json_object` and `json_tuple` attempt to normalize floating point
+numbers. There is no implementation on the GPU right now that tries to normalize escape characters.
+
+### `from_json` Function
+
+`JsonToStructs` or `from_json` is based on the same code as reading a JSON lines file. There are
+a few differences with it.
-### `to_json` function
+The main difference is that `from_json` supports parsing Maps and Arrays directly from a JSON column, whereas
+JSON Scan only supports parsing top level structs. The GPU implementation of `from_json` has support for parsing
+a `MAP` as a top level schema, but does not currently support arrays at the top level.
-The `to_json` function is disabled by default because it is experimental and has some known incompatibilities
-with Spark, and can be enabled by setting `spark.rapids.sql.expression.StructsToJson=true`.
+### `to_json` Function
Known issues are:
@@ -442,7 +419,7 @@ Known issues are:
produce `-4.1243574E26` but the GPU may produce `-4.124357351E26`.
- Not all JSON options are respected
-### get_json_object
+### `get_json_object` Function
Known issue:
- [Floating-point number normalization error](https://github.com/NVIDIA/spark-rapids-jni/issues/1922). `get_json_object` floating-point number normalization on the GPU could sometimes return incorrect results if the string contains high-precision values, see the String to Float and Float to String section for more details.
diff --git a/docs/supported_ops.md b/docs/supported_ops.md
index 2fa11f8aa6e..acf7133af40 100644
--- a/docs/supported_ops.md
+++ b/docs/supported_ops.md
@@ -9279,7 +9279,7 @@ are limited.
JsonToStructs |
`from_json` |
Returns a struct value with the given `jsonStr` and `schema` |
-This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case |
+None |
project |
jsonStr |
|
@@ -9320,7 +9320,7 @@ are limited.
|
|
NS |
-PS MAP only supports keys and values that are of STRING type; UTC is only supported TZ for child TIMESTAMP; unsupported child types NULL, BINARY, CALENDAR, MAP, UDT, DAYTIME, YEARMONTH |
+PS MAP only supports keys and values that are of STRING type and is only supported at the top level; UTC is only supported TZ for child TIMESTAMP; unsupported child types NULL, BINARY, CALENDAR, MAP, UDT, DAYTIME, YEARMONTH |
PS UTC is only supported TZ for child TIMESTAMP; unsupported child types NULL, BINARY, CALENDAR, MAP, UDT, DAYTIME, YEARMONTH |
|
|
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
index bdeebaabbfc..45905f0b9e0 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
@@ -3780,7 +3780,8 @@ object GpuOverrides extends Logging {
ExprChecks.projectOnly(
TypeSig.STRUCT.nested(jsonStructReadTypes) +
TypeSig.MAP.nested(TypeSig.STRING).withPsNote(TypeEnum.MAP,
- "MAP only supports keys and values that are of STRING type"),
+ "MAP only supports keys and values that are of STRING type " +
+ "and is only supported at the top level"),
(TypeSig.STRUCT + TypeSig.MAP + TypeSig.ARRAY).nested(TypeSig.all),
Seq(ParamCheck("jsonStr", TypeSig.STRING, TypeSig.STRING))),
(a, conf, p, r) => new UnaryExprMeta[JsonToStructs](a, conf, p, r) {
@@ -3821,10 +3822,7 @@ object GpuOverrides extends Logging {
override def convertToGpu(child: Expression): GpuExpression =
// GPU implementation currently does not support duplicated json key names in input
GpuJsonToStructs(a.schema, a.options, child, a.timeZoneId)
- }).disabledByDefault("it is currently in beta and undergoes continuous enhancements."+
- " Please consult the "+
- "[compatibility documentation](../compatibility.md#json-supporting-types)"+
- " to determine whether you can enable this configuration for your use case"),
+ }),
expr[StructsToJson](
"Converts structs to JSON text format",
ExprChecks.projectOnly(
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
index e279385be82..e22b8f53497 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
@@ -1348,12 +1348,12 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern")
.doc("When set to true enables all json input and output acceleration. " +
"(only input is currently supported anyways)")
.booleanConf
- .createWithDefault(false)
+ .createWithDefault(true)
val ENABLE_JSON_READ = conf("spark.rapids.sql.format.json.read.enabled")
.doc("When set to true enables json input acceleration")
.booleanConf
- .createWithDefault(false)
+ .createWithDefault(true)
val ENABLE_READ_JSON_FLOATS = conf("spark.rapids.sql.json.read.float.enabled")
.doc("JSON reading is not 100% compatible when reading floats.")
diff --git a/tools/generated_files/320/supportedDataSource.csv b/tools/generated_files/320/supportedDataSource.csv
index 2573406ec3b..2eae4ed00ce 100644
--- a/tools/generated_files/320/supportedDataSource.csv
+++ b/tools/generated_files/320/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
diff --git a/tools/generated_files/320/supportedExprs.csv b/tools/generated_files/320/supportedExprs.csv
index 808d8fb4df3..e4a4db760b0 100644
--- a/tools/generated_files/320/supportedExprs.csv
+++ b/tools/generated_files/320/supportedExprs.csv
@@ -288,8 +288,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/321/supportedDataSource.csv b/tools/generated_files/321/supportedDataSource.csv
index 2573406ec3b..2eae4ed00ce 100644
--- a/tools/generated_files/321/supportedDataSource.csv
+++ b/tools/generated_files/321/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
diff --git a/tools/generated_files/321/supportedExprs.csv b/tools/generated_files/321/supportedExprs.csv
index 808d8fb4df3..e4a4db760b0 100644
--- a/tools/generated_files/321/supportedExprs.csv
+++ b/tools/generated_files/321/supportedExprs.csv
@@ -288,8 +288,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/321cdh/supportedDataSource.csv b/tools/generated_files/321cdh/supportedDataSource.csv
index 2573406ec3b..2eae4ed00ce 100644
--- a/tools/generated_files/321cdh/supportedDataSource.csv
+++ b/tools/generated_files/321cdh/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
diff --git a/tools/generated_files/321cdh/supportedExprs.csv b/tools/generated_files/321cdh/supportedExprs.csv
index 808d8fb4df3..e4a4db760b0 100644
--- a/tools/generated_files/321cdh/supportedExprs.csv
+++ b/tools/generated_files/321cdh/supportedExprs.csv
@@ -288,8 +288,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/322/supportedDataSource.csv b/tools/generated_files/322/supportedDataSource.csv
index 2573406ec3b..2eae4ed00ce 100644
--- a/tools/generated_files/322/supportedDataSource.csv
+++ b/tools/generated_files/322/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
diff --git a/tools/generated_files/322/supportedExprs.csv b/tools/generated_files/322/supportedExprs.csv
index 808d8fb4df3..e4a4db760b0 100644
--- a/tools/generated_files/322/supportedExprs.csv
+++ b/tools/generated_files/322/supportedExprs.csv
@@ -288,8 +288,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/323/supportedDataSource.csv b/tools/generated_files/323/supportedDataSource.csv
index 2573406ec3b..2eae4ed00ce 100644
--- a/tools/generated_files/323/supportedDataSource.csv
+++ b/tools/generated_files/323/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
diff --git a/tools/generated_files/323/supportedExprs.csv b/tools/generated_files/323/supportedExprs.csv
index 808d8fb4df3..e4a4db760b0 100644
--- a/tools/generated_files/323/supportedExprs.csv
+++ b/tools/generated_files/323/supportedExprs.csv
@@ -288,8 +288,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/324/supportedDataSource.csv b/tools/generated_files/324/supportedDataSource.csv
index 2573406ec3b..2eae4ed00ce 100644
--- a/tools/generated_files/324/supportedDataSource.csv
+++ b/tools/generated_files/324/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
diff --git a/tools/generated_files/324/supportedExprs.csv b/tools/generated_files/324/supportedExprs.csv
index 808d8fb4df3..e4a4db760b0 100644
--- a/tools/generated_files/324/supportedExprs.csv
+++ b/tools/generated_files/324/supportedExprs.csv
@@ -288,8 +288,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/330/supportedDataSource.csv b/tools/generated_files/330/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/330/supportedDataSource.csv
+++ b/tools/generated_files/330/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/330/supportedExprs.csv b/tools/generated_files/330/supportedExprs.csv
index fcea9c8cb40..0073281cb32 100644
--- a/tools/generated_files/330/supportedExprs.csv
+++ b/tools/generated_files/330/supportedExprs.csv
@@ -297,8 +297,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/330cdh/supportedDataSource.csv b/tools/generated_files/330cdh/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/330cdh/supportedDataSource.csv
+++ b/tools/generated_files/330cdh/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/330cdh/supportedExprs.csv b/tools/generated_files/330cdh/supportedExprs.csv
index fcea9c8cb40..0073281cb32 100644
--- a/tools/generated_files/330cdh/supportedExprs.csv
+++ b/tools/generated_files/330cdh/supportedExprs.csv
@@ -297,8 +297,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/331/supportedDataSource.csv b/tools/generated_files/331/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/331/supportedDataSource.csv
+++ b/tools/generated_files/331/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/331/supportedExprs.csv b/tools/generated_files/331/supportedExprs.csv
index 4eccb898337..f62af4c9513 100644
--- a/tools/generated_files/331/supportedExprs.csv
+++ b/tools/generated_files/331/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/332/supportedDataSource.csv b/tools/generated_files/332/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/332/supportedDataSource.csv
+++ b/tools/generated_files/332/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/332/supportedExprs.csv b/tools/generated_files/332/supportedExprs.csv
index 4eccb898337..f62af4c9513 100644
--- a/tools/generated_files/332/supportedExprs.csv
+++ b/tools/generated_files/332/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/332cdh/supportedDataSource.csv b/tools/generated_files/332cdh/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/332cdh/supportedDataSource.csv
+++ b/tools/generated_files/332cdh/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/332cdh/supportedExprs.csv b/tools/generated_files/332cdh/supportedExprs.csv
index 4eccb898337..f62af4c9513 100644
--- a/tools/generated_files/332cdh/supportedExprs.csv
+++ b/tools/generated_files/332cdh/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/333/supportedDataSource.csv b/tools/generated_files/333/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/333/supportedDataSource.csv
+++ b/tools/generated_files/333/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/333/supportedExprs.csv b/tools/generated_files/333/supportedExprs.csv
index 4eccb898337..f62af4c9513 100644
--- a/tools/generated_files/333/supportedExprs.csv
+++ b/tools/generated_files/333/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/334/supportedDataSource.csv b/tools/generated_files/334/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/334/supportedDataSource.csv
+++ b/tools/generated_files/334/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/334/supportedExprs.csv b/tools/generated_files/334/supportedExprs.csv
index 4eccb898337..f62af4c9513 100644
--- a/tools/generated_files/334/supportedExprs.csv
+++ b/tools/generated_files/334/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/340/supportedDataSource.csv b/tools/generated_files/340/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/340/supportedDataSource.csv
+++ b/tools/generated_files/340/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/340/supportedExprs.csv b/tools/generated_files/340/supportedExprs.csv
index 80bc405b058..01a48b40249 100644
--- a/tools/generated_files/340/supportedExprs.csv
+++ b/tools/generated_files/340/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/341/supportedDataSource.csv b/tools/generated_files/341/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/341/supportedDataSource.csv
+++ b/tools/generated_files/341/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/341/supportedExprs.csv b/tools/generated_files/341/supportedExprs.csv
index 80bc405b058..01a48b40249 100644
--- a/tools/generated_files/341/supportedExprs.csv
+++ b/tools/generated_files/341/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/342/supportedDataSource.csv b/tools/generated_files/342/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/342/supportedDataSource.csv
+++ b/tools/generated_files/342/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/342/supportedExprs.csv b/tools/generated_files/342/supportedExprs.csv
index 80bc405b058..01a48b40249 100644
--- a/tools/generated_files/342/supportedExprs.csv
+++ b/tools/generated_files/342/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/343/supportedDataSource.csv b/tools/generated_files/343/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/343/supportedDataSource.csv
+++ b/tools/generated_files/343/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/343/supportedExprs.csv b/tools/generated_files/343/supportedExprs.csv
index 80bc405b058..01a48b40249 100644
--- a/tools/generated_files/343/supportedExprs.csv
+++ b/tools/generated_files/343/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/344/supportedDataSource.csv b/tools/generated_files/344/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/344/supportedDataSource.csv
+++ b/tools/generated_files/344/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/344/supportedExprs.csv b/tools/generated_files/344/supportedExprs.csv
index 80bc405b058..01a48b40249 100644
--- a/tools/generated_files/344/supportedExprs.csv
+++ b/tools/generated_files/344/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/350/supportedDataSource.csv b/tools/generated_files/350/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/350/supportedDataSource.csv
+++ b/tools/generated_files/350/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/350/supportedExprs.csv b/tools/generated_files/350/supportedExprs.csv
index f45289388fc..4cbfc7c1c27 100644
--- a/tools/generated_files/350/supportedExprs.csv
+++ b/tools/generated_files/350/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/351/supportedDataSource.csv b/tools/generated_files/351/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/351/supportedDataSource.csv
+++ b/tools/generated_files/351/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/351/supportedExprs.csv b/tools/generated_files/351/supportedExprs.csv
index f45289388fc..4cbfc7c1c27 100644
--- a/tools/generated_files/351/supportedExprs.csv
+++ b/tools/generated_files/351/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/352/supportedDataSource.csv b/tools/generated_files/352/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/352/supportedDataSource.csv
+++ b/tools/generated_files/352/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/352/supportedExprs.csv b/tools/generated_files/352/supportedExprs.csv
index f45289388fc..4cbfc7c1c27 100644
--- a/tools/generated_files/352/supportedExprs.csv
+++ b/tools/generated_files/352/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/353/supportedDataSource.csv b/tools/generated_files/353/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/353/supportedDataSource.csv
+++ b/tools/generated_files/353/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/353/supportedExprs.csv b/tools/generated_files/353/supportedExprs.csv
index f45289388fc..4cbfc7c1c27 100644
--- a/tools/generated_files/353/supportedExprs.csv
+++ b/tools/generated_files/353/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/400/supportedDataSource.csv b/tools/generated_files/400/supportedDataSource.csv
index 77f30cbe1de..82df521b39b 100644
--- a/tools/generated_files/400/supportedDataSource.csv
+++ b/tools/generated_files/400/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S
diff --git a/tools/generated_files/400/supportedExprs.csv b/tools/generated_files/400/supportedExprs.csv
index 890f959eab5..4cfa1020889 100644
--- a/tools/generated_files/400/supportedExprs.csv
+++ b/tools/generated_files/400/supportedExprs.csv
@@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
diff --git a/tools/generated_files/supportedDataSource.csv b/tools/generated_files/supportedDataSource.csv
index 2573406ec3b..2eae4ed00ce 100644
--- a/tools/generated_files/supportedDataSource.csv
+++ b/tools/generated_files/supportedDataSource.csv
@@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS
HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
-JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
+JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,NA,NA
diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv
index 808d8fb4df3..e4a4db760b0 100644
--- a/tools/generated_files/supportedExprs.csv
+++ b/tools/generated_files/supportedExprs.csv
@@ -288,8 +288,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P
IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
-JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA
From 938db2123f1bd7397d6270de0e2910cd94823098 Mon Sep 17 00:00:00 2001
From: MithunR
Date: Mon, 25 Nov 2024 12:07:11 -0800
Subject: [PATCH 081/103] Fix aqe_test failures on [databricks] 14.3. (#11750)
* Fix aqe_test failures on [databricks] 14.3.
Fixes #11643.
This commit fixes the AQE/DPP tests that were reported in #11643 to
be failing on Databricks 14.3.
This is the result of a deficient shim for GpuSubqueryBroadcastMeta
being active for Databricks 14.3. The deficient shim errantly
extended the non-Databricks base shim.
This commit moves the commonality in Databricks shims to a common
base class that is then customized for the changes in Databricks 14.3.
Signed-off-by: MithunR
---
integration_tests/src/main/python/aqe_test.py | 6 +-
.../execution/GpuSubqueryBroadcastExec.scala | 6 +-
.../execution/GpuSubqueryBroadcastMeta.scala | 102 ++-------------
.../GpuSubqueryBroadcastMeta330DBBase.scala | 121 ++++++++++++++++++
.../execution/GpuSubqueryBroadcastMeta.scala | 3 +-
.../execution/GpuSubqueryBroadcastMeta.scala | 35 +++++
6 files changed, 171 insertions(+), 102 deletions(-)
create mode 100644 sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta330DBBase.scala
create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala
diff --git a/integration_tests/src/main/python/aqe_test.py b/integration_tests/src/main/python/aqe_test.py
index f9dddfae038..5b3b04efdfb 100755
--- a/integration_tests/src/main/python/aqe_test.py
+++ b/integration_tests/src/main/python/aqe_test.py
@@ -338,10 +338,10 @@ def do_it(spark):
# this should be fixed by https://github.com/NVIDIA/spark-rapids/issues/11120
aqe_join_with_dpp_fallback=["FilterExec"] if (is_databricks_runtime() or is_before_spark_330()) else []
+if is_databricks_version_or_later(14, 3):
+ aqe_join_with_dpp_fallback.append("CollectLimitExec")
# Verify that DPP and AQE can coexist in even some odd cases involving multiple tables
-@pytest.mark.skipif(condition=is_databricks_version_or_later(14, 3),
- reason="https://github.com/NVIDIA/spark-rapids/issues/11643")
@ignore_order(local=True)
@allow_non_gpu(*aqe_join_with_dpp_fallback)
def test_aqe_join_with_dpp(spark_tmp_path):
@@ -395,8 +395,6 @@ def run_test(spark):
assert_gpu_and_cpu_are_equal_collect(run_test, conf=_adaptive_conf)
# Verify that DPP and AQE can coexist in even some odd cases involving 2 tables with multiple columns
-@pytest.mark.skipif(condition=is_databricks_version_or_later(14, 3),
- reason="https://github.com/NVIDIA/spark-rapids/issues/11643")
@ignore_order(local=True)
@allow_non_gpu(*aqe_join_with_dpp_fallback)
def test_aqe_join_with_dpp_multi_columns(spark_tmp_path):
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala
index 72ed0e79504..e529e268f3f 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala
@@ -126,8 +126,10 @@ abstract class GpuSubqueryBroadcastMetaBase(
} else {
willNotWorkOnGpu("underlying BroadcastExchange can not run in the GPU.")
}
- case _ =>
- throw new AssertionError("should not reach here")
+
+ case unexpected =>
+ throw new AssertionError("Unexpected child exec in AdaptiveSparkPlan: " +
+ s"${unexpected.getClass.getName}")
}
case _ =>
diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta.scala
index 76255b3e5a6..ae32800e77a 100644
--- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta.scala
+++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta.scala
@@ -21,105 +21,19 @@
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.rapids.execution
-import com.nvidia.spark.rapids.{BaseExprMeta, DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta, SparkPlanMeta}
+import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta}
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.plans.physical.IdentityBroadcastMode
-import org.apache.spark.sql.execution.{SparkPlan, SubqueryBroadcastExec}
-import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec}
-import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec}
-import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode
+import org.apache.spark.sql.execution.SubqueryBroadcastExec
-class GpuSubqueryBroadcastMeta(
- s: SubqueryBroadcastExec,
- conf: RapidsConf,
- p: Option[RapidsMeta[_, _, _]],
- r: DataFromReplacementRule) extends
- SparkPlanMeta[SubqueryBroadcastExec](s, conf, p, r) {
- private var broadcastBuilder: () => SparkPlan = _
-
- override val childExprs: Seq[BaseExprMeta[_]] = Nil
-
- override val childPlans: Seq[SparkPlanMeta[SparkPlan]] = Nil
-
- override def tagPlanForGpu(): Unit = s.child match {
- // DPP: For AQE off, in this case, we handle DPP by converting the underlying
- // BroadcastExchangeExec to GpuBroadcastExchangeExec.
- // This is slightly different from the Apache Spark case, because Spark
- // sends the underlying plan into the plugin in advance via the PlanSubqueries rule.
- // Here, we have the full non-GPU subquery plan, so we convert the whole
- // thing.
- case ex @ BroadcastExchangeExec(_, child) =>
- val exMeta = new GpuBroadcastMeta(ex.copy(child = child), conf, p, r)
- exMeta.tagForGpu()
- if (exMeta.canThisBeReplaced) {
- broadcastBuilder = () => exMeta.convertToGpu()
- } else {
- willNotWorkOnGpu("underlying BroadcastExchange can not run in the GPU.")
- }
- // DPP: For AQE on, we have an almost completely different scenario then before,
- // Databricks uses a BroadcastQueryStageExec and either:
- // 1) provide an underlying BroadcastExchangeExec that we will have to convert
- // somehow
- // 2) might already do the reuse work for us. The ReusedExchange is now a
- // part of the SubqueryBroadcast, so we send it back here as underlying the
- // GpuSubqueryBroadcastExchangeExec
- case bqse: BroadcastQueryStageExec =>
- bqse.plan match {
- case ex: BroadcastExchangeExec =>
- val exMeta = new GpuBroadcastMeta(ex, conf, p, r)
- exMeta.tagForGpu()
- if (exMeta.canThisBeReplaced) {
- broadcastBuilder = () => exMeta.convertToGpu()
- } else {
- willNotWorkOnGpu("underlying BroadcastExchange can not run in the GPU.")
- }
- case reuse: ReusedExchangeExec =>
- reuse.child match {
- case _: GpuBroadcastExchangeExec =>
- // A BroadcastExchange has already been replaced, so it can run on the GPU
- broadcastBuilder = () => reuse
- case _ =>
- willNotWorkOnGpu("underlying BroadcastExchange can not run in the GPU.")
- }
- }
- case _ =>
- willNotWorkOnGpu("the subquery to broadcast can not entirely run in the GPU.")
- }
- /**
- * Simply returns the original plan. Because its only child, BroadcastExchange, doesn't
- * need to change if SubqueryBroadcastExec falls back to the CPU.
- */
- override def convertToCpu(): SparkPlan = s
+class GpuSubqueryBroadcastMeta(s: SubqueryBroadcastExec,
+ conf: RapidsConf,
+ p: Option[RapidsMeta[_, _, _]],
+ r: DataFromReplacementRule)
+ extends GpuSubqueryBroadcastMeta330DBBase(s, conf, p, r) {
override def convertToGpu(): GpuExec = {
GpuSubqueryBroadcastExec(s.name, Seq(s.index), s.buildKeys, broadcastBuilder())(
getBroadcastModeKeyExprs)
}
- /** Extract the broadcast mode key expressions if there are any. */
- private def getBroadcastModeKeyExprs: Option[Seq[Expression]] = {
- val broadcastMode = s.child match {
- case b: BroadcastExchangeExec =>
- b.mode
- case bqse: BroadcastQueryStageExec =>
- bqse.plan match {
- case b: BroadcastExchangeExec =>
- b.mode
- case reuse: ReusedExchangeExec =>
- reuse.child match {
- case g: GpuBroadcastExchangeExec =>
- g.mode
- }
- case _ =>
- throw new AssertionError("should not reach here")
- }
- }
-
- broadcastMode match {
- case HashedRelationBroadcastMode(keys, _) => Some(keys)
- case IdentityBroadcastMode => None
- case m => throw new UnsupportedOperationException(s"Unknown broadcast mode $m")
- }
- }
-}
+}
\ No newline at end of file
diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta330DBBase.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta330DBBase.scala
new file mode 100644
index 00000000000..a6248127bad
--- /dev/null
+++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta330DBBase.scala
@@ -0,0 +1,121 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/*** spark-rapids-shim-json-lines
+{"spark": "330db"}
+{"spark": "332db"}
+{"spark": "341db"}
+{"spark": "350db143"}
+spark-rapids-shim-json-lines ***/
+package org.apache.spark.sql.rapids.execution
+
+import com.nvidia.spark.rapids.{BaseExprMeta, DataFromReplacementRule, RapidsConf, RapidsMeta, SparkPlanMeta}
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.physical.IdentityBroadcastMode
+import org.apache.spark.sql.execution.{SparkPlan, SubqueryBroadcastExec}
+import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec}
+import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec}
+import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode
+
+abstract class GpuSubqueryBroadcastMeta330DBBase(s: SubqueryBroadcastExec,
+ conf: RapidsConf,
+ p: Option[RapidsMeta[_, _, _]],
+ r: DataFromReplacementRule) extends
+ SparkPlanMeta[SubqueryBroadcastExec](s, conf, p, r) {
+ protected var broadcastBuilder: () => SparkPlan = _
+
+ override val childExprs: Seq[BaseExprMeta[_]] = Nil
+
+ override val childPlans: Seq[SparkPlanMeta[SparkPlan]] = Nil
+
+ override def tagPlanForGpu(): Unit = s.child match {
+ // DPP: For AQE off, in this case, we handle DPP by converting the underlying
+ // BroadcastExchangeExec to GpuBroadcastExchangeExec.
+ // This is slightly different from the Apache Spark case, because Spark
+ // sends the underlying plan into the plugin in advance via the PlanSubqueries rule.
+ // Here, we have the full non-GPU subquery plan, so we convert the whole
+ // thing.
+ case ex @ BroadcastExchangeExec(_, child) =>
+ val exMeta = new GpuBroadcastMeta(ex.copy(child = child), conf, p, r)
+ exMeta.tagForGpu()
+ if (exMeta.canThisBeReplaced) {
+ broadcastBuilder = () => exMeta.convertToGpu()
+ } else {
+ willNotWorkOnGpu("underlying BroadcastExchange can not run in the GPU.")
+ }
+ // DPP: For AQE on, we have an almost completely different scenario then before,
+ // Databricks uses a BroadcastQueryStageExec and either:
+ // 1) provide an underlying BroadcastExchangeExec that we will have to convert
+ // somehow
+ // 2) might already do the reuse work for us. The ReusedExchange is now a
+ // part of the SubqueryBroadcast, so we send it back here as underlying the
+ // GpuSubqueryBroadcastExchangeExec
+ case bqse: BroadcastQueryStageExec =>
+ bqse.plan match {
+ case ex: BroadcastExchangeExec =>
+ val exMeta = new GpuBroadcastMeta(ex, conf, p, r)
+ exMeta.tagForGpu()
+ if (exMeta.canThisBeReplaced) {
+ broadcastBuilder = () => exMeta.convertToGpu()
+ } else {
+ willNotWorkOnGpu("underlying BroadcastExchange can not run in the GPU.")
+ }
+ case reuse: ReusedExchangeExec =>
+ reuse.child match {
+ case _: GpuBroadcastExchangeExec =>
+ // A BroadcastExchange has already been replaced, so it can run on the GPU
+ broadcastBuilder = () => reuse
+ case _ =>
+ willNotWorkOnGpu("underlying BroadcastExchange can not run in the GPU.")
+ }
+ }
+ case _ =>
+ willNotWorkOnGpu("the subquery to broadcast can not entirely run in the GPU.")
+ }
+ /**
+ * Simply returns the original plan. Because its only child, BroadcastExchange, doesn't
+ * need to change if SubqueryBroadcastExec falls back to the CPU.
+ */
+ override def convertToCpu(): SparkPlan = s
+
+ /** Extract the broadcast mode key expressions if there are any. */
+ protected def getBroadcastModeKeyExprs: Option[Seq[Expression]] = {
+ val broadcastMode = s.child match {
+ case b: BroadcastExchangeExec =>
+ b.mode
+ case bqse: BroadcastQueryStageExec =>
+ bqse.plan match {
+ case b: BroadcastExchangeExec =>
+ b.mode
+ case reuse: ReusedExchangeExec =>
+ reuse.child match {
+ case g: GpuBroadcastExchangeExec =>
+ g.mode
+ }
+ case _ =>
+ throw new AssertionError("should not reach here")
+ }
+ }
+
+ broadcastMode match {
+ case HashedRelationBroadcastMode(keys, _) => Some(keys)
+ case IdentityBroadcastMode => None
+ case m => throw new UnsupportedOperationException(s"Unknown broadcast mode $m")
+ }
+ }
+}
+
diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala
index 2f362531646..10e3fa68b76 100644
--- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala
+++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala
@@ -15,7 +15,6 @@
*/
/*** spark-rapids-shim-json-lines
{"spark": "350db143"}
-{"spark": "400"}
spark-rapids-shim-json-lines ***/
package org.apache.spark.sql.rapids.execution
@@ -28,7 +27,7 @@ class GpuSubqueryBroadcastMeta(
conf: RapidsConf,
p: Option[RapidsMeta[_, _, _]],
r: DataFromReplacementRule) extends
- GpuSubqueryBroadcastMetaBase(s, conf, p, r) {
+ GpuSubqueryBroadcastMeta330DBBase(s, conf, p, r) {
override def convertToGpu(): GpuExec = {
GpuSubqueryBroadcastExec(s.name, s.indices, s.buildKeys, broadcastBuilder())(
getBroadcastModeKeyExprs)
diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala
new file mode 100644
index 00000000000..c16564f523e
--- /dev/null
+++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala
@@ -0,0 +1,35 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/*** spark-rapids-shim-json-lines
+{"spark": "400"}
+spark-rapids-shim-json-lines ***/
+package org.apache.spark.sql.rapids.execution
+
+import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta}
+
+import org.apache.spark.sql.execution.SubqueryBroadcastExec
+
+class GpuSubqueryBroadcastMeta(
+ s: SubqueryBroadcastExec,
+ conf: RapidsConf,
+ p: Option[RapidsMeta[_, _, _]],
+ r: DataFromReplacementRule) extends
+ GpuSubqueryBroadcastMetaBase(s, conf, p, r) {
+ override def convertToGpu(): GpuExec = {
+ GpuSubqueryBroadcastExec(s.name, s.indices, s.buildKeys, broadcastBuilder())(
+ getBroadcastModeKeyExprs)
+ }
+}
From 6b90b2fffb9035921fab6cd105469645c09a7b4d Mon Sep 17 00:00:00 2001
From: Jihoon Son
Date: Mon, 25 Nov 2024 14:55:44 -0800
Subject: [PATCH 082/103] Add support for asynchronous writing for parquet
(#11730)
* Support async writing for query output
Signed-off-by: Jihoon Son
* doc change
* use a long timeout
* fix test failure due to a race
* fix flaky test
* address comments
* fix the config name for hold gpu
* Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala
Simplify case arm
Co-authored-by: Gera Shegalov
* address comments
* missing doc change
* use trampoline
---------
Signed-off-by: Jihoon Son
Co-authored-by: Gera Shegalov
---
.../spark/rapids/ColumnarOutputWriter.scala | 36 +++-
.../spark/rapids/GpuParquetFileFormat.scala | 13 +-
.../com/nvidia/spark/rapids/Plugin.scala | 3 +
.../com/nvidia/spark/rapids/RapidsConf.scala | 35 ++++
.../rapids/io/async/AsyncOutputStream.scala | 186 ++++++++++++++++++
.../rapids/io/async/ThrottlingExecutor.scala | 43 ++++
.../rapids/io/async/TrafficController.scala | 142 +++++++++++++
.../io/async/AsyncOutputStreamSuite.scala | 162 +++++++++++++++
.../io/async/ThrottlingExecutorSuite.scala | 145 ++++++++++++++
.../io/async/TrafficControllerSuite.scala | 101 ++++++++++
10 files changed, 855 insertions(+), 11 deletions(-)
create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala
create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutor.scala
create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/TrafficController.scala
create mode 100644 sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStreamSuite.scala
create mode 100644 sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutorSuite.scala
create mode 100644 sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/TrafficControllerSuite.scala
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala
index 69157c046b6..df62683d346 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2019-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2019-2024, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -25,11 +25,13 @@ import com.nvidia.spark.Retryable
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRestoreOnRetry, withRetry, withRetryNoSplit}
+import com.nvidia.spark.rapids.io.async.{AsyncOutputStream, TrafficController}
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FSDataOutputStream, Path}
+import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.TaskAttemptContext
import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
import org.apache.spark.sql.rapids.{ColumnarWriteTaskStatsTracker, GpuWriteTaskStatsTracker}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -70,21 +72,31 @@ abstract class ColumnarOutputWriterFactory extends Serializable {
abstract class ColumnarOutputWriter(context: TaskAttemptContext,
dataSchema: StructType,
rangeName: String,
- includeRetry: Boolean) extends HostBufferConsumer {
+ includeRetry: Boolean,
+ holdGpuBetweenBatches: Boolean = false) extends HostBufferConsumer with Logging {
protected val tableWriter: TableWriter
protected val conf: Configuration = context.getConfiguration
- // This is implemented as a method to make it easier to subclass
- // ColumnarOutputWriter in the tests, and override this behavior.
- protected def getOutputStream: FSDataOutputStream = {
+ private val trafficController: Option[TrafficController] = TrafficController.getInstance
+
+ private def openOutputStream(): OutputStream = {
val hadoopPath = new Path(path)
val fs = hadoopPath.getFileSystem(conf)
fs.create(hadoopPath, false)
}
- protected val outputStream: FSDataOutputStream = getOutputStream
+ // This is implemented as a method to make it easier to subclass
+ // ColumnarOutputWriter in the tests, and override this behavior.
+ protected def getOutputStream: OutputStream = {
+ trafficController.map(controller => {
+ logWarning("Async output write enabled")
+ new AsyncOutputStream(() => openOutputStream(), controller)
+ }).getOrElse(openOutputStream())
+ }
+
+ protected val outputStream: OutputStream = getOutputStream
private[this] val tempBuffer = new Array[Byte](128 * 1024)
private[this] var anythingWritten = false
@@ -166,7 +178,11 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext,
}
// we successfully buffered to host memory, release the semaphore and write
// the buffered data to the FS
- GpuSemaphore.releaseIfNecessary(TaskContext.get)
+ if (!holdGpuBetweenBatches) {
+ logDebug("Releasing semaphore between batches")
+ GpuSemaphore.releaseIfNecessary(TaskContext.get)
+ }
+
writeBufferedData()
updateStatistics(writeStartTime, gpuTime, statsTrackers)
spillableBatch.numRows()
@@ -202,6 +218,10 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext,
// buffer an empty batch on close() to work around issues in cuDF
// where corrupt files can be written if nothing is encoded via the writer.
anythingWritten = true
+
+ // tableWriter.write() serializes the table into the HostMemoryBuffer, and buffers it
+ // by calling handleBuffer() on the ColumnarOutputWriter. It may not write to the
+ // output stream just yet.
tableWriter.write(table)
}
}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala
index 25105386b3d..2b5f246e56a 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala
@@ -271,13 +271,19 @@ class GpuParquetFileFormat extends ColumnarFileFormat with Logging {
s"Set Parquet option ${ParquetOutputFormat.JOB_SUMMARY_LEVEL} to NONE.")
}
+ val asyncOutputWriteEnabled = RapidsConf.ENABLE_ASYNC_OUTPUT_WRITE.get(sqlConf)
+ // holdGpuBetweenBatches is on by default if asyncOutputWriteEnabled is on
+ val holdGpuBetweenBatches = RapidsConf.ASYNC_QUERY_OUTPUT_WRITE_HOLD_GPU_IN_TASK.get(sqlConf)
+ .getOrElse(asyncOutputWriteEnabled)
+
new ColumnarOutputWriterFactory {
override def newInstance(
path: String,
dataSchema: StructType,
context: TaskAttemptContext): ColumnarOutputWriter = {
new GpuParquetWriter(path, dataSchema, compressionType, outputTimestampType.toString,
- dateTimeRebaseMode, timestampRebaseMode, context, parquetFieldIdWriteEnabled)
+ dateTimeRebaseMode, timestampRebaseMode, context, parquetFieldIdWriteEnabled,
+ holdGpuBetweenBatches)
}
override def getFileExtension(context: TaskAttemptContext): String = {
@@ -299,8 +305,9 @@ class GpuParquetWriter(
dateRebaseMode: DateTimeRebaseMode,
timestampRebaseMode: DateTimeRebaseMode,
context: TaskAttemptContext,
- parquetFieldIdEnabled: Boolean)
- extends ColumnarOutputWriter(context, dataSchema, "Parquet", true) {
+ parquetFieldIdEnabled: Boolean,
+ holdGpuBetweenBatches: Boolean)
+ extends ColumnarOutputWriter(context, dataSchema, "Parquet", true, holdGpuBetweenBatches) {
override def throwIfRebaseNeededInExceptionMode(batch: ColumnarBatch): Unit = {
val cols = GpuColumnVector.extractBases(batch)
cols.foreach { col =>
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala
index e20b21da520..5127c7899a8 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala
@@ -31,6 +31,7 @@ import com.nvidia.spark.DFUDFPlugin
import com.nvidia.spark.rapids.RapidsConf.AllowMultipleJars
import com.nvidia.spark.rapids.RapidsPluginUtils.buildInfoEvent
import com.nvidia.spark.rapids.filecache.{FileCache, FileCacheLocalityManager, FileCacheLocalityMsg}
+import com.nvidia.spark.rapids.io.async.TrafficController
import com.nvidia.spark.rapids.jni.GpuTimeZoneDB
import com.nvidia.spark.rapids.python.PythonWorkerSemaphore
import org.apache.commons.lang3.exception.ExceptionUtils
@@ -554,6 +555,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging {
extraExecutorPlugins.foreach(_.init(pluginContext, extraConf))
GpuSemaphore.initialize()
FileCache.init(pluginContext)
+ TrafficController.initialize(conf)
} catch {
// Exceptions in executor plugin can cause a single thread to die but the executor process
// sticks around without any useful info until it hearbeat times out. Print what happened
@@ -656,6 +658,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging {
extraExecutorPlugins.foreach(_.shutdown())
FileCache.shutdown()
GpuCoreDumpHandler.shutdown()
+ TrafficController.shutdown()
}
override def onTaskFailed(failureReason: TaskFailedReason): Unit = {
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
index e22b8f53497..ab7a788d205 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
@@ -2406,6 +2406,36 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression.
.booleanConf
.createWithDefault(false)
+ val ENABLE_ASYNC_OUTPUT_WRITE =
+ conf("spark.rapids.sql.asyncWrite.queryOutput.enabled")
+ .doc("Option to turn on the async query output write. During the final output write, the " +
+ "task first copies the output to the host memory, and then writes it into the storage. " +
+ "When this option is enabled, the task will asynchronously write the output in the host " +
+ "memory to the storage. Only the Parquet format is supported currently.")
+ .internal()
+ .booleanConf
+ .createWithDefault(false)
+
+ val ASYNC_QUERY_OUTPUT_WRITE_HOLD_GPU_IN_TASK =
+ conf("spark.rapids.sql.queryOutput.holdGpuInTask")
+ .doc("Option to hold GPU semaphore between batch processing during the final output write. " +
+ "This option could degrade query performance if it is enabled without the async query " +
+ "output write. It is recommended to consider enabling this option only when " +
+ s"${ENABLE_ASYNC_OUTPUT_WRITE.key} is set. This option is off by default when the async " +
+ "query output write is disabled; otherwise, it is on.")
+ .internal()
+ .booleanConf
+ .createOptional
+
+ val ASYNC_WRITE_MAX_IN_FLIGHT_HOST_MEMORY_BYTES =
+ conf("spark.rapids.sql.asyncWrite.maxInFlightHostMemoryBytes")
+ .doc("Maximum number of host memory bytes per executor that can be in-flight for async " +
+ "query output write. Tasks may be blocked if the total host memory bytes in-flight " +
+ "exceeds this value.")
+ .internal()
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefault(2L * 1024 * 1024 * 1024)
+
private def printSectionHeader(category: String): Unit =
println(s"\n### $category")
@@ -2663,6 +2693,9 @@ class RapidsConf(conf: Map[String, String]) extends Logging {
lazy val isFoldableNonLitAllowed: Boolean = get(FOLDABLE_NON_LIT_ALLOWED)
+ lazy val asyncWriteMaxInFlightHostMemoryBytes: Long =
+ get(ASYNC_WRITE_MAX_IN_FLIGHT_HOST_MEMORY_BYTES)
+
/**
* Convert a string value to the injection configuration OomInjection.
*
@@ -3248,6 +3281,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging {
lazy val caseWhenFuseEnabled: Boolean = get(CASE_WHEN_FUSE)
+ lazy val isAsyncOutputWriteEnabled: Boolean = get(ENABLE_ASYNC_OUTPUT_WRITE)
+
private val optimizerDefaults = Map(
// this is not accurate because CPU projections do have a cost due to appending values
// to each row that is produced, but this needs to be a really small number because
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala
new file mode 100644
index 00000000000..40904a96dd2
--- /dev/null
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala
@@ -0,0 +1,186 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.io.async
+
+import java.io.{IOException, OutputStream}
+import java.util.concurrent.{Callable, TimeUnit}
+import java.util.concurrent.atomic.{AtomicLong, AtomicReference}
+
+import com.nvidia.spark.rapids.RapidsPluginImplicits._
+
+import org.apache.spark.sql.rapids.execution.TrampolineUtil
+
+/**
+ * OutputStream that performs writes asynchronously. Writes are scheduled on a background thread
+ * and executed in the order they were scheduled. This class is not thread-safe and should only be
+ * used by a single thread.
+ */
+class AsyncOutputStream(openFn: Callable[OutputStream], trafficController: TrafficController)
+ extends OutputStream {
+
+ private var closed = false
+
+ private val executor = new ThrottlingExecutor(
+ TrampolineUtil.newDaemonCachedThreadPool("AsyncOutputStream", 1, 1),
+ trafficController)
+
+ // Open the underlying stream asynchronously as soon as the AsyncOutputStream is constructed,
+ // so that the open can be done in parallel with other operations. This could help with
+ // performance if the open is slow.
+ private val openFuture = executor.submit(openFn, 0)
+ // Let's give it enough time to open the stream. Something bad should have happened if it
+ // takes more than 5 minutes to open a stream.
+ private val openTimeoutMin = 5
+
+ private lazy val delegate: OutputStream = {
+ openFuture.get(openTimeoutMin, TimeUnit.MINUTES)
+ }
+
+ class Metrics {
+ var numBytesScheduled: Long = 0
+ // This is thread-safe as it is updated by the background thread and can be read by
+ // any threads.
+ val numBytesWritten: AtomicLong = new AtomicLong(0)
+ }
+
+ val metrics = new Metrics
+
+ /**
+ * The last error that occurred in the background thread, or None if no error occurred.
+ * Once it is set, all subsequent writes that are already scheduled will fail and no new
+ * writes will be accepted.
+ *
+ * This is thread-safe as it is set by the background thread and can be read by any threads.
+ */
+ val lastError: AtomicReference[Option[Throwable]] =
+ new AtomicReference[Option[Throwable]](None)
+
+ @throws[IOException]
+ private def throwIfError(): Unit = {
+ lastError.get() match {
+ case Some(t: IOException) => throw t
+ case Some(t) => throw new IOException(t)
+ case None =>
+ }
+ }
+
+ @throws[IOException]
+ private def ensureOpen(): Unit = {
+ if (closed) {
+ throw new IOException("Stream closed")
+ }
+ }
+
+ private def scheduleWrite(fn: () => Unit, bytesToWrite: Int): Unit = {
+ throwIfError()
+ ensureOpen()
+
+ metrics.numBytesScheduled += bytesToWrite
+ executor.submit(() => {
+ throwIfError()
+ ensureOpen()
+
+ try {
+ fn()
+ metrics.numBytesWritten.addAndGet(bytesToWrite)
+ } catch {
+ case t: Throwable =>
+ // Update the error state
+ lastError.set(Some(t))
+ }
+ }, bytesToWrite)
+ }
+
+ override def write(b: Int): Unit = {
+ scheduleWrite(() => delegate.write(b), 1)
+ }
+
+ override def write(b: Array[Byte]): Unit = {
+ scheduleWrite(() => delegate.write(b), b.length)
+ }
+
+ /**
+ * Schedules a write of the given bytes to the underlying stream. The write is executed
+ * asynchronously on a background thread. The method returns immediately, and the write may not
+ * have completed when the method returns.
+ *
+ * If an error has occurred in the background thread and [[lastError]] has been set, this function
+ * will throw an IOException immediately.
+ *
+ * If an error has occurred in the background thread while executing a previous write after the
+ * current write has been scheduled, the current write will fail with the same error.
+ */
+ @throws[IOException]
+ override def write(b: Array[Byte], off: Int, len: Int): Unit = {
+ scheduleWrite(() => delegate.write(b, off, len), len)
+ }
+
+ /**
+ * Flushes all pending writes to the underlying stream. This method blocks until all pending
+ * writes have been completed. If an error has occurred in the background thread, this method
+ * will throw an IOException.
+ *
+ * If an error has occurred in the background thread and [[lastError]] has been set, this function
+ * will throw an IOException immediately.
+ *
+ * If an error has occurred in the background thread while executing a previous task after the
+ * current flush has been scheduled, the current flush will fail with the same error.
+ */
+ @throws[IOException]
+ override def flush(): Unit = {
+ throwIfError()
+ ensureOpen()
+
+ val f = executor.submit(() => {
+ throwIfError()
+ ensureOpen()
+
+ delegate.flush()
+ }, 0)
+
+ f.get()
+ }
+
+ /**
+ * Closes the underlying stream and releases any resources associated with it. All pending writes
+ * are flushed before closing the stream. This method blocks until all pending writes have been
+ * completed.
+ *
+ * If an error has occurred while flushing, this function will throw an IOException.
+ *
+ * If an error has occurred while executing a previous task before this function is called,
+ * this function will throw the same error. All resources and the underlying stream are still
+ * guaranteed to be closed.
+ */
+ @throws[IOException]
+ override def close(): Unit = {
+ if (!closed) {
+ Seq[AutoCloseable](
+ () => {
+ // Wait for all pending writes to complete
+ // This will throw an exception if one of the writes fails
+ flush()
+ },
+ () => {
+ // Give the executor a chance to shutdown gracefully.
+ executor.shutdownNow(10, TimeUnit.SECONDS)
+ },
+ delegate,
+ () => closed = true).safeClose()
+ }
+ }
+}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutor.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutor.scala
new file mode 100644
index 00000000000..45889bf89ac
--- /dev/null
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutor.scala
@@ -0,0 +1,43 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.io.async
+
+import java.util.concurrent.{Callable, ExecutorService, Future, TimeUnit}
+
+/**
+ * Thin wrapper around an ExecutorService that adds throttling.
+ */
+class ThrottlingExecutor(
+ val executor: ExecutorService, throttler: TrafficController) {
+
+ def submit[T](callable: Callable[T], hostMemoryBytes: Long): Future[T] = {
+ val task = new Task[T](hostMemoryBytes, callable)
+ throttler.blockUntilRunnable(task)
+ executor.submit(() => {
+ try {
+ task.call()
+ } finally {
+ throttler.taskCompleted(task)
+ }
+ })
+ }
+
+ def shutdownNow(timeout: Long, timeUnit: TimeUnit): Unit = {
+ executor.shutdownNow()
+ executor.awaitTermination(timeout, timeUnit)
+ }
+}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/TrafficController.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/TrafficController.scala
new file mode 100644
index 00000000000..0110f2d89ca
--- /dev/null
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/TrafficController.scala
@@ -0,0 +1,142 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.io.async
+
+import java.util.concurrent.Callable
+import javax.annotation.concurrent.GuardedBy
+
+import com.nvidia.spark.rapids.RapidsConf
+
+/**
+ * Simple wrapper around a [[Callable]] that also keeps track of the host memory bytes used by
+ * the task.
+ *
+ * Note: we may want to add more metadata to the task in the future, such as the device memory,
+ * as we implement more throttling strategies.
+ */
+class Task[T](val hostMemoryBytes: Long, callable: Callable[T]) extends Callable[T] {
+ override def call(): T = callable.call()
+}
+
+/**
+ * Throttle interface to be implemented by different throttling strategies.
+ *
+ * Currently, only HostMemoryThrottle is implemented, which limits the maximum in-flight host
+ * memory bytes. In the future, we can add more throttling strategies, such as limiting the
+ * device memory usage, the number of tasks, etc.
+ */
+trait Throttle {
+
+ /**
+ * Returns true if the task can be accepted, false otherwise.
+ * TrafficController will block the task from being scheduled until this method returns true.
+ */
+ def canAccept[T](task: Task[T]): Boolean
+
+ /**
+ * Callback to be called when a task is scheduled.
+ */
+ def taskScheduled[T](task: Task[T]): Unit
+
+ /**
+ * Callback to be called when a task is completed, either successfully or with an exception.
+ */
+ def taskCompleted[T](task: Task[T]): Unit
+}
+
+/**
+ * Throttle implementation that limits the total host memory used by the in-flight tasks.
+ */
+class HostMemoryThrottle(val maxInFlightHostMemoryBytes: Long) extends Throttle {
+ private var totalHostMemoryBytes: Long = 0
+
+ override def canAccept[T](task: Task[T]): Boolean = {
+ totalHostMemoryBytes + task.hostMemoryBytes <= maxInFlightHostMemoryBytes
+ }
+
+ override def taskScheduled[T](task: Task[T]): Unit = {
+ totalHostMemoryBytes += task.hostMemoryBytes
+ }
+
+ override def taskCompleted[T](task: Task[T]): Unit = {
+ totalHostMemoryBytes -= task.hostMemoryBytes
+ }
+
+ def getTotalHostMemoryBytes: Long = totalHostMemoryBytes
+}
+
+/**
+ * TrafficController is responsible for blocking tasks from being scheduled when the throttle
+ * is exceeded. It also keeps track of the number of tasks that are currently scheduled.
+ *
+ * This class is thread-safe as it is used by multiple tasks.
+ */
+class TrafficController protected[rapids] (throttle: Throttle) {
+
+ @GuardedBy("this")
+ private var numTasks: Int = 0
+
+ /**
+ * Blocks the task from being scheduled until the throttle allows it. If there is no task
+ * currently scheduled, the task is scheduled immediately even if the throttle is exceeded.
+ */
+ def blockUntilRunnable[T](task: Task[T]): Unit = synchronized {
+ if (numTasks > 0) {
+ while (!throttle.canAccept(task)) {
+ wait(100)
+ }
+ }
+ numTasks += 1
+ throttle.taskScheduled(task)
+ }
+
+ def taskCompleted[T](task: Task[T]): Unit = synchronized {
+ numTasks -= 1
+ throttle.taskCompleted(task)
+ notify()
+ }
+
+ def numScheduledTasks: Int = synchronized {
+ numTasks
+ }
+}
+
+object TrafficController {
+
+ private var instance: TrafficController = _
+
+ /**
+ * Initializes the TrafficController singleton instance.
+ * This is called once per executor.
+ */
+ def initialize(conf: RapidsConf): Unit = synchronized {
+ if (conf.isAsyncOutputWriteEnabled && instance == null) {
+ instance = new TrafficController(
+ new HostMemoryThrottle(conf.asyncWriteMaxInFlightHostMemoryBytes))
+ }
+ }
+
+ def getInstance: Option[TrafficController] = synchronized {
+ Option(instance)
+ }
+
+ def shutdown(): Unit = synchronized {
+ if (instance != null) {
+ instance = null
+ }
+ }
+}
diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStreamSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStreamSuite.scala
new file mode 100644
index 00000000000..a4fa35349ce
--- /dev/null
+++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStreamSuite.scala
@@ -0,0 +1,162 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.io.async
+
+import java.io.{BufferedOutputStream, File, FileOutputStream, IOException, OutputStream}
+import java.util.concurrent.Callable
+
+import com.nvidia.spark.rapids.Arm.withResource
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.funsuite.AnyFunSuite
+
+class AsyncOutputStreamSuite extends AnyFunSuite with BeforeAndAfterEach {
+
+ private val bufLen = 128 * 1024
+ private val buf: Array[Byte] = new Array[Byte](bufLen)
+ private val maxBufCount = 10
+ private val trafficController = new TrafficController(
+ new HostMemoryThrottle(bufLen * maxBufCount))
+
+ def openStream(): AsyncOutputStream = {
+ new AsyncOutputStream(() => {
+ val file = File.createTempFile("async-write-test", "tmp")
+ new BufferedOutputStream(new FileOutputStream(file))
+ }, trafficController)
+ }
+
+ test("open, write, and close") {
+ val numBufs = 1000
+ val stream = openStream()
+ withResource(stream) { os =>
+ for (_ <- 0 until numBufs) {
+ os.write(buf)
+ }
+ }
+ assertResult(bufLen * numBufs)(stream.metrics.numBytesScheduled)
+ assertResult(bufLen * numBufs)(stream.metrics.numBytesWritten.get())
+ }
+
+ test("write after closed") {
+ val os = openStream()
+ os.close()
+ assertThrows[IOException] {
+ os.write(buf)
+ }
+ }
+
+ test("flush after closed") {
+ val os = openStream()
+ os.close()
+ assertThrows[IOException] {
+ os.flush()
+ }
+ }
+
+ class ThrowingOutputStream extends OutputStream {
+
+ var failureCount = 0
+
+ override def write(i: Int): Unit = {
+ failureCount += 1
+ throw new IOException(s"Failed ${failureCount} times")
+ }
+
+ override def write(b: Array[Byte], off: Int, len: Int): Unit = {
+ failureCount += 1
+ throw new IOException(s"Failed ${failureCount} times")
+ }
+ }
+
+ def assertThrowsWithMsg[T](fn: Callable[T], clue: String,
+ expectedMsgPrefix: String): Unit = {
+ withClue(clue) {
+ try {
+ fn.call()
+ } catch {
+ case t: Throwable =>
+ assertIOExceptionMsg(t, expectedMsgPrefix)
+ }
+ }
+ }
+
+ def assertIOExceptionMsg(t: Throwable, expectedMsgPrefix: String): Unit = {
+ if (t.getClass.isAssignableFrom(classOf[IOException])) {
+ if (!t.getMessage.contains(expectedMsgPrefix)) {
+ fail(s"Unexpected exception message: ${t.getMessage}")
+ }
+ } else {
+ if (t.getCause != null) {
+ assertIOExceptionMsg(t.getCause, expectedMsgPrefix)
+ } else {
+ fail(s"Unexpected exception: $t")
+ }
+ }
+ }
+
+ test("write after error") {
+ val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController)
+
+ // The first call to `write` should succeed
+ os.write(buf)
+
+ // Wait for the first write to fail
+ while (os.lastError.get().isEmpty) {
+ Thread.sleep(100)
+ }
+
+ // The second `write` call should fail with the exception thrown by the first write failure
+ assertThrowsWithMsg(() => os.write(buf),
+ "The second write should fail with the exception thrown by the first write failure",
+ "Failed 1 times")
+
+ // `close` throws the same exception
+ assertThrowsWithMsg(() => os.close(),
+ "The second write should fail with the exception thrown by the first write failure",
+ "Failed 1 times")
+
+ assertResult(bufLen)(os.metrics.numBytesScheduled)
+ assertResult(0)(os.metrics.numBytesWritten.get())
+ assert(os.lastError.get().get.isInstanceOf[IOException])
+ }
+
+ test("flush after error") {
+ val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController)
+
+ // The first write should succeed
+ os.write(buf)
+
+ // The flush should fail with the exception thrown by the write failure
+ assertThrowsWithMsg(() => os.flush(),
+ "The flush should fail with the exception thrown by the write failure",
+ "Failed 1 times")
+
+ // `close` throws the same exception
+ assertThrowsWithMsg(() => os.close(),
+ "The flush should fail with the exception thrown by the write failure",
+ "Failed 1 times")
+ }
+
+ test("close after error") {
+ val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController)
+
+ os.write(buf)
+
+ assertThrowsWithMsg(() => os.close(),
+ "Close should fail with the exception thrown by the write failure",
+ "Failed 1 times")
+ }
+}
diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutorSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutorSuite.scala
new file mode 100644
index 00000000000..a8acf240878
--- /dev/null
+++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutorSuite.scala
@@ -0,0 +1,145 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.io.async
+
+import java.util.concurrent.{Callable, CountDownLatch, ExecutionException, Executors, Future, RejectedExecutionException, TimeUnit}
+
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.funsuite.AnyFunSuite
+
+class ThrottlingExecutorSuite extends AnyFunSuite with BeforeAndAfterEach {
+
+ // Some tests might take longer than usual in the limited CI environment.
+ // Use a long timeout to avoid flakiness.
+ val longTimeoutSec = 5
+
+ var throttle: HostMemoryThrottle = _
+ var trafficController: TrafficController = _
+ var executor: ThrottlingExecutor = _
+
+ class TestTask extends Callable[Unit] {
+ val latch = new CountDownLatch(1)
+ override def call(): Unit = {
+ latch.await()
+ }
+ }
+
+ override def beforeEach(): Unit = {
+ throttle = new HostMemoryThrottle(100)
+ trafficController = new TrafficController(throttle)
+ executor = new ThrottlingExecutor(
+ Executors.newSingleThreadExecutor(),
+ trafficController
+ )
+ }
+
+ override def afterEach(): Unit = {
+ executor.shutdownNow(longTimeoutSec, TimeUnit.SECONDS)
+ }
+
+ test("tasks submitted should update the state") {
+ val task1 = new TestTask
+ val future1 = executor.submit(task1, 10)
+ assertResult(1)(trafficController.numScheduledTasks)
+ assertResult(10)(throttle.getTotalHostMemoryBytes)
+
+ val task2 = new TestTask
+ val future2 = executor.submit(task2, 20)
+ assertResult(2)(trafficController.numScheduledTasks)
+ assertResult(30)(throttle.getTotalHostMemoryBytes)
+
+ task1.latch.countDown()
+ future1.get(longTimeoutSec, TimeUnit.SECONDS)
+ assertResult(1)(trafficController.numScheduledTasks)
+ assertResult(20)(throttle.getTotalHostMemoryBytes)
+
+ task2.latch.countDown()
+ future2.get(longTimeoutSec, TimeUnit.SECONDS)
+ assertResult(0)(trafficController.numScheduledTasks)
+ assertResult(0)(throttle.getTotalHostMemoryBytes)
+ }
+
+ test("tasks submission fails if total weight exceeds maxWeight") {
+ val task1 = new TestTask
+ val future1 = executor.submit(task1, 10)
+ assertResult(1)(trafficController.numScheduledTasks)
+ assertResult(10)(throttle.getTotalHostMemoryBytes)
+
+ val task2 = new TestTask
+ val task2Weight = 100
+ val exec = Executors.newSingleThreadExecutor()
+ val future2 = exec.submit(new Runnable {
+ override def run(): Unit = executor.submit(task2, task2Weight)
+ })
+ Thread.sleep(100)
+ assert(!future2.isDone)
+ assertResult(1)(trafficController.numScheduledTasks)
+ assertResult(10)(throttle.getTotalHostMemoryBytes)
+
+ task1.latch.countDown()
+ future1.get(longTimeoutSec, TimeUnit.SECONDS)
+ future2.get(longTimeoutSec, TimeUnit.SECONDS)
+ assertResult(1)(trafficController.numScheduledTasks)
+ assertResult(task2Weight)(throttle.getTotalHostMemoryBytes)
+ }
+
+ test("submit one task heavier than maxWeight") {
+ val future = executor.submit(() => Thread.sleep(10), throttle.maxInFlightHostMemoryBytes + 1)
+ future.get(longTimeoutSec, TimeUnit.SECONDS)
+ assert(future.isDone)
+ assertResult(0)(trafficController.numScheduledTasks)
+ assertResult(0)(throttle.getTotalHostMemoryBytes)
+ }
+
+ test("submit multiple tasks such that total weight does not exceed maxWeight") {
+ val numTasks = 10
+ val taskRunTime = 10
+ var future: Future[Unit] = null
+ for (_ <- 0 to numTasks) {
+ future = executor.submit(() => Thread.sleep(taskRunTime), 1)
+ }
+ // Give enough time for all tasks to complete
+ future.get(numTasks * taskRunTime * 5, TimeUnit.MILLISECONDS)
+ assertResult(0)(trafficController.numScheduledTasks)
+ assertResult(0)(throttle.getTotalHostMemoryBytes)
+ }
+
+ test("shutdown while a task is blocked") {
+ val task1 = new TestTask
+ val future1 = executor.submit(task1, 10)
+ assertResult(1)(trafficController.numScheduledTasks)
+ assertResult(10)(throttle.getTotalHostMemoryBytes)
+
+ val task2 = new TestTask
+ val task2Weight = 100
+ val exec = Executors.newSingleThreadExecutor()
+ val future2 = exec.submit(new Runnable {
+ override def run(): Unit = executor.submit(task2, task2Weight)
+ })
+ executor.shutdownNow(longTimeoutSec, TimeUnit.SECONDS)
+
+ def assertCause(t: Throwable, cause: Class[_]): Unit = {
+ assert(t.getCause != null)
+ assert(cause.isInstance(t.getCause))
+ }
+
+ val e1 = intercept[ExecutionException](future1.get())
+ assertCause(e1, classOf[InterruptedException])
+ val e2 = intercept[ExecutionException](future2.get())
+ assertCause(e2, classOf[RejectedExecutionException])
+ }
+}
diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/TrafficControllerSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/TrafficControllerSuite.scala
new file mode 100644
index 00000000000..32868ff6055
--- /dev/null
+++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/TrafficControllerSuite.scala
@@ -0,0 +1,101 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.io.async
+
+import java.util.concurrent.{ExecutionException, Executors, ExecutorService, TimeUnit}
+
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.funsuite.AnyFunSuite
+
+class TrafficControllerSuite extends AnyFunSuite with BeforeAndAfterEach {
+
+ private var throttle: HostMemoryThrottle = _
+ private var controller: TrafficController = _
+ private var executor: ExecutorService = _
+
+ override def beforeEach(): Unit = {
+ throttle = new HostMemoryThrottle(100)
+ controller = new TrafficController(throttle)
+ executor = Executors.newSingleThreadExecutor()
+ }
+
+ override def afterEach(): Unit = {
+ executor.shutdownNow()
+ executor.awaitTermination(1, TimeUnit.SECONDS)
+ }
+
+ class TestTask(taskMemoryBytes: Long) extends Task[Unit](taskMemoryBytes, () => {}) {}
+
+ test("schedule tasks without blocking") {
+ val taskMemoryBytes = 50
+ val t1 = new TestTask(taskMemoryBytes)
+ controller.blockUntilRunnable(t1)
+ assertResult(1)(controller.numScheduledTasks)
+ assertResult(taskMemoryBytes)(throttle.getTotalHostMemoryBytes)
+
+ val t2 = new TestTask(50)
+ controller.blockUntilRunnable(t2)
+ assertResult(2)(controller.numScheduledTasks)
+ assertResult(2 * taskMemoryBytes)(throttle.getTotalHostMemoryBytes)
+
+ controller.taskCompleted(t1)
+ assertResult(1)(controller.numScheduledTasks)
+ assertResult(taskMemoryBytes)(throttle.getTotalHostMemoryBytes)
+ }
+
+ test("schedule task with blocking") {
+ val taskMemoryBytes = 50
+ val t1 = new TestTask(taskMemoryBytes)
+ controller.blockUntilRunnable(t1)
+
+ val t2 = new TestTask(taskMemoryBytes)
+ controller.blockUntilRunnable(t2)
+
+ val t3 = new TestTask(taskMemoryBytes)
+ val f = executor.submit(new Runnable {
+ override def run(): Unit = controller.blockUntilRunnable(t3)
+ })
+ Thread.sleep(100)
+ assert(!f.isDone)
+
+ controller.taskCompleted(t1)
+ f.get(1, TimeUnit.SECONDS)
+ }
+
+ test("shutdown while blocking") {
+ val t1 = new TestTask(10)
+ controller.blockUntilRunnable(t1)
+
+ val t2 = new TestTask(110)
+
+ val f = executor.submit(new Runnable {
+ override def run(): Unit = {
+ controller.blockUntilRunnable(t2)
+ }
+ })
+
+ executor.shutdownNow()
+ try {
+ f.get(1, TimeUnit.SECONDS)
+ fail("Should be interrupted")
+ } catch {
+ case ee: ExecutionException =>
+ assert(ee.getCause.isInstanceOf[InterruptedException])
+ case _: Throwable => fail("Should be interrupted")
+ }
+ }
+}
From f5be35e2f50c6ebf64d7914f34fda36772c87729 Mon Sep 17 00:00:00 2001
From: Jason Lowe
Date: Mon, 25 Nov 2024 20:26:45 -0600
Subject: [PATCH 083/103] Fix Kudo batch serializer to only read header in
hasNext (#11766)
Signed-off-by: Jason Lowe
---
.../rapids/GpuColumnarBatchSerializer.scala | 51 +++++++++++--------
1 file changed, 29 insertions(+), 22 deletions(-)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
index 116b8b97504..44a58370c33 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
@@ -27,7 +27,7 @@ import ai.rapids.cudf.JCudfSerialization.SerializedTableHeader
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion
-import com.nvidia.spark.rapids.jni.kudo.{KudoSerializer, KudoTable}
+import com.nvidia.spark.rapids.jni.kudo.{KudoSerializer, KudoTable, KudoTableHeader}
import org.apache.spark.TaskContext
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer, SerializerInstance}
@@ -495,47 +495,52 @@ object KudoSerializedTableColumn {
class KudoSerializedBatchIterator(dIn: DataInputStream)
extends BaseSerializedTableIterator {
- private[this] var nextTable: Option[KudoTable] = None
+ private[this] var nextHeader: Option[KudoTableHeader] = None
private[this] var streamClosed: Boolean = false
// Don't install the callback if in a unit test
Option(TaskContext.get()).foreach { tc =>
onTaskCompletion(tc) {
- nextTable.foreach(_.close())
- nextTable = None
dIn.close()
}
}
- private def tryReadNext(): Unit = {
+ private def tryReadNextHeader(): Unit = {
if (!streamClosed) {
- withResource(new NvtxRange("Read Kudo Table", NvtxColor.YELLOW)) { _ =>
- val kudoTable = KudoTable.from(dIn)
- if (kudoTable.isPresent) {
- nextTable = Some(kudoTable.get())
- } else {
+ withResource(new NvtxRange("Read Kudo Header", NvtxColor.YELLOW)) { _ =>
+ require(nextHeader.isEmpty)
+ nextHeader = Option(KudoTableHeader.readFrom(dIn).orElse(null))
+ if (nextHeader.isEmpty) {
dIn.close()
streamClosed = true
- nextTable = None
}
}
}
}
override def hasNext: Boolean = {
- nextTable match {
- case Some(_) => true
- case None =>
- tryReadNext()
- nextTable.isDefined
+ if (nextHeader.isEmpty) {
+ tryReadNextHeader()
}
+ nextHeader.isDefined
}
override def next(): (Int, ColumnarBatch) = {
if (hasNext) {
- val ret = KudoSerializedTableColumn.from(nextTable.get)
- nextTable = None
- (0, ret)
+ val header = nextHeader.get
+ nextHeader = None
+ val buffer = if (header.getNumColumns == 0) {
+ null
+ } else {
+ withResource(new NvtxRange("Read Kudo Body", NvtxColor.YELLOW)) { _ =>
+ val buffer = HostMemoryBuffer.allocate(header.getTotalDataLen, false)
+ closeOnExcept(buffer) { _ =>
+ buffer.copyFromStream(0, dIn, header.getTotalDataLen)
+ }
+ buffer
+ }
+ }
+ (0, KudoSerializedTableColumn.from(new KudoTable(header, buffer)))
} else {
throw new NoSuchElementException("Walked off of the end...")
}
@@ -547,7 +552,9 @@ class KudoSerializedBatchIterator(dIn: DataInputStream)
* @return the length of the data to read, or None if the stream is closed or ended
*/
override def peekNextBatchSize(): Option[Long] = {
- tryReadNext()
- nextTable.flatMap(t => Option(t.getBuffer)).map(_.getLength)
+ if (nextHeader.isEmpty) {
+ tryReadNextHeader()
+ }
+ nextHeader.map(_.getTotalDataLen)
}
-}
\ No newline at end of file
+}
From 2b6ac118112c973a7848cb4fc7a26ab68797fb4b Mon Sep 17 00:00:00 2001
From: Gera Shegalov
Date: Tue, 26 Nov 2024 06:36:21 -0800
Subject: [PATCH 084/103] Avoid using StringBuffer in single-threaded methods.
(#11759)
Signed-off-by: Gera Shegalov
---
.../org/apache/spark/sql/rapids/test/cpuJsonExpressions.scala | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/test/cpuJsonExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/test/cpuJsonExpressions.scala
index 97d271b076f..0dd048967a8 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/test/cpuJsonExpressions.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/test/cpuJsonExpressions.scala
@@ -203,7 +203,7 @@ object GetJsonObjectMask {
oneToOneMap: Map[Char, Char],
digitMap: Map[Char, Char]): String = {
if (originStr != null) {
- val buf = new StringBuffer(originStr.length)
+ val buf = new StringBuilder(originStr.length)
var idx = 0
while (idx < originStr.length) {
val originChar = originStr(idx)
From e3dce9ec393d84f68c00da6e5631f67abffe94e0 Mon Sep 17 00:00:00 2001
From: Renjie Liu
Date: Tue, 26 Nov 2024 23:11:24 +0800
Subject: [PATCH 085/103] Fix query hang when using rapids multithread shuffle
manager with kudo (#11771)
* Fix query hang when using kudo and multi thread shuffle manager
Signed-off-by: liurenjie1024
* Fix NPE
---------
Signed-off-by: liurenjie1024
---
.../rapids/GpuColumnarBatchSerializer.scala | 94 +++++++++++--------
.../RapidsShuffleInternalManagerBase.scala | 1 -
2 files changed, 53 insertions(+), 42 deletions(-)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
index 44a58370c33..54252253d38 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala
@@ -319,10 +319,12 @@ object SerializedTableColumn {
if (batch.numCols == 1) {
val cv = batch.column(0)
cv match {
- case serializedTableColumn: SerializedTableColumn
- if serializedTableColumn.hostBuffer != null =>
- sum += serializedTableColumn.hostBuffer.getLength
+ case serializedTableColumn: SerializedTableColumn =>
+ sum += Option(serializedTableColumn.hostBuffer).map(_.getLength).getOrElse(0L)
+ case kudo: KudoSerializedTableColumn =>
+ sum += Option(kudo.kudoTable.getBuffer).map(_.getLength).getOrElse(0L)
case _ =>
+ throw new IllegalStateException(s"Unexpected column type: ${cv.getClass}" )
}
}
sum
@@ -496,65 +498,75 @@ object KudoSerializedTableColumn {
class KudoSerializedBatchIterator(dIn: DataInputStream)
extends BaseSerializedTableIterator {
private[this] var nextHeader: Option[KudoTableHeader] = None
+ private[this] var toBeReturned: Option[ColumnarBatch] = None
private[this] var streamClosed: Boolean = false
// Don't install the callback if in a unit test
Option(TaskContext.get()).foreach { tc =>
onTaskCompletion(tc) {
+ toBeReturned.foreach(_.close())
+ toBeReturned = None
dIn.close()
}
}
- private def tryReadNextHeader(): Unit = {
- if (!streamClosed) {
- withResource(new NvtxRange("Read Kudo Header", NvtxColor.YELLOW)) { _ =>
- require(nextHeader.isEmpty)
- nextHeader = Option(KudoTableHeader.readFrom(dIn).orElse(null))
- if (nextHeader.isEmpty) {
- dIn.close()
- streamClosed = true
+ override def peekNextBatchSize(): Option[Long] = {
+ if (streamClosed) {
+ None
+ } else {
+ if (nextHeader.isEmpty) {
+ withResource(new NvtxRange("Read Header", NvtxColor.YELLOW)) { _ =>
+ val header = Option(KudoTableHeader.readFrom(dIn).orElse(null))
+ if (header.isDefined) {
+ nextHeader = header
+ } else {
+ dIn.close()
+ streamClosed = true
+ nextHeader = None
+ }
}
}
+ nextHeader.map(_.getTotalDataLen)
}
}
- override def hasNext: Boolean = {
+ private def tryReadNext(): Option[ColumnarBatch] = {
if (nextHeader.isEmpty) {
- tryReadNextHeader()
- }
- nextHeader.isDefined
- }
-
- override def next(): (Int, ColumnarBatch) = {
- if (hasNext) {
- val header = nextHeader.get
- nextHeader = None
- val buffer = if (header.getNumColumns == 0) {
- null
- } else {
- withResource(new NvtxRange("Read Kudo Body", NvtxColor.YELLOW)) { _ =>
- val buffer = HostMemoryBuffer.allocate(header.getTotalDataLen, false)
- closeOnExcept(buffer) { _ =>
- buffer.copyFromStream(0, dIn, header.getTotalDataLen)
+ None
+ } else {
+ withResource(new NvtxRange("Read Batch", NvtxColor.YELLOW)) { _ =>
+ val header = nextHeader.get
+ if (header.getNumColumns > 0) {
+ // This buffer will later be concatenated into another host buffer before being
+ // sent to the GPU, so no need to use pinned memory for these buffers.
+ closeOnExcept(HostMemoryBuffer.allocate(header.getTotalDataLen, false)) { hostBuffer =>
+ hostBuffer.copyFromStream(0, dIn, header.getTotalDataLen)
+ val kudoTable = new KudoTable(header, hostBuffer)
+ Some(KudoSerializedTableColumn.from(kudoTable))
}
- buffer
+ } else {
+ Some(KudoSerializedTableColumn.from(new KudoTable(header, null)))
}
}
- (0, KudoSerializedTableColumn.from(new KudoTable(header, buffer)))
- } else {
- throw new NoSuchElementException("Walked off of the end...")
}
}
- /**
- * Attempt to read the next header from the stream.
- *
- * @return the length of the data to read, or None if the stream is closed or ended
- */
- override def peekNextBatchSize(): Option[Long] = {
- if (nextHeader.isEmpty) {
- tryReadNextHeader()
+ override def hasNext: Boolean = {
+ peekNextBatchSize()
+ nextHeader.isDefined
+ }
+
+ override def next(): (Int, ColumnarBatch) = {
+ if (toBeReturned.isEmpty) {
+ peekNextBatchSize()
+ toBeReturned = tryReadNext()
+ if (nextHeader.isEmpty || toBeReturned.isEmpty) {
+ throw new NoSuchElementException("Walked off of the end...")
+ }
}
- nextHeader.map(_.getTotalDataLen)
+ val ret = toBeReturned.get
+ toBeReturned = None
+ nextHeader = None
+ (0, ret)
}
}
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala
index a44580c3bf5..05bc76c3fab 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala
@@ -21,7 +21,6 @@ import java.util.Optional
import java.util.concurrent.{Callable, ConcurrentHashMap, ExecutionException, Executors, Future, LinkedBlockingQueue, TimeUnit}
import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
-import scala.collection
import scala.collection.mutable
import scala.collection.mutable.ListBuffer
From 4fa0a1dee986e05733dbdbf4971c42ad5e0e84ec Mon Sep 17 00:00:00 2001
From: "Hongbin Ma (Mahone)"
Date: Tue, 26 Nov 2024 23:44:45 +0800
Subject: [PATCH 086/103] repartition-based fallback for hash aggregate v3
(#11712)
Signed-off-by: Hongbin Ma (Mahone)
Signed-off-by: Firestarman
Co-authored-by: Firestarman
---
.../scala/com/nvidia/spark/rapids/Arm.scala | 16 +-
.../rapids/AutoClosableArrayBuffer.scala | 54 ++
.../spark/rapids/GpuAggregateExec.scala | 725 ++++++++++--------
.../com/nvidia/spark/rapids/GpuExec.scala | 6 +
...GpuUnboundedToUnboundedAggWindowExec.scala | 29 +-
5 files changed, 476 insertions(+), 354 deletions(-)
create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/AutoClosableArrayBuffer.scala
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Arm.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Arm.scala
index 926f770a683..b0cd798c179 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Arm.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Arm.scala
@@ -16,7 +16,7 @@
package com.nvidia.spark.rapids
import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import scala.util.control.ControlThrowable
import com.nvidia.spark.rapids.RapidsPluginImplicits._
@@ -134,6 +134,20 @@ object Arm extends ArmScalaSpecificImpl {
}
}
+ /** Executes the provided code block, closing the resources only if an exception occurs */
+ def closeOnExcept[T <: AutoCloseable, V](r: ListBuffer[T])(block: ListBuffer[T] => V): V = {
+ try {
+ block(r)
+ } catch {
+ case t: ControlThrowable =>
+ // Don't close for these cases..
+ throw t
+ case t: Throwable =>
+ r.safeClose(t)
+ throw t
+ }
+ }
+
/** Executes the provided code block, closing the resources only if an exception occurs */
def closeOnExcept[T <: AutoCloseable, V](r: mutable.Queue[T])(block: mutable.Queue[T] => V): V = {
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AutoClosableArrayBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AutoClosableArrayBuffer.scala
new file mode 100644
index 00000000000..fb1e10b9c9e
--- /dev/null
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AutoClosableArrayBuffer.scala
@@ -0,0 +1,54 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.nvidia.spark.rapids
+
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
+/**
+ * Just a simple wrapper to make working with buffers of AutoClosable things play
+ * nicely with withResource.
+ */
+class AutoClosableArrayBuffer[T <: AutoCloseable] extends AutoCloseable {
+ val data = new ArrayBuffer[T]()
+
+ def append(scb: T): Unit = data.append(scb)
+
+ def last: T = data.last
+
+ def removeLast(): T = data.remove(data.length - 1)
+
+ def foreach[U](f: T => U): Unit = data.foreach(f)
+
+ def map[U](f: T => U): Seq[U] = data.map(f).toSeq
+
+ def toArray[B >: T : ClassTag]: Array[B] = data.toArray
+
+ def size(): Int = data.size
+
+ def clear(): Unit = data.clear()
+
+ def forall(p: T => Boolean): Boolean = data.forall(p)
+
+ def iterator: Iterator[T] = data.iterator
+
+ override def toString: String = s"AutoCloseable(${super.toString})"
+
+ override def close(): Unit = {
+ data.foreach(_.close())
+ data.clear()
+ }
+}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
index b5360a62f94..60f6dd68509 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
@@ -16,11 +16,9 @@
package com.nvidia.spark.rapids
-import java.util
-
import scala.annotation.tailrec
-import scala.collection.JavaConverters.collectionAsScalaIterableConverter
import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
import ai.rapids.cudf
import ai.rapids.cudf.{NvtxColor, NvtxRange}
@@ -37,7 +35,7 @@ import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeReference, AttributeSeq, AttributeSet, Expression, ExprId, If, NamedExpression, NullsFirst, SortOrder}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeReference, AttributeSeq, AttributeSet, Expression, ExprId, If, NamedExpression, SortOrder}
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, HashPartitioning, Partitioning, UnspecifiedDistribution}
@@ -47,11 +45,11 @@ import org.apache.spark.sql.execution.{ExplainUtils, SortExec, SparkPlan}
import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.rapids.aggregate.{CpuToGpuAggregateBufferConverter, CudfAggregate, GpuAggregateExpression, GpuToCpuAggregateBufferConverter}
-import org.apache.spark.sql.rapids.execution.{GpuShuffleMeta, TrampolineUtil}
+import org.apache.spark.sql.rapids.execution.{GpuBatchSubPartitioner, GpuShuffleMeta, TrampolineUtil}
import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.ColumnarBatch
-object AggregateUtils {
+object AggregateUtils extends Logging {
private val aggs = List("min", "max", "avg", "sum", "count", "first", "last")
@@ -98,8 +96,10 @@ object AggregateUtils {
inputTypes: Seq[DataType],
outputTypes: Seq[DataType],
isReductionOnly: Boolean): Long = {
+
def typesToSize(types: Seq[DataType]): Long =
types.map(GpuBatchUtils.estimateGpuMemory(_, nullable = false, rowCount = 1)).sum
+
val inputRowSize = typesToSize(inputTypes)
val outputRowSize = typesToSize(outputTypes)
// The cudf hash table implementation allocates four 32-bit integers per input row.
@@ -120,22 +120,198 @@ object AggregateUtils {
}
// Calculate the max rows that can be processed during computation within the budget
- val maxRows = totalBudget / computationBytesPerRow
+ // Make sure it's not less than 1, otherwise some corner test cases may fail
+ val maxRows = Math.max(totalBudget / computationBytesPerRow, 1)
// Finally compute the input target batching size taking into account the cudf row limits
Math.min(inputRowSize * maxRows, Int.MaxValue)
}
+
+ /**
+ * Concatenate batches together and perform a merge aggregation on the result. The input batches
+ * will be closed as part of this operation.
+ *
+ * @param batches batches to concatenate and merge aggregate
+ * @return lazy spillable batch which has NOT been marked spillable
+ */
+ def concatenateAndMerge(
+ batches: mutable.ArrayBuffer[SpillableColumnarBatch],
+ metrics: GpuHashAggregateMetrics,
+ concatAndMergeHelper: AggHelper): SpillableColumnarBatch = {
+ // TODO: concatenateAndMerge (and calling code) could output a sequence
+ // of batches for the partial aggregate case. This would be done in case
+ // a retry failed a certain number of times.
+ val concatBatch = withResource(batches) { _ =>
+ val concatSpillable = concatenateBatches(metrics, batches.toSeq)
+ withResource(concatSpillable) {
+ _.getColumnarBatch()
+ }
+ }
+ computeAggregateAndClose(metrics, concatBatch, concatAndMergeHelper)
+ }
+
+ /**
+ * Try to concat and merge neighbour input batches to reduce the number of output batches.
+ * For some cases where input is highly aggregate-able, we can merge multiple input batches
+ * into a single output batch. In such cases we can skip repartition at all.
+ */
+ def streamAggregateNeighours(
+ aggregatedBatches: CloseableBufferedIterator[SpillableColumnarBatch],
+ metrics: GpuHashAggregateMetrics,
+ targetMergeBatchSize: Long,
+ configuredTargetBatchSize: Long,
+ helper: AggHelper
+ ): Iterator[SpillableColumnarBatch] = {
+ new Iterator[SpillableColumnarBatch] {
+
+ override def hasNext: Boolean = aggregatedBatches.hasNext
+
+ override def next(): SpillableColumnarBatch = {
+ closeOnExcept(new ArrayBuffer[SpillableColumnarBatch]) { stagingBatches => {
+ var currentSize = 0L
+ while (aggregatedBatches.hasNext) {
+ val nextBatch = aggregatedBatches.head
+ if (currentSize + nextBatch.sizeInBytes > targetMergeBatchSize) {
+ if (stagingBatches.size == 1) {
+ return stagingBatches.head
+ } else if (stagingBatches.isEmpty) {
+ aggregatedBatches.next
+ return nextBatch
+ }
+ val merged = concatenateAndMerge(stagingBatches, metrics, helper)
+ stagingBatches.clear
+ currentSize = 0L
+ if (merged.sizeInBytes < configuredTargetBatchSize * 0.5) {
+ stagingBatches += merged
+ currentSize += merged.sizeInBytes
+ } else {
+ return merged
+ }
+ } else {
+ stagingBatches.append(nextBatch)
+ currentSize += nextBatch.sizeInBytes
+ aggregatedBatches.next
+ }
+ }
+
+ if (stagingBatches.size == 1) {
+ return stagingBatches.head
+ }
+ concatenateAndMerge(stagingBatches, metrics, helper)
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Read the input batches and repartition them into buckets.
+ */
+ def iterateAndRepartition(
+ aggregatedBatches: Iterator[SpillableColumnarBatch],
+ metrics: GpuHashAggregateMetrics,
+ targetMergeBatchSize: Long,
+ helper: AggHelper,
+ hashKeys: Seq[GpuExpression],
+ hashBucketNum: Int,
+ hashSeed: Int,
+ batchesByBucket: ArrayBuffer[AutoClosableArrayBuffer[SpillableColumnarBatch]]
+ ): Boolean = {
+
+ var repartitionHappened = false
+ if (hashSeed > 200) {
+ throw new IllegalStateException("Too many times of repartition, may hit a bug?")
+ }
+
+ def repartitionAndClose(batch: SpillableColumnarBatch): Unit = {
+
+ // OPTIMIZATION
+ if (!aggregatedBatches.hasNext && batchesByBucket.forall(_.size() == 0)) {
+ // If this is the only batch (after merging neighbours) to be repartitioned,
+ // we can just add it to the first bucket and skip repartitioning.
+ // This is a common case when total input size can fit into a single batch.
+ batchesByBucket.head.append(batch)
+ return
+ }
+
+ withResource(new NvtxWithMetrics("agg repartition",
+ NvtxColor.CYAN, metrics.repartitionTime)) { _ =>
+
+ withResource(new GpuBatchSubPartitioner(
+ Seq(batch).map(batch => {
+ withResource(batch) { _ =>
+ batch.getColumnarBatch()
+ }
+ }).iterator,
+ hashKeys, hashBucketNum, hashSeed, "aggRepartition")) {
+ partitioner => {
+ (0 until partitioner.partitionsCount).foreach { id =>
+ closeOnExcept(batchesByBucket) { _ => {
+ val newBatches = partitioner.releaseBatchesByPartition(id)
+ newBatches.foreach { newBatch =>
+ if (newBatch.numRows() > 0) {
+ batchesByBucket(id).append(newBatch)
+ } else {
+ newBatch.safeClose()
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ repartitionHappened = true
+ }
+
+ while (aggregatedBatches.hasNext) {
+ repartitionAndClose(aggregatedBatches.next)
+ }
+
+ // Deal with the over sized buckets
+ def needRepartitionAgain(bucket: AutoClosableArrayBuffer[SpillableColumnarBatch]) = {
+ bucket.map(_.sizeInBytes).sum > targetMergeBatchSize &&
+ bucket.size() != 1 &&
+ !bucket.forall(_.numRows() == 1) // this is for test
+ }
+
+ if (repartitionHappened && batchesByBucket.exists(needRepartitionAgain)) {
+ logDebug("Some of the repartition buckets are over sized, trying to split them")
+
+ val newBuckets = batchesByBucket.flatMap(bucket => {
+ if (needRepartitionAgain(bucket)) {
+ val nextLayerBuckets =
+ ArrayBuffer.fill(hashBucketNum)(new AutoClosableArrayBuffer[SpillableColumnarBatch]())
+ // Recursively merge and repartition the over sized bucket
+ repartitionHappened =
+ iterateAndRepartition(
+ new CloseableBufferedIterator(bucket.iterator), metrics, targetMergeBatchSize,
+ helper, hashKeys, hashBucketNum, hashSeed + 7,
+ nextLayerBuckets) || repartitionHappened
+ nextLayerBuckets
+ } else {
+ ArrayBuffer.apply(bucket)
+ }
+ })
+ batchesByBucket.clear()
+ batchesByBucket.appendAll(newBuckets)
+ }
+
+ repartitionHappened
+ }
}
/** Utility class to hold all of the metrics related to hash aggregation */
case class GpuHashAggregateMetrics(
numOutputRows: GpuMetric,
numOutputBatches: GpuMetric,
- numTasksFallBacked: GpuMetric,
+ numTasksRepartitioned: GpuMetric,
+ numTasksSkippedAgg: GpuMetric,
opTime: GpuMetric,
computeAggTime: GpuMetric,
concatTime: GpuMetric,
sortTime: GpuMetric,
+ repartitionTime: GpuMetric,
numAggOps: GpuMetric,
numPreSplits: GpuMetric,
singlePassTasks: GpuMetric,
@@ -208,7 +384,7 @@ class AggHelper(
private val groupingAttributes = groupingExpressions.map(_.toAttribute)
private val aggBufferAttributes = groupingAttributes ++
- aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
+ aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
// `GpuAggregateFunction` can add a pre and post step for update
// and merge aggregates.
@@ -228,7 +404,7 @@ class AggHelper(
postStep ++= groupingAttributes
postStepAttr ++= groupingAttributes
postStepDataTypes ++=
- groupingExpressions.map(_.dataType)
+ groupingExpressions.map(_.dataType)
private var ix = groupingAttributes.length
for (aggExp <- aggregateExpressions) {
@@ -380,9 +556,9 @@ class AggHelper(
withResource(new NvtxRange("groupby", NvtxColor.BLUE)) { _ =>
withResource(GpuColumnVector.from(preProcessed)) { preProcessedTbl =>
val groupOptions = cudf.GroupByOptions.builder()
- .withIgnoreNullKeys(false)
- .withKeysSorted(doSortAgg)
- .build()
+ .withIgnoreNullKeys(false)
+ .withKeysSorted(doSortAgg)
+ .build()
val cudfAggsOnColumn = cudfAggregates.zip(aggOrdinals).map {
case (cudfAgg, ord) => cudfAgg.groupByAggregate.onColumn(ord)
@@ -390,8 +566,8 @@ class AggHelper(
// perform the aggregate
val aggTbl = preProcessedTbl
- .groupBy(groupOptions, groupingOrdinals: _*)
- .aggregate(cudfAggsOnColumn.toSeq: _*)
+ .groupBy(groupOptions, groupingOrdinals: _*)
+ .aggregate(cudfAggsOnColumn.toSeq: _*)
withResource(aggTbl) { _ =>
GpuColumnVector.from(aggTbl, postStepDataTypes.toArray)
@@ -555,8 +731,8 @@ object GpuAggFirstPassIterator {
metrics: GpuHashAggregateMetrics
): Iterator[SpillableColumnarBatch] = {
val preprocessProjectIter = cbIter.map { cb =>
- val sb = SpillableColumnarBatch (cb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY)
- aggHelper.preStepBound.projectAndCloseWithRetrySingleBatch (sb)
+ val sb = SpillableColumnarBatch(cb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY)
+ aggHelper.preStepBound.projectAndCloseWithRetrySingleBatch(sb)
}
computeAggregateWithoutPreprocessAndClose(metrics, preprocessProjectIter, aggHelper)
}
@@ -597,18 +773,18 @@ object GpuAggFinalPassIterator {
modeInfo: AggregateModeInfo): BoundExpressionsModeAggregates = {
val groupingAttributes = groupingExpressions.map(_.toAttribute)
val aggBufferAttributes = groupingAttributes ++
- aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
+ aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
val boundFinalProjections = if (modeInfo.hasFinalMode || modeInfo.hasCompleteMode) {
val finalProjections = groupingAttributes ++
- aggregateExpressions.map(_.aggregateFunction.evaluateExpression)
+ aggregateExpressions.map(_.aggregateFunction.evaluateExpression)
Some(GpuBindReferences.bindGpuReferences(finalProjections, aggBufferAttributes))
} else {
None
}
// allAttributes can be different things, depending on aggregation mode:
- // - Partial mode: grouping key + cudf aggregates (e.g. no avg, intead sum::count
+ // - Partial mode: grouping key + cudf aggregates (e.g. no avg, instead sum::count
// - Final mode: grouping key + spark aggregates (e.g. avg)
val finalAttributes = groupingAttributes ++ aggregateAttributes
@@ -689,17 +865,22 @@ object GpuAggFinalPassIterator {
/**
* Iterator that takes another columnar batch iterator as input and emits new columnar batches that
* are aggregated based on the specified grouping and aggregation expressions. This iterator tries
- * to perform a hash-based aggregation but is capable of falling back to a sort-based aggregation
- * which can operate on data that is either larger than can be represented by a cudf column or
- * larger than can fit in GPU memory.
+ * to perform a hash-based aggregation but is capable of falling back to a repartition-based
+ * aggregation which can operate on data that is either larger than can be represented by a cudf
+ * column or larger than can fit in GPU memory.
+ *
+ * In general, GpuMergeAggregateIterator works in this flow:
*
- * The iterator starts by pulling all batches from the input iterator, performing an initial
- * projection and aggregation on each individual batch via `aggregateInputBatches()`. The resulting
- * aggregated batches are cached in memory as spillable batches. Once all input batches have been
- * aggregated, `tryMergeAggregatedBatches()` is called to attempt a merge of the aggregated batches
- * into a single batch. If this is successful then the resulting batch can be returned, otherwise
- * `buildSortFallbackIterator` is used to sort the aggregated batches by the grouping keys and
- * performs a final merge aggregation pass on the sorted batches.
+ * (1) The iterator starts by pulling all batches from the input iterator, performing an initial
+ * projection and aggregation on each individual batch via `GpuAggFirstPassIterator`, we call it
+ * "First Pass Aggregate".
+ * (2) Then the batches after first pass agg is sent to "streamAggregateNeighours", where it tries
+ * to concat & merge the neighbour batches into fewer batches, then "iterateAndRepartition"
+ * repartition the batch into fixed size buckets. Recursive repartition will be applied on
+ * over-sized buckets until each bucket * is within the target size.
+ * We call this phase "Second Pass Aggregate".
+ * (3) At "Third Pass Aggregate", we take each bucket and perform a final aggregation on all batches
+ * in the bucket, check "RepartitionAggregateIterator" for details.
*
* @param firstPassIter iterator that has done a first aggregation pass over the input data.
* @param inputAttributes input attributes to identify the input columns from the input batches
@@ -710,13 +891,12 @@ object GpuAggFinalPassIterator {
* @param modeInfo identifies which aggregation modes are being used
* @param metrics metrics that will be updated during aggregation
* @param configuredTargetBatchSize user-specified value for the targeted input batch size
- * @param useTieredProject user-specified option to enable tiered projections
* @param allowNonFullyAggregatedOutput if allowed to skip third pass Agg
* @param skipAggPassReductionRatio skip if the ratio of rows after a pass is bigger than this value
* @param localInputRowsCount metric to track the number of input rows processed locally
*/
class GpuMergeAggregateIterator(
- firstPassIter: Iterator[SpillableColumnarBatch],
+ firstPassIter: CloseableBufferedIterator[SpillableColumnarBatch],
inputAttributes: Seq[Attribute],
groupingExpressions: Seq[NamedExpression],
aggregateExpressions: Seq[GpuAggregateExpression],
@@ -728,18 +908,22 @@ class GpuMergeAggregateIterator(
conf: SQLConf,
allowNonFullyAggregatedOutput: Boolean,
skipAggPassReductionRatio: Double,
- localInputRowsCount: LocalGpuMetric)
- extends Iterator[ColumnarBatch] with AutoCloseable with Logging {
+ localInputRowsCount: LocalGpuMetric
+)
+ extends Iterator[ColumnarBatch] with AutoCloseable with Logging {
private[this] val isReductionOnly = groupingExpressions.isEmpty
private[this] val targetMergeBatchSize = computeTargetMergeBatchSize(configuredTargetBatchSize)
- private[this] val aggregatedBatches = new util.ArrayDeque[SpillableColumnarBatch]
- private[this] var outOfCoreIter: Option[GpuOutOfCoreSortIterator] = None
- /** Iterator for fetching aggregated batches either if:
- * 1. a sort-based fallback has occurred
- * 2. skip third pass agg has occurred
- **/
- private[this] var fallbackIter: Option[Iterator[ColumnarBatch]] = None
+ private[this] val defaultHashBucketNum = 16
+ private[this] val defaultHashSeed = 107
+ private[this] var batchesByBucket =
+ ArrayBuffer.fill(defaultHashBucketNum)(new AutoClosableArrayBuffer[SpillableColumnarBatch]())
+
+ private[this] var firstBatchChecked = false
+
+ private[this] var bucketIter: Option[RepartitionAggregateIterator] = None
+
+ private[this] var realIter: Option[Iterator[ColumnarBatch]] = None
/** Whether a batch is pending for a reduction-only aggregation */
private[this] var hasReductionOnlyBatch: Boolean = isReductionOnly
@@ -752,286 +936,168 @@ class GpuMergeAggregateIterator(
}
override def hasNext: Boolean = {
- fallbackIter.map(_.hasNext).getOrElse {
+ realIter.map(_.hasNext).getOrElse {
// reductions produce a result even if the input is empty
- hasReductionOnlyBatch || !aggregatedBatches.isEmpty || firstPassIter.hasNext
+ hasReductionOnlyBatch || firstPassIter.hasNext
}
}
override def next(): ColumnarBatch = {
- fallbackIter.map(_.next()).getOrElse {
- var shouldSkipThirdPassAgg = false
-
- // aggregate and merge all pending inputs
- if (firstPassIter.hasNext) {
- // first pass agg
- val rowsAfterFirstPassAgg = aggregateInputBatches()
-
- // by now firstPassIter has been traversed, so localInputRowsCount is finished updating
- if (isReductionOnly ||
- skipAggPassReductionRatio * localInputRowsCount.value >= rowsAfterFirstPassAgg) {
- // second pass agg
- tryMergeAggregatedBatches()
-
- val rowsAfterSecondPassAgg = aggregatedBatches.asScala.foldLeft(0L) {
- (totalRows, batch) => totalRows + batch.numRows()
- }
- shouldSkipThirdPassAgg =
- rowsAfterSecondPassAgg > skipAggPassReductionRatio * rowsAfterFirstPassAgg
- } else {
- shouldSkipThirdPassAgg = true
- logInfo(s"Rows after first pass aggregation $rowsAfterFirstPassAgg exceeds " +
- s"${skipAggPassReductionRatio * 100}% of " +
- s"localInputRowsCount ${localInputRowsCount.value}, skip the second pass agg")
- }
- }
+ realIter.map(_.next()).getOrElse {
- if (aggregatedBatches.size() > 1) {
- // Unable to merge to a single output, so must fall back
- if (allowNonFullyAggregatedOutput && shouldSkipThirdPassAgg) {
- // skip third pass agg, return the aggregated batches directly
- logInfo(s"Rows after second pass aggregation exceeds " +
- s"${skipAggPassReductionRatio * 100}% of " +
- s"rows after first pass, skip the third pass agg")
- fallbackIter = Some(new Iterator[ColumnarBatch] {
- override def hasNext: Boolean = !aggregatedBatches.isEmpty
-
- override def next(): ColumnarBatch = {
- withResource(aggregatedBatches.pop()) { spillableBatch =>
- spillableBatch.getColumnarBatch()
- }
- }
- })
- } else {
- // fallback to sort agg, this is the third pass agg
- fallbackIter = Some(buildSortFallbackIterator())
+ // Handle reduction-only aggregation
+ if (isReductionOnly) {
+ val batches = ArrayBuffer.apply[SpillableColumnarBatch]()
+ while (firstPassIter.hasNext) {
+ batches += firstPassIter.next()
}
- fallbackIter.get.next()
- } else if (aggregatedBatches.isEmpty) {
- if (hasReductionOnlyBatch) {
+
+ if (batches.isEmpty || batches.forall(_.numRows() == 0)) {
hasReductionOnlyBatch = false
- generateEmptyReductionBatch()
+ return generateEmptyReductionBatch()
} else {
- throw new NoSuchElementException("batches exhausted")
+ hasReductionOnlyBatch = false
+ val concat = AggregateUtils.concatenateAndMerge(batches, metrics, concatAndMergeHelper)
+ return withResource(concat) { cb =>
+ cb.getColumnarBatch()
+ }
}
- } else {
- // this will be the last batch
- hasReductionOnlyBatch = false
- withResource(aggregatedBatches.pop()) { spillableBatch =>
- spillableBatch.getColumnarBatch()
+ }
+
+ // Handle the case of skipping second and third pass of aggregation
+ // This only work when spark.rapids.sql.agg.skipAggPassReductionRatio < 1
+ if (!firstBatchChecked && firstPassIter.hasNext
+ && allowNonFullyAggregatedOutput) {
+ firstBatchChecked = true
+
+ val peek = firstPassIter.head
+ // It's only based on first batch of first pass agg, so it's an estimate
+ val firstPassReductionRatioEstimate = 1.0 * peek.numRows() / localInputRowsCount.value
+ if (firstPassReductionRatioEstimate > skipAggPassReductionRatio) {
+ logDebug("Skipping second and third pass aggregation due to " +
+ "too high reduction ratio in first pass: " +
+ s"$firstPassReductionRatioEstimate")
+ // if so, skip any aggregation, return the origin batch directly
+
+ realIter = Some(ConcatIterator.apply(firstPassIter, configuredTargetBatchSize))
+ metrics.numTasksSkippedAgg += 1
+ return realIter.get.next()
+ } else {
+ logInfo(s"The reduction ratio in first pass is not high enough to skip " +
+ s"second and third pass aggregation: peek.numRows: ${peek.numRows()}, " +
+ s"localInputRowsCount.value: ${localInputRowsCount.value}")
}
}
+ firstBatchChecked = true
+
+ val groupingAttributes = groupingExpressions.map(_.toAttribute)
+ val aggBufferAttributes = groupingAttributes ++
+ aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
+ val hashKeys: Seq[GpuExpression] =
+ GpuBindReferences.bindGpuReferences(groupingAttributes, aggBufferAttributes.toSeq)
+
+ val repartitionHappened = AggregateUtils.iterateAndRepartition(
+ AggregateUtils.streamAggregateNeighours(
+ firstPassIter,
+ metrics,
+ targetMergeBatchSize,
+ configuredTargetBatchSize,
+ concatAndMergeHelper)
+ , metrics, targetMergeBatchSize, concatAndMergeHelper,
+ hashKeys, defaultHashBucketNum, defaultHashSeed, batchesByBucket)
+ if (repartitionHappened) {
+ metrics.numTasksRepartitioned += 1
+ }
+
+ realIter = Some(ConcatIterator.apply(
+ new CloseableBufferedIterator(buildBucketIterator()), configuredTargetBatchSize))
+ realIter.get.next()
}
}
override def close(): Unit = {
- aggregatedBatches.forEach(_.safeClose())
- aggregatedBatches.clear()
- outOfCoreIter.foreach(_.close())
- outOfCoreIter = None
- fallbackIter = None
+ batchesByBucket.foreach(_.close())
+ batchesByBucket.clear()
hasReductionOnlyBatch = false
}
private def computeTargetMergeBatchSize(confTargetSize: Long): Long = {
val mergedTypes = groupingExpressions.map(_.dataType) ++ aggregateExpressions.map(_.dataType)
- AggregateUtils.computeTargetBatchSize(confTargetSize, mergedTypes, mergedTypes,isReductionOnly)
+ AggregateUtils.computeTargetBatchSize(confTargetSize, mergedTypes, mergedTypes, isReductionOnly)
}
- /** Aggregate all input batches and place the results in the aggregatedBatches queue. */
- private def aggregateInputBatches(): Long = {
- var rowsAfter = 0L
- // cache everything in the first pass
- while (firstPassIter.hasNext) {
- val batch = firstPassIter.next()
- rowsAfter += batch.numRows()
- aggregatedBatches.add(batch)
- }
- rowsAfter
- }
+ private lazy val concatAndMergeHelper =
+ new AggHelper(inputAttributes, groupingExpressions, aggregateExpressions,
+ forceMerge = true, conf, isSorted = false)
+
+ private case class ConcatIterator(
+ input: CloseableBufferedIterator[SpillableColumnarBatch],
+ targetSize: Long)
+ extends Iterator[ColumnarBatch] {
+
+ override def hasNext: Boolean = input.hasNext
+
+ override def next(): ColumnarBatch = {
+ // combine all the data into a single batch
+ val spillCbs = ArrayBuffer[SpillableColumnarBatch]()
+ var totalBytes = 0L
+ closeOnExcept(spillCbs) { _ =>
+ while (input.hasNext && (spillCbs.isEmpty ||
+ (totalBytes + input.head.sizeInBytes) < targetSize)) {
+ val tmp = input.next
+ totalBytes += tmp.sizeInBytes
+ spillCbs += tmp
+ }
- /**
- * Attempt to merge adjacent batches in the aggregatedBatches queue until either there is only
- * one batch or merging adjacent batches would exceed the target batch size.
- */
- private def tryMergeAggregatedBatches(): Unit = {
- while (aggregatedBatches.size() > 1) {
- val concatTime = metrics.concatTime
- val opTime = metrics.opTime
- withResource(new NvtxWithMetrics("agg merge pass", NvtxColor.BLUE, concatTime,
- opTime)) { _ =>
- // continue merging as long as some batches are able to be combined
- if (!mergePass()) {
- if (aggregatedBatches.size() > 1 && isReductionOnly) {
- // We were unable to merge the aggregated batches within the target batch size limit,
- // which means normally we would fallback to a sort-based approach. However for
- // reduction-only aggregation there are no keys to use for a sort. The only way this
- // can work is if all batches are merged. This will exceed the target batch size limit,
- // but at this point it is either risk an OOM/cudf error and potentially work or
- // not work at all.
- logWarning(s"Unable to merge reduction-only aggregated batches within " +
- s"target batch limit of $targetMergeBatchSize, attempting to merge remaining " +
- s"${aggregatedBatches.size()} batches beyond limit")
- withResource(mutable.ArrayBuffer[SpillableColumnarBatch]()) { batchesToConcat =>
- aggregatedBatches.forEach(b => batchesToConcat += b)
- aggregatedBatches.clear()
- val batch = concatenateAndMerge(batchesToConcat)
- // batch does not need to be marked spillable since it is the last and only batch
- // and will be immediately retrieved on the next() call.
- aggregatedBatches.add(batch)
- }
- }
- return
+ val concat = GpuAggregateIterator.concatenateBatches(metrics, spillCbs.toSeq)
+ withResource(concat) { _ =>
+ concat.getColumnarBatch()
}
}
}
}
- /**
- * Perform a single pass over the aggregated batches attempting to merge adjacent batches.
- * @return true if at least one merge operation occurred
- */
- private def mergePass(): Boolean = {
- val batchesToConcat: mutable.ArrayBuffer[SpillableColumnarBatch] = mutable.ArrayBuffer.empty
- var wasBatchMerged = false
- // Current size in bytes of the batches targeted for the next concatenation
- var concatSize: Long = 0L
- var batchesLeftInPass = aggregatedBatches.size()
-
- while (batchesLeftInPass > 0) {
- closeOnExcept(batchesToConcat) { _ =>
- var isConcatSearchFinished = false
- // Old batches are picked up at the front of the queue and freshly merged batches are
- // appended to the back of the queue. Although tempting to allow the pass to "wrap around"
- // and pick up batches freshly merged in this pass, it's avoided to prevent changing the
- // order of aggregated batches.
- while (batchesLeftInPass > 0 && !isConcatSearchFinished) {
- val candidate = aggregatedBatches.getFirst
- val potentialSize = concatSize + candidate.sizeInBytes
- isConcatSearchFinished = concatSize > 0 && potentialSize > targetMergeBatchSize
- if (!isConcatSearchFinished) {
- batchesLeftInPass -= 1
- batchesToConcat += aggregatedBatches.removeFirst()
- concatSize = potentialSize
- }
- }
- }
+ private case class RepartitionAggregateIterator(opTime: GpuMetric)
+ extends Iterator[SpillableColumnarBatch] {
- val mergedBatch = if (batchesToConcat.length > 1) {
- wasBatchMerged = true
- concatenateAndMerge(batchesToConcat)
- } else {
- // Unable to find a neighboring buffer to produce a valid merge in this pass,
- // so simply put this buffer back on the queue for other passes.
- batchesToConcat.remove(0)
- }
+ batchesByBucket = batchesByBucket.filter(_.size() > 0)
- // Add the merged batch to the end of the aggregated batch queue. Only a single pass over
- // the batches is being performed due to the batch count check above, so the single-pass
- // loop will terminate before picking up this new batch.
- aggregatedBatches.addLast(mergedBatch)
- batchesToConcat.clear()
- concatSize = 0
- }
+ override def hasNext: Boolean = batchesByBucket.nonEmpty
- wasBatchMerged
- }
+ override def next(): SpillableColumnarBatch = {
+ withResource(new NvtxWithMetrics("RepartitionAggregateIterator.next",
+ NvtxColor.BLUE, opTime)) { _ =>
- private lazy val concatAndMergeHelper =
- new AggHelper(inputAttributes, groupingExpressions, aggregateExpressions,
- forceMerge = true, conf = conf)
-
- /**
- * Concatenate batches together and perform a merge aggregation on the result. The input batches
- * will be closed as part of this operation.
- * @param batches batches to concatenate and merge aggregate
- * @return lazy spillable batch which has NOT been marked spillable
- */
- private def concatenateAndMerge(
- batches: mutable.ArrayBuffer[SpillableColumnarBatch]): SpillableColumnarBatch = {
- // TODO: concatenateAndMerge (and calling code) could output a sequence
- // of batches for the partial aggregate case. This would be done in case
- // a retry failed a certain number of times.
- val concatBatch = withResource(batches) { _ =>
- val concatSpillable = concatenateBatches(metrics, batches.toSeq)
- withResource(concatSpillable) { _.getColumnarBatch() }
- }
- computeAggregateAndClose(metrics, concatBatch, concatAndMergeHelper)
- }
-
- /** Build an iterator that uses a sort-based approach to merge aggregated batches together. */
- private def buildSortFallbackIterator(): Iterator[ColumnarBatch] = {
- logInfo(s"Falling back to sort-based aggregation with ${aggregatedBatches.size()} batches")
- metrics.numTasksFallBacked += 1
- val aggregatedBatchIter = new Iterator[ColumnarBatch] {
- override def hasNext: Boolean = !aggregatedBatches.isEmpty
+ if (batchesByBucket.last.size() == 1) {
+ batchesByBucket.remove(batchesByBucket.size - 1).removeLast()
+ } else {
+ // put as many buckets as possible together to aggregate, to reduce agg times
+ closeOnExcept(new ArrayBuffer[AutoClosableArrayBuffer[SpillableColumnarBatch]]) {
+ toAggregateBuckets =>
+ var currentSize = 0L
+ while (batchesByBucket.nonEmpty &&
+ batchesByBucket.last.size() + currentSize < targetMergeBatchSize) {
+ val bucket = batchesByBucket.remove(batchesByBucket.size - 1)
+ currentSize += bucket.map(_.sizeInBytes).sum
+ toAggregateBuckets += bucket
+ }
- override def next(): ColumnarBatch = {
- withResource(aggregatedBatches.removeFirst()) { spillable =>
- spillable.getColumnarBatch()
+ AggregateUtils.concatenateAndMerge(
+ toAggregateBuckets.flatMap(_.data), metrics, concatAndMergeHelper)
+ }
}
}
}
+ }
- if (isReductionOnly) {
- // Normally this should never happen because `tryMergeAggregatedBatches` should have done
- // a last-ditch effort to concatenate all batches together regardless of target limits.
- throw new IllegalStateException("Unable to fallback to sort-based aggregation " +
- "without grouping keys")
- }
-
- val groupingAttributes = groupingExpressions.map(_.toAttribute)
- val ordering = groupingAttributes.map(SortOrder(_, Ascending, NullsFirst, Seq.empty))
- val aggBufferAttributes = groupingAttributes ++
- aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
- val sorter = new GpuSorter(ordering, aggBufferAttributes)
- val aggBatchTypes = aggBufferAttributes.map(_.dataType)
- // Use the out of core sort iterator to sort the batches by grouping key
- outOfCoreIter = Some(GpuOutOfCoreSortIterator(
- aggregatedBatchIter,
- sorter,
- configuredTargetBatchSize,
- opTime = metrics.opTime,
- sortTime = metrics.sortTime,
- outputBatches = NoopMetric,
- outputRows = NoopMetric))
-
- // The out of core sort iterator does not guarantee that a batch contains all of the values
- // for a particular key, so add a key batching iterator to enforce this. That allows each batch
- // to be merge-aggregated safely since all values associated with a particular key are
- // guaranteed to be in the same batch.
- val keyBatchingIter = new GpuKeyBatchingIterator(
- outOfCoreIter.get,
- sorter,
- aggBatchTypes.toArray,
- configuredTargetBatchSize,
- numInputRows = NoopMetric,
- numInputBatches = NoopMetric,
- numOutputRows = NoopMetric,
- numOutputBatches = NoopMetric,
- concatTime = metrics.concatTime,
- opTime = metrics.opTime)
-
- // Finally wrap the key batching iterator with a merge aggregation on the output batches.
- new Iterator[ColumnarBatch] {
- override def hasNext: Boolean = keyBatchingIter.hasNext
-
- private val mergeSortedHelper =
- new AggHelper(inputAttributes, groupingExpressions, aggregateExpressions,
- forceMerge = true, conf, isSorted = true)
-
- override def next(): ColumnarBatch = {
- // batches coming out of the sort need to be merged
- val resultSpillable =
- computeAggregateAndClose(metrics, keyBatchingIter.next(), mergeSortedHelper)
- withResource(resultSpillable) { _ =>
- resultSpillable.getColumnarBatch()
- }
- }
- }
+ /** Build an iterator merging aggregated batches in each bucket. */
+ private def buildBucketIterator(): Iterator[SpillableColumnarBatch] = {
+ bucketIter = Some(RepartitionAggregateIterator(opTime = metrics.opTime))
+ bucketIter.get
}
+
/**
* Generates the result of a reduction-only aggregation on empty input by emitting the
* initial value of each aggregator.
@@ -1117,13 +1183,13 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan](
)
if (arrayWithStructsGroupings) {
willNotWorkOnGpu("ArrayTypes with Struct children in grouping expressions are not " +
- "supported")
+ "supported")
}
tagForReplaceMode()
if (agg.aggregateExpressions.exists(expr => expr.isDistinct)
- && agg.aggregateExpressions.exists(expr => expr.filter.isDefined)) {
+ && agg.aggregateExpressions.exists(expr => expr.filter.isDefined)) {
// Distinct with Filter is not supported on the GPU currently,
// This makes sure that if we end up here, the plan falls back to the CPU
// which will do the right thing.
@@ -1195,15 +1261,15 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan](
// (due to First). Fall back to CPU in this case.
if (AggregateUtils.shouldFallbackMultiDistinct(agg.aggregateExpressions)) {
willNotWorkOnGpu("Aggregates of non-distinct functions with multiple distinct " +
- "functions are non-deterministic for non-distinct functions as it is " +
- "computed using First.")
+ "functions are non-deterministic for non-distinct functions as it is " +
+ "computed using First.")
}
}
}
if (!conf.partialMergeDistinctEnabled && aggPattern.contains(PartialMerge)) {
willNotWorkOnGpu("Replacing Partial Merge aggregates disabled. " +
- s"Set ${conf.partialMergeDistinctEnabled} to true if desired")
+ s"Set ${conf.partialMergeDistinctEnabled} to true if desired")
}
}
@@ -1256,11 +1322,11 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan](
// This is a short term heuristic until we can better understand the cost
// of sort vs the cost of doing aggregations so we can better decide.
lazy val hasSingleBasicGroupingKey = agg.groupingExpressions.length == 1 &&
- agg.groupingExpressions.headOption.map(_.dataType).exists {
- case StringType | BooleanType | ByteType | ShortType | IntegerType |
- LongType | _: DecimalType | DateType | TimestampType => true
- case _ => false
- }
+ agg.groupingExpressions.headOption.map(_.dataType).exists {
+ case StringType | BooleanType | ByteType | ShortType | IntegerType |
+ LongType | _: DecimalType | DateType | TimestampType => true
+ case _ => false
+ }
val gpuChild = childPlans.head.convertIfNeeded()
val gpuAggregateExpressions =
@@ -1314,11 +1380,11 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan](
}
val allowSinglePassAgg = (conf.forceSinglePassPartialSortAgg ||
- (conf.allowSinglePassPartialSortAgg &&
- hasSingleBasicGroupingKey &&
- estimatedPreProcessGrowth > 1.1)) &&
- canUsePartialSortAgg &&
- groupingCanBeSorted
+ (conf.allowSinglePassPartialSortAgg &&
+ hasSingleBasicGroupingKey &&
+ estimatedPreProcessGrowth > 1.1)) &&
+ canUsePartialSortAgg &&
+ groupingCanBeSorted
GpuHashAggregateExec(
aggRequiredChildDistributionExpressions,
@@ -1332,7 +1398,8 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan](
conf.forceSinglePassPartialSortAgg,
allowSinglePassAgg,
allowNonFullyAggregatedOutput,
- conf.skipAggPassReductionRatio)
+ conf.skipAggPassReductionRatio
+ )
}
}
@@ -1351,7 +1418,7 @@ abstract class GpuTypedImperativeSupportedAggregateExecMeta[INPUT <: BaseAggrega
private val mayNeedAggBufferConversion: Boolean =
agg.aggregateExpressions.exists { expr =>
expr.aggregateFunction.isInstanceOf[TypedImperativeAggregate[_]] &&
- (expr.mode == Partial || expr.mode == PartialMerge)
+ (expr.mode == Partial || expr.mode == PartialMerge)
}
// overriding data types of Aggregation Buffers if necessary
@@ -1420,6 +1487,7 @@ abstract class GpuTypedImperativeSupportedAggregateExecMeta[INPUT <: BaseAggrega
allowSinglePassAgg = false,
allowNonFullyAggregatedOutput = false,
1)
+
} else {
super.convertToGpu()
}
@@ -1523,8 +1591,8 @@ object GpuTypedImperativeSupportedAggregateExecMeta {
// [A]. there will be a R2C or C2R transition between them
// [B]. there exists TypedImperativeAggregate functions in each of them
(stages(i).canThisBeReplaced ^ stages(i + 1).canThisBeReplaced) &&
- containTypedImperativeAggregate(stages(i)) &&
- containTypedImperativeAggregate(stages(i + 1))
+ containTypedImperativeAggregate(stages(i)) &&
+ containTypedImperativeAggregate(stages(i + 1))
}
// Return if all internal aggregation buffers are compatible with GPU Overrides.
@@ -1602,10 +1670,10 @@ object GpuTypedImperativeSupportedAggregateExecMeta {
fromCpuToGpu: Boolean): Seq[NamedExpression] = {
val converters = mutable.Queue[Either[
- CpuToGpuAggregateBufferConverter, GpuToCpuAggregateBufferConverter]]()
+ CpuToGpuAggregateBufferConverter, GpuToCpuAggregateBufferConverter]]()
mergeAggMeta.childExprs.foreach {
case e if e.childExprs.length == 1 &&
- e.childExprs.head.isInstanceOf[TypedImperativeAggExprMeta[_]] =>
+ e.childExprs.head.isInstanceOf[TypedImperativeAggExprMeta[_]] =>
e.wrapped.asInstanceOf[AggregateExpression].mode match {
case Final | PartialMerge =>
val typImpAggMeta = e.childExprs.head.asInstanceOf[TypedImperativeAggExprMeta[_]]
@@ -1660,16 +1728,16 @@ class GpuHashAggregateMeta(
conf: RapidsConf,
parent: Option[RapidsMeta[_, _, _]],
rule: DataFromReplacementRule)
- extends GpuBaseAggregateMeta(agg, agg.requiredChildDistributionExpressions,
- conf, parent, rule)
+ extends GpuBaseAggregateMeta(agg, agg.requiredChildDistributionExpressions,
+ conf, parent, rule)
class GpuSortAggregateExecMeta(
override val agg: SortAggregateExec,
conf: RapidsConf,
parent: Option[RapidsMeta[_, _, _]],
rule: DataFromReplacementRule)
- extends GpuTypedImperativeSupportedAggregateExecMeta(agg,
- agg.requiredChildDistributionExpressions, conf, parent, rule) {
+ extends GpuTypedImperativeSupportedAggregateExecMeta(agg,
+ agg.requiredChildDistributionExpressions, conf, parent, rule) {
override def tagPlanForGpu(): Unit = {
super.tagPlanForGpu()
@@ -1716,14 +1784,14 @@ class GpuObjectHashAggregateExecMeta(
conf: RapidsConf,
parent: Option[RapidsMeta[_, _, _]],
rule: DataFromReplacementRule)
- extends GpuTypedImperativeSupportedAggregateExecMeta(agg,
- agg.requiredChildDistributionExpressions, conf, parent, rule)
+ extends GpuTypedImperativeSupportedAggregateExecMeta(agg,
+ agg.requiredChildDistributionExpressions, conf, parent, rule)
object GpuHashAggregateExecBase {
def calcInputAttributes(aggregateExpressions: Seq[GpuAggregateExpression],
- childOutput: Seq[Attribute],
- inputAggBufferAttributes: Seq[Attribute]): Seq[Attribute] = {
+ childOutput: Seq[Attribute],
+ inputAggBufferAttributes: Seq[Attribute]): Seq[Attribute] = {
val modes = aggregateExpressions.map(_.mode).distinct
if (modes.contains(Final) || modes.contains(PartialMerge)) {
// SPARK-31620: when planning aggregates, the partial aggregate uses aggregate function's
@@ -1754,7 +1822,7 @@ object GpuHashAggregateExecBase {
}
/**
- * The GPU version of SortAggregateExec that is intended for partial aggregations that are not
+ * The GPU version of AggregateExec that is intended for partial aggregations that are not
* reductions and so it sorts the input data ahead of time to do it in a single pass.
*
* @param requiredChildDistributionExpressions this is unchanged by the GPU. It is used in
@@ -1767,7 +1835,6 @@ object GpuHashAggregateExecBase {
* node should project)
* @param child incoming plan (where we get input columns from)
* @param configuredTargetBatchSize user-configured maximum device memory size of a batch
- * @param configuredTieredProjectEnabled configurable optimization to use tiered projections
* @param allowNonFullyAggregatedOutput whether we can skip the third pass of aggregation
* (can omit non fully aggregated data for non-final
* stage of aggregation)
@@ -1802,11 +1869,13 @@ case class GpuHashAggregateExec(
protected override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL
protected override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL
override lazy val additionalMetrics: Map[String, GpuMetric] = Map(
- NUM_TASKS_FALL_BACKED -> createMetric(MODERATE_LEVEL, DESCRIPTION_NUM_TASKS_FALL_BACKED),
+ NUM_TASKS_REPARTITIONED -> createMetric(MODERATE_LEVEL, DESCRIPTION_NUM_TASKS_REPARTITIONED),
+ NUM_TASKS_SKIPPED_AGG -> createMetric(MODERATE_LEVEL, DESCRIPTION_NUM_TASKS_SKIPPED_AGG),
OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME),
AGG_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_AGG_TIME),
CONCAT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_CONCAT_TIME),
SORT_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_SORT_TIME),
+ REPARTITION_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_REPARTITION_TIME),
"NUM_AGGS" -> createMetric(DEBUG_LEVEL, "num agg operations"),
"NUM_PRE_SPLITS" -> createMetric(DEBUG_LEVEL, "num pre splits"),
"NUM_TASKS_SINGLE_PASS" -> createMetric(MODERATE_LEVEL, "number of single pass tasks"),
@@ -1833,11 +1902,13 @@ case class GpuHashAggregateExec(
val aggMetrics = GpuHashAggregateMetrics(
numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS),
numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES),
- numTasksFallBacked = gpuLongMetric(NUM_TASKS_FALL_BACKED),
+ numTasksRepartitioned = gpuLongMetric(NUM_TASKS_REPARTITIONED),
+ numTasksSkippedAgg = gpuLongMetric(NUM_TASKS_SKIPPED_AGG),
opTime = gpuLongMetric(OP_TIME),
computeAggTime = gpuLongMetric(AGG_TIME),
concatTime = gpuLongMetric(CONCAT_TIME),
sortTime = gpuLongMetric(SORT_TIME),
+ repartitionTime = gpuLongMetric(REPARTITION_TIME),
numAggOps = gpuLongMetric("NUM_AGGS"),
numPreSplits = gpuLongMetric("NUM_PRE_SPLITS"),
singlePassTasks = gpuLongMetric("NUM_TASKS_SINGLE_PASS"),
@@ -1867,11 +1938,12 @@ case class GpuHashAggregateExec(
val postBoundReferences = GpuAggFinalPassIterator.setupReferences(groupingExprs,
aggregateExprs, aggregateAttrs, resultExprs, modeInfo)
- new DynamicGpuPartialSortAggregateIterator(cbIter, inputAttrs, groupingExprs,
+ new DynamicGpuPartialAggregateIterator(cbIter, inputAttrs, groupingExprs,
boundGroupExprs, aggregateExprs, aggregateAttrs, resultExprs, modeInfo,
localEstimatedPreProcessGrowth, alreadySorted, expectedOrdering,
postBoundReferences, targetBatchSize, aggMetrics, conf,
- localForcePre, localAllowPre, allowNonFullyAggregatedOutput, skipAggPassReductionRatio)
+ localForcePre, localAllowPre, allowNonFullyAggregatedOutput, skipAggPassReductionRatio
+ )
}
}
@@ -1914,8 +1986,8 @@ case class GpuHashAggregateExec(
// Used in de-duping and optimizer rules
override def producedAttributes: AttributeSet =
AttributeSet(aggregateAttributes) ++
- AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++
- AttributeSet(aggregateBufferAttributes)
+ AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++
+ AttributeSet(aggregateBufferAttributes)
// AllTuples = distribution with a single partition and all tuples of the dataset are co-located.
// Clustered = dataset with tuples co-located in the same partition if they share a specific value
@@ -1938,7 +2010,7 @@ case class GpuHashAggregateExec(
*/
override lazy val allAttributes: AttributeSeq =
child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++
- aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
+ aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)
override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields)
@@ -1957,8 +2029,8 @@ case class GpuHashAggregateExec(
s"""${loreArgs.mkString(", ")}"""
} else {
s"$nodeName (keys=$keyString, functions=$functionString)," +
- s" filters=${aggregateExpressions.map(_.filter)})" +
- s""" ${loreArgs.mkString(", ")}"""
+ s" filters=${aggregateExpressions.map(_.filter)})" +
+ s""" ${loreArgs.mkString(", ")}"""
}
}
//
@@ -1972,7 +2044,7 @@ case class GpuHashAggregateExec(
}
}
-class DynamicGpuPartialSortAggregateIterator(
+class DynamicGpuPartialAggregateIterator(
cbIter: Iterator[ColumnarBatch],
inputAttrs: Seq[Attribute],
groupingExprs: Seq[NamedExpression],
@@ -1999,7 +2071,7 @@ class DynamicGpuPartialSortAggregateIterator(
// When doing a reduction we don't have the aggIter setup for the very first time
// so we have to match what happens for the normal reduction operations.
override def hasNext: Boolean = aggIter.map(_.hasNext)
- .getOrElse(isReductionOnly || cbIter.hasNext)
+ .getOrElse(isReductionOnly || cbIter.hasNext)
private[this] def estimateCardinality(cb: ColumnarBatch): Int = {
withResource(boundGroupExprs.project(cb)) { groupingKeys =>
@@ -2052,7 +2124,8 @@ class DynamicGpuPartialSortAggregateIterator(
inputAttrs.map(_.dataType).toArray, preProcessAggHelper.preStepBound,
metrics.opTime, metrics.numPreSplits)
- val firstPassIter = GpuAggFirstPassIterator(sortedSplitIter, preProcessAggHelper, metrics)
+ val firstPassIter = GpuAggFirstPassIterator(sortedSplitIter, preProcessAggHelper,
+ metrics)
// Technically on a partial-agg, which this only works for, this last iterator should
// be a noop except for some metrics. But for consistency between all of the
@@ -2071,6 +2144,7 @@ class DynamicGpuPartialSortAggregateIterator(
metrics.opTime, metrics.numPreSplits)
val localInputRowsMetrics = new LocalGpuMetric
+
val firstPassIter = GpuAggFirstPassIterator(
splitInputIter.map(cb => {
localInputRowsMetrics += cb.numRows()
@@ -2080,7 +2154,7 @@ class DynamicGpuPartialSortAggregateIterator(
metrics)
val mergeIter = new GpuMergeAggregateIterator(
- firstPassIter,
+ new CloseableBufferedIterator(firstPassIter),
inputAttrs,
groupingExprs,
aggregateExprs,
@@ -2092,7 +2166,8 @@ class DynamicGpuPartialSortAggregateIterator(
conf,
allowNonFullyAggregatedOutput,
skipAggPassReductionRatio,
- localInputRowsMetrics)
+ localInputRowsMetrics
+ )
GpuAggFinalPassIterator.makeIter(mergeIter, postBoundReferences, metrics)
}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala
index 0ffead09de6..3d9b6285a91 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala
@@ -66,6 +66,7 @@ object GpuMetric extends Logging {
val COLLECT_TIME = "collectTime"
val CONCAT_TIME = "concatTime"
val SORT_TIME = "sortTime"
+ val REPARTITION_TIME = "repartitionTime"
val AGG_TIME = "computeAggTime"
val JOIN_TIME = "joinTime"
val FILTER_TIME = "filterTime"
@@ -73,6 +74,8 @@ object GpuMetric extends Logging {
val BUILD_TIME = "buildTime"
val STREAM_TIME = "streamTime"
val NUM_TASKS_FALL_BACKED = "numTasksFallBacked"
+ val NUM_TASKS_REPARTITIONED = "numTasksRepartitioned"
+ val NUM_TASKS_SKIPPED_AGG = "numTasksSkippedAgg"
val READ_FS_TIME = "readFsTime"
val WRITE_BUFFER_TIME = "writeBufferTime"
val FILECACHE_FOOTER_HITS = "filecacheFooterHits"
@@ -104,6 +107,7 @@ object GpuMetric extends Logging {
val DESCRIPTION_COLLECT_TIME = "collect batch time"
val DESCRIPTION_CONCAT_TIME = "concat batch time"
val DESCRIPTION_SORT_TIME = "sort time"
+ val DESCRIPTION_REPARTITION_TIME = "repartition time"
val DESCRIPTION_AGG_TIME = "aggregation time"
val DESCRIPTION_JOIN_TIME = "join time"
val DESCRIPTION_FILTER_TIME = "filter time"
@@ -111,6 +115,8 @@ object GpuMetric extends Logging {
val DESCRIPTION_BUILD_TIME = "build time"
val DESCRIPTION_STREAM_TIME = "stream time"
val DESCRIPTION_NUM_TASKS_FALL_BACKED = "number of sort fallback tasks"
+ val DESCRIPTION_NUM_TASKS_REPARTITIONED = "number of tasks repartitioned for agg"
+ val DESCRIPTION_NUM_TASKS_SKIPPED_AGG = "number of tasks skipped aggregation"
val DESCRIPTION_READ_FS_TIME = "time to read fs data"
val DESCRIPTION_WRITE_BUFFER_TIME = "time to write data to buffer"
val DESCRIPTION_FILECACHE_FOOTER_HITS = "cached footer hits"
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/window/GpuUnboundedToUnboundedAggWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/window/GpuUnboundedToUnboundedAggWindowExec.scala
index d685efe68e0..7c5b55cd0bd 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/window/GpuUnboundedToUnboundedAggWindowExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/window/GpuUnboundedToUnboundedAggWindowExec.scala
@@ -17,10 +17,9 @@
package com.nvidia.spark.rapids.window
import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-import scala.reflect.ClassTag
import ai.rapids.cudf
-import com.nvidia.spark.rapids.{ConcatAndConsumeAll, GpuAlias, GpuBindReferences, GpuBoundReference, GpuColumnVector, GpuExpression, GpuLiteral, GpuMetric, GpuProjectExec, SpillableColumnarBatch, SpillPriorities}
+import com.nvidia.spark.rapids.{AutoClosableArrayBuffer, ConcatAndConsumeAll, GpuAlias, GpuBindReferences, GpuBoundReference, GpuColumnVector, GpuExpression, GpuLiteral, GpuMetric, GpuProjectExec, SpillableColumnarBatch, SpillPriorities}
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq
import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRetry, withRetryNoSplit}
@@ -36,32 +35,6 @@ import org.apache.spark.sql.rapids.aggregate.{CudfAggregate, GpuAggregateExpress
import org.apache.spark.sql.types.{DataType, IntegerType, LongType}
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
-
-/**
- * Just a simple wrapper to make working with buffers of AutoClosable things play
- * nicely with withResource.
- */
-class AutoClosableArrayBuffer[T <: AutoCloseable]() extends AutoCloseable {
- private val data = new ArrayBuffer[T]()
-
- def append(scb: T): Unit = data.append(scb)
-
- def last: T = data.last
-
- def removeLast(): T = data.remove(data.length - 1)
-
- def foreach[U](f: T => U): Unit = data.foreach(f)
-
- def toArray[B >: T : ClassTag]: Array[B] = data.toArray
-
- override def toString: String = s"AutoCloseable(${super.toString})"
-
- override def close(): Unit = {
- data.foreach(_.close())
- data.clear()
- }
-}
-
/**
* Utilities for conversion between SpillableColumnarBatch, ColumnarBatch, and cudf.Table.
*/
From 82c26f1de2cbdb13fa0d9e041baa4b738ca85d5e Mon Sep 17 00:00:00 2001
From: knoguchi22
Date: Tue, 26 Nov 2024 13:53:03 -0500
Subject: [PATCH 087/103] Append knoguchi22 to blossom-ci whitelist [skip ci]
(#11777)
* Append knoguchi to blossom-ci whitelist [skip ci]
* Fixing the typo in username.
Signed-off-by: Koji Noguchi
---------
Signed-off-by: Koji Noguchi
---
.github/workflows/blossom-ci.yml | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml
index 93557017b08..1d7b0ab8e0b 100644
--- a/.github/workflows/blossom-ci.yml
+++ b/.github/workflows/blossom-ci.yml
@@ -77,7 +77,8 @@ jobs:
github.actor == 'Feng-Jiang28' ||
github.actor == 'SurajAralihalli' ||
github.actor == 'jihoonson' ||
- github.actor == 'ustcfy'
+ github.actor == 'ustcfy' ||
+ github.actor == 'knoguchi22'
)
steps:
- name: Check if comment is issued by authorized person
From ff0ca0f4d52b197a644c332b1ffcbe9c0351fb1f Mon Sep 17 00:00:00 2001
From: Jason Lowe
Date: Tue, 26 Nov 2024 15:07:05 -0600
Subject: [PATCH 088/103] Ability to decompress snappy and zstd Parquet files
via CPU [databricks] (#11752)
* Ability to decompress Parquet data on CPU
Signed-off-by: Jason Lowe
* Add tests
* Refactor to reduce duplicated code
* scala2.13 fix
* Address review comments
* Fix Databricks build
* Update scala2.13 poms
---------
Signed-off-by: Jason Lowe
---
.../src/main/python/parquet_test.py | 9 +-
jenkins/databricks/install_deps.py | 2 +
scala2.13/shim-deps/databricks/pom.xml | 6 +
shim-deps/databricks/pom.xml | 6 +
.../iceberg/parquet/GpuParquetReader.java | 2 +
.../spark/source/GpuMultiFileBatchReader.java | 6 +-
.../nvidia/spark/rapids/GpuParquetScan.scala | 376 ++++++++++++++++--
.../spark/rapids/HostMemoryStreams.scala | 12 +
.../com/nvidia/spark/rapids/RapidsConf.scala | 31 ++
9 files changed, 406 insertions(+), 44 deletions(-)
diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py
index a223d6559ed..6aa234003ba 100644
--- a/integration_tests/src/main/python/parquet_test.py
+++ b/integration_tests/src/main/python/parquet_test.py
@@ -299,12 +299,19 @@ def test_parquet_read_round_trip_binary_as_string(std_input_path, read_func, rea
@pytest.mark.parametrize('compress', parquet_compress_options)
@pytest.mark.parametrize('reader_confs', reader_opt_confs)
@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"])
-def test_parquet_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list, reader_confs):
+@pytest.mark.parametrize('cpu_decompress', [True, False])
+def test_parquet_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list, reader_confs, cpu_decompress):
data_path = spark_tmp_path + '/PARQUET_DATA'
with_cpu_session(
lambda spark : binary_op_df(spark, long_gen).write.parquet(data_path),
conf={'spark.sql.parquet.compression.codec': compress})
all_confs = copy_and_update(reader_confs, {'spark.sql.sources.useV1SourceList': v1_enabled_list})
+ if cpu_decompress:
+ all_confs = copy_and_update(all_confs, {
+ 'spark.rapids.sql.format.parquet.decompressCpu' : 'true',
+ 'spark.rapids.sql.format.parquet.decompressCpu.snappy' : 'true',
+ 'spark.rapids.sql.format.parquet.decompressCpu.zstd' : 'true'
+ })
assert_gpu_and_cpu_are_equal_collect(
lambda spark : spark.read.parquet(data_path),
conf=all_confs)
diff --git a/jenkins/databricks/install_deps.py b/jenkins/databricks/install_deps.py
index 11e2162957e..23453912827 100644
--- a/jenkins/databricks/install_deps.py
+++ b/jenkins/databricks/install_deps.py
@@ -135,6 +135,8 @@ def define_deps(spark_version, scala_version):
f'{prefix_ws_sp_mvn_hadoop}--org.apache.avro--avro-mapred--org.apache.avro__avro-mapred__*.jar'),
Artifact('org.apache.avro', 'avro',
f'{prefix_ws_sp_mvn_hadoop}--org.apache.avro--avro--org.apache.avro__avro__*.jar'),
+ Artifact('com.github.luben', 'zstd-jni',
+ f'{prefix_ws_sp_mvn_hadoop}--com.github.luben--zstd-jni--com.github.luben__zstd-jni__*.jar'),
]
# Parquet
diff --git a/scala2.13/shim-deps/databricks/pom.xml b/scala2.13/shim-deps/databricks/pom.xml
index 9d6ff787ef1..484e2896f61 100644
--- a/scala2.13/shim-deps/databricks/pom.xml
+++ b/scala2.13/shim-deps/databricks/pom.xml
@@ -231,6 +231,12 @@
${spark.version}
compile
+
+ com.github.luben
+ zstd-jni
+ ${spark.version}
+ compile
+
org.apache.arrow
arrow-format
diff --git a/shim-deps/databricks/pom.xml b/shim-deps/databricks/pom.xml
index edfa3d6f896..5f36e529aa7 100644
--- a/shim-deps/databricks/pom.xml
+++ b/shim-deps/databricks/pom.xml
@@ -231,6 +231,12 @@
${spark.version}
compile
+
+ com.github.luben
+ zstd-jni
+ ${spark.version}
+ compile
+
org.apache.arrow
arrow-format
diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java
index 47b649af6ed..c61f7c6b6f7 100644
--- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java
+++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java
@@ -25,6 +25,7 @@
import scala.collection.Seq;
+import com.nvidia.spark.rapids.CpuCompressionConfig$;
import com.nvidia.spark.rapids.DateTimeRebaseCorrected$;
import com.nvidia.spark.rapids.GpuMetric;
import com.nvidia.spark.rapids.GpuParquetUtils;
@@ -144,6 +145,7 @@ public org.apache.iceberg.io.CloseableIterator iterator() {
partReaderSparkSchema, debugDumpPrefix, debugDumpAlways,
maxBatchSizeRows, maxBatchSizeBytes, targetBatchSizeBytes, useChunkedReader,
maxChunkedReaderMemoryUsageSizeBytes,
+ CpuCompressionConfig$.MODULE$.disabled(),
metrics,
DateTimeRebaseCorrected$.MODULE$, // dateRebaseMode
DateTimeRebaseCorrected$.MODULE$, // timestampRebaseMode
diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java
index 9c36fe76020..b32e5e755cb 100644
--- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java
+++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java
@@ -352,7 +352,8 @@ protected FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles,
return new MultiFileCloudParquetPartitionReader(conf, pFiles,
this::filterParquetBlocks, caseSensitive, parquetDebugDumpPrefix, parquetDebugDumpAlways,
maxBatchSizeRows, maxBatchSizeBytes, targetBatchSizeBytes, maxGpuColumnSizeBytes,
- useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes, metrics, partitionSchema,
+ useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes,
+ CpuCompressionConfig$.MODULE$.disabled(), metrics, partitionSchema,
numThreads, maxNumFileProcessed,
false, // ignoreMissingFiles
false, // ignoreCorruptFiles
@@ -411,7 +412,7 @@ protected FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles,
JavaConverters.asJavaCollection(filteredInfo.parquetBlockMeta.blocks()).stream()
.map(b -> ParquetSingleDataBlockMeta.apply(
filteredInfo.parquetBlockMeta.filePath(),
- ParquetDataBlock.apply(b),
+ ParquetDataBlock.apply(b, CpuCompressionConfig$.MODULE$.disabled()),
InternalRow.empty(),
ParquetSchemaWrapper.apply(filteredInfo.parquetBlockMeta.schema()),
filteredInfo.parquetBlockMeta.readSchema(),
@@ -431,6 +432,7 @@ protected FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles,
caseSensitive, parquetDebugDumpPrefix, parquetDebugDumpAlways,
maxBatchSizeRows, maxBatchSizeBytes, targetBatchSizeBytes, maxGpuColumnSizeBytes,
useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes,
+ CpuCompressionConfig$.MODULE$.disabled(),
metrics, partitionSchema, numThreads,
false, // ignoreMissingFiles
false, // ignoreCorruptFiles
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala
index e38dab50d72..03eb48de6fb 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala
@@ -16,7 +16,7 @@
package com.nvidia.spark.rapids
-import java.io.{Closeable, EOFException, FileNotFoundException, IOException, OutputStream}
+import java.io.{Closeable, EOFException, FileNotFoundException, InputStream, IOException, OutputStream}
import java.net.URI
import java.nio.ByteBuffer
import java.nio.channels.SeekableByteChannel
@@ -31,6 +31,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.language.implicitConversions
import ai.rapids.cudf._
+import com.github.luben.zstd.ZstdDecompressCtx
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
import com.nvidia.spark.rapids.GpuMetric._
import com.nvidia.spark.rapids.ParquetPartitionReader.{CopyRange, LocalCopy}
@@ -47,6 +48,7 @@ import org.apache.parquet.bytes.BytesUtils
import org.apache.parquet.bytes.BytesUtils.readIntLittleEndian
import org.apache.parquet.column.ColumnDescriptor
import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.Util
import org.apache.parquet.format.converter.ParquetMetadataConverter
import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
import org.apache.parquet.hadoop.ParquetFileWriter.MAGIC
@@ -54,6 +56,7 @@ import org.apache.parquet.hadoop.metadata._
import org.apache.parquet.io.{InputFile, SeekableInputStream}
import org.apache.parquet.schema.{DecimalMetadata, GroupType, MessageType, OriginalType, PrimitiveType, Type}
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
+import org.xerial.snappy.Snappy
import org.apache.spark.TaskContext
import org.apache.spark.broadcast.Broadcast
@@ -1106,6 +1109,7 @@ case class GpuParquetMultiFilePartitionReaderFactory(
}.getOrElse(rapidsConf.getMultithreadedReaderKeepOrder)
private val alluxioReplacementTaskTime =
AlluxioCfgUtils.enabledAlluxioReplacementAlgoTaskTime(rapidsConf)
+ private val compressCfg = CpuCompressionConfig.forParquet(rapidsConf)
// We can't use the coalescing files reader when InputFileName, InputFileBlockStart,
// or InputFileBlockLength because we are combining all the files into a single buffer
@@ -1137,7 +1141,7 @@ case class GpuParquetMultiFilePartitionReaderFactory(
new MultiFileCloudParquetPartitionReader(conf, files, filterFunc, isCaseSensitive,
debugDumpPrefix, debugDumpAlways, maxReadBatchSizeRows, maxReadBatchSizeBytes,
targetBatchSizeBytes, maxGpuColumnSizeBytes,
- useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes,
+ useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes, compressCfg,
metrics, partitionSchema, numThreads, maxNumFileProcessed, ignoreMissingFiles,
ignoreCorruptFiles, readUseFieldId, alluxioPathReplacementMap.getOrElse(Map.empty),
alluxioReplacementTaskTime, queryUsesInputFile, keepReadsInOrderFromConf, combineConf)
@@ -1244,7 +1248,7 @@ case class GpuParquetMultiFilePartitionReaderFactory(
clippedBlocks ++= singleFileInfo.blocks.map(block =>
ParquetSingleDataBlockMeta(
singleFileInfo.filePath,
- ParquetDataBlock(block),
+ ParquetDataBlock(block, compressCfg),
metaAndFile.file.partitionValues,
ParquetSchemaWrapper(singleFileInfo.schema),
singleFileInfo.readSchema,
@@ -1262,7 +1266,7 @@ case class GpuParquetMultiFilePartitionReaderFactory(
new MultiFileParquetPartitionReader(conf, files, clippedBlocks.toSeq, isCaseSensitive,
debugDumpPrefix, debugDumpAlways, maxReadBatchSizeRows, maxReadBatchSizeBytes,
targetBatchSizeBytes, maxGpuColumnSizeBytes,
- useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes,
+ useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes, compressCfg,
metrics, partitionSchema, numThreads, ignoreMissingFiles, ignoreCorruptFiles, readUseFieldId)
}
@@ -1307,6 +1311,7 @@ case class GpuParquetPartitionReaderFactory(
private val readUseFieldId = ParquetSchemaClipShims.useFieldId(sqlConf)
private val footerReadType = GpuParquetScan.footerReaderHeuristic(
rapidsConf.parquetReaderFooterType, dataSchema, readDataSchema, readUseFieldId)
+ private val compressCfg = CpuCompressionConfig.forParquet(rapidsConf)
override def supportColumnarReads(partition: InputPartition): Boolean = true
@@ -1335,12 +1340,29 @@ case class GpuParquetPartitionReaderFactory(
new ParquetPartitionReader(conf, file, singleFileInfo.filePath, singleFileInfo.blocks,
singleFileInfo.schema, isCaseSensitive, readDataSchema, debugDumpPrefix, debugDumpAlways,
maxReadBatchSizeRows, maxReadBatchSizeBytes, targetSizeBytes,
- useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes,
+ useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes, compressCfg,
metrics, singleFileInfo.dateRebaseMode,
singleFileInfo.timestampRebaseMode, singleFileInfo.hasInt96Timestamps, readUseFieldId)
}
}
+case class CpuCompressionConfig(
+ decompressSnappyCpu: Boolean,
+ decompressZstdCpu: Boolean) {
+ val decompressAnyCpu: Boolean = decompressSnappyCpu || decompressZstdCpu
+}
+
+object CpuCompressionConfig {
+ def forParquet(conf: RapidsConf): CpuCompressionConfig = {
+ val cpuEnable = conf.parquetDecompressCpu
+ CpuCompressionConfig(
+ decompressSnappyCpu = cpuEnable && conf.parquetDecompressCpuSnappy,
+ decompressZstdCpu = cpuEnable && conf.parquetDecompressCpuZstd)
+ }
+
+ def disabled(): CpuCompressionConfig = CpuCompressionConfig(false, false)
+}
+
trait ParquetPartitionReaderBase extends Logging with ScanWithMetrics
with MultiFileReaderFunctions {
// the size of Parquet magic (at start+end) and footer length values
@@ -1353,6 +1375,8 @@ trait ParquetPartitionReaderBase extends Logging with ScanWithMetrics
def isSchemaCaseSensitive: Boolean
+ def compressCfg: CpuCompressionConfig
+
val copyBufferSize = conf.getInt("parquet.read.allocation.size", 8 * 1024 * 1024)
def checkIfNeedToSplitBlocks(currentDateRebaseMode: DateTimeRebaseMode,
@@ -1418,13 +1442,8 @@ trait ParquetPartitionReaderBase extends Logging with ScanWithMetrics
schema: MessageType,
handleCoalesceFiles: Boolean): Long = {
// start with the size of Parquet magic (at start+end) and footer length values
- var size: Long = PARQUET_META_SIZE
-
- // Calculate the total amount of column data that will be copied
- // NOTE: Avoid using block.getTotalByteSize here as that is the
- // uncompressed size rather than the size in the file.
- size += currentChunkedBlocks.flatMap(_.getColumns.asScala.map(_.getTotalSize)).sum
-
+ val headerSize: Long = PARQUET_META_SIZE
+ val blocksSize = ParquetPartitionReader.computeOutputSize(currentChunkedBlocks, compressCfg)
val footerSize = calculateParquetFooterSize(currentChunkedBlocks, schema)
val extraMemory = if (handleCoalesceFiles) {
val numCols = currentChunkedBlocks.head.getColumns().size()
@@ -1432,8 +1451,7 @@ trait ParquetPartitionReaderBase extends Logging with ScanWithMetrics
} else {
0
}
- val totalSize = size + footerSize + extraMemory
- totalSize
+ headerSize + blocksSize + footerSize + extraMemory
}
protected def writeFooter(
@@ -1532,7 +1550,7 @@ trait ParquetPartitionReaderBase extends Logging with ScanWithMetrics
* metadata but with the file offsets updated to reflect the new position of the column data
* as written to the output.
*
- * @param in the input stream for the original Parquet file
+ * @param filePath the path to the Parquet file
* @param out the output stream to receive the data
* @param blocks block metadata from the original file that will appear in the computed file
* @param realStartOffset starting file offset of the first block
@@ -1575,6 +1593,258 @@ trait ParquetPartitionReaderBase extends Logging with ScanWithMetrics
computeBlockMetaData(blocks, realStartOffset)
}
+ private class BufferedFileInput(
+ filePath: Path,
+ blocks: Seq[BlockMetaData],
+ metrics: Map[String, GpuMetric]) extends InputStream {
+ private[this] val in = filePath.getFileSystem(conf).open(filePath)
+ private[this] val buffer: Array[Byte] = new Array[Byte](copyBufferSize)
+ private[this] var bufferSize: Int = 0
+ private[this] var bufferFilePos: Long = in.getPos
+ private[this] var bufferPos: Int = 0
+ private[this] val columnIter = blocks.flatMap(_.getColumns.asScala).iterator
+ private[this] var currentColumn: Option[ColumnChunkMetaData] = None
+ private[this] val readTime: GpuMetric = metrics.getOrElse(READ_FS_TIME, NoopMetric)
+
+ override def read(): Int = {
+ while (bufferPos == bufferSize) {
+ fillBuffer()
+ }
+ val result = buffer(bufferPos)
+ bufferPos += 1
+ result
+ }
+
+ override def read(b: Array[Byte]): Int = read(b, 0, b.length)
+
+ override def read(dest: Array[Byte], off: Int, len: Int): Int = {
+ var bytesLeft = len
+ while (bytesLeft > 0) {
+ if (bufferPos == bufferSize) {
+ fillBuffer()
+ }
+ val numBytes = Math.min(bytesLeft, bufferSize - bufferPos)
+ System.arraycopy(buffer, bufferPos, dest, off + len - bytesLeft, numBytes)
+ bufferPos += numBytes
+ bytesLeft -= numBytes
+ }
+ len
+ }
+
+ def read(out: HostMemoryOutputStream, len: Long): Unit = {
+ var bytesLeft = len
+ while (bytesLeft > 0) {
+ if (bufferPos == bufferSize) {
+ fillBuffer()
+ }
+ // downcast is safe because bufferSize is an int
+ val numBytes = Math.min(bytesLeft, bufferSize - bufferPos).toInt
+ out.write(buffer, bufferPos, numBytes)
+ bufferPos += numBytes
+ bytesLeft -= numBytes
+ }
+ }
+
+ def read(out: HostMemoryBuffer, len: Long): Unit = {
+ var bytesLeft = len
+ while (bytesLeft > 0) {
+ if (bufferPos == bufferSize) {
+ fillBuffer()
+ }
+ // downcast is safe because bufferSize is an int
+ val numBytes = Math.min(bytesLeft, bufferSize - bufferPos).toInt
+ out.setBytes(len - bytesLeft, buffer, bufferPos, numBytes)
+ bufferPos += numBytes
+ bytesLeft -= numBytes
+ }
+ }
+
+ override def skip(n: Long): Long = {
+ seek(getPos + n)
+ n
+ }
+
+ def getPos: Long = bufferFilePos + bufferPos
+
+ def seek(desiredPos: Long): Unit = {
+ require(desiredPos >= getPos, "Only supports seeking forward")
+ val posDiff = desiredPos - bufferFilePos
+ if (posDiff >= 0 && posDiff < bufferSize) {
+ bufferPos = posDiff.toInt
+ } else {
+ in.seek(desiredPos)
+ bufferFilePos = desiredPos
+ bufferSize = 0
+ bufferPos = 0
+ }
+ }
+
+ override def close(): Unit = {
+ readTime.ns {
+ in.close()
+ }
+ }
+
+ private def fillBuffer(): Unit = {
+ // TODO: Add FileCache support https://github.com/NVIDIA/spark-rapids/issues/11775
+ var bytesToCopy = currentColumn.map { c =>
+ Math.max(0, c.getStartingPos + c.getTotalSize - getPos)
+ }.getOrElse(0L)
+ var done = bytesToCopy >= buffer.length
+ while (!done && columnIter.hasNext) {
+ val column = columnIter.next()
+ currentColumn = Some(column)
+ done = if (getPos + bytesToCopy == column.getStartingPos) {
+ bytesToCopy += column.getTotalSize
+ bytesToCopy >= buffer.length
+ } else {
+ true
+ }
+ }
+ if (bytesToCopy <= 0) {
+ throw new EOFException("read beyond column data range")
+ }
+ bufferFilePos = in.getPos
+ bufferPos = 0
+ bufferSize = Math.min(bytesToCopy, buffer.length).toInt
+ readTime.ns {
+ in.readFully(buffer, 0, bufferSize)
+ }
+ }
+ }
+
+ /**
+ * Copies the data corresponding to the clipped blocks in the original file and compute the
+ * block metadata for the output. The output blocks will contain the same column chunk
+ * metadata but with the file offsets updated to reflect the new position of the column data
+ * as written to the output.
+ *
+ * @param filePath the path to the Parquet file
+ * @param out the output stream to receive the data
+ * @param blocks block metadata from the original file that will appear in the computed file
+ * @param realStartOffset starting file offset of the first block
+ * @return updated block metadata corresponding to the output
+ */
+ protected def copyAndUncompressBlocksData(
+ filePath: Path,
+ out: HostMemoryOutputStream,
+ blocks: Seq[BlockMetaData],
+ realStartOffset: Long,
+ metrics: Map[String, GpuMetric],
+ compressCfg: CpuCompressionConfig): Seq[BlockMetaData] = {
+ val outStartPos = out.getPos
+ val writeTime = metrics.getOrElse(WRITE_BUFFER_TIME, NoopMetric)
+ withResource(new BufferedFileInput(filePath, blocks, metrics)) { in =>
+ val newBlocks = blocks.map { block =>
+ val newColumns = block.getColumns.asScala.map { column =>
+ var columnTotalSize = column.getTotalSize
+ var columnCodec = column.getCodec
+ val columnStartingPos = realStartOffset + out.getPos - outStartPos
+ val columnDictOffset = if (column.getDictionaryPageOffset > 0) {
+ column.getDictionaryPageOffset + columnStartingPos - column.getStartingPos
+ } else {
+ 0
+ }
+ writeTime.ns {
+ columnCodec match {
+ case CompressionCodecName.SNAPPY if compressCfg.decompressSnappyCpu =>
+ val columnStartPos = out.getPos
+ decompressSnappy(in, out, column)
+ columnCodec = CompressionCodecName.UNCOMPRESSED
+ columnTotalSize = out.getPos - columnStartPos
+ case CompressionCodecName.ZSTD if compressCfg.decompressZstdCpu =>
+ val columnStartPos = out.getPos
+ decompressZstd(in, out, column)
+ columnCodec = CompressionCodecName.UNCOMPRESSED
+ columnTotalSize = out.getPos - columnStartPos
+ case _ =>
+ in.seek(column.getStartingPos)
+ in.read(out, columnTotalSize)
+ }
+ }
+ ColumnChunkMetaData.get(
+ column.getPath,
+ column.getPrimitiveType,
+ columnCodec,
+ column.getEncodingStats,
+ column.getEncodings,
+ column.getStatistics,
+ columnStartingPos,
+ columnDictOffset,
+ column.getValueCount,
+ columnTotalSize,
+ columnTotalSize)
+ }
+ GpuParquetUtils.newBlockMeta(block.getRowCount, newColumns.toSeq)
+ }
+ newBlocks
+ }
+ }
+
+ private def decompressSnappy(
+ in: BufferedFileInput,
+ out: HostMemoryOutputStream,
+ column: ColumnChunkMetaData): Unit = {
+ val endPos = column.getStartingPos + column.getTotalSize
+ in.seek(column.getStartingPos)
+ var inData: Option[HostMemoryBuffer] = None
+ try {
+ while (in.getPos != endPos) {
+ val pageHeader = Util.readPageHeader(in)
+ val compressedSize = pageHeader.getCompressed_page_size
+ val uncompressedSize = pageHeader.getUncompressed_page_size
+ pageHeader.unsetCrc()
+ pageHeader.setCompressed_page_size(uncompressedSize)
+ Util.writePageHeader(pageHeader, out)
+ if (inData.map(_.getLength).getOrElse(0L) < compressedSize) {
+ inData.foreach(_.close())
+ inData = Some(HostMemoryBuffer.allocate(compressedSize, false))
+ }
+ inData.foreach { compressedBuffer =>
+ in.read(compressedBuffer, compressedSize)
+ val bbIn = compressedBuffer.asByteBuffer(0, compressedSize)
+ val bbOut = out.writeAsByteBuffer(uncompressedSize)
+ Snappy.uncompress(bbIn, bbOut)
+ }
+ }
+ } finally {
+ inData.foreach(_.close())
+ }
+ }
+
+ private def decompressZstd(
+ in: BufferedFileInput,
+ out: HostMemoryOutputStream,
+ column: ColumnChunkMetaData): Unit = {
+ val endPos = column.getStartingPos + column.getTotalSize
+ in.seek(column.getStartingPos)
+ var inData: Option[HostMemoryBuffer] = None
+ try {
+ withResource(new ZstdDecompressCtx()) { ctx =>
+ while (in.getPos != endPos) {
+ val pageHeader = Util.readPageHeader(in)
+ val compressedSize = pageHeader.getCompressed_page_size
+ val uncompressedSize = pageHeader.getUncompressed_page_size
+ pageHeader.unsetCrc()
+ pageHeader.setCompressed_page_size(uncompressedSize)
+ Util.writePageHeader(pageHeader, out)
+ if (inData.map(_.getLength).getOrElse(0L) < compressedSize) {
+ inData.foreach(_.close())
+ inData = Some(HostMemoryBuffer.allocate(compressedSize, false))
+ }
+ inData.foreach { compressedBuffer =>
+ in.read(compressedBuffer, compressedSize)
+ val bbIn = compressedBuffer.asByteBuffer(0, compressedSize)
+ val bbOut = out.writeAsByteBuffer(uncompressedSize)
+ ctx.decompress(bbOut, bbIn)
+ }
+ }
+ }
+ } finally {
+ inData.foreach(_.close())
+ }
+ }
+
private def copyRemoteBlocksData(
remoteCopies: Seq[CopyRange],
filePath: Path,
@@ -1666,7 +1936,11 @@ trait ParquetPartitionReaderBase extends Logging with ScanWithMetrics
closeOnExcept(HostMemoryBuffer.allocate(estTotalSize)) { hmb =>
val out = new HostMemoryOutputStream(hmb)
out.write(ParquetPartitionReader.PARQUET_MAGIC)
- val outputBlocks = copyBlocksData(filePath, out, blocks, out.getPos, metrics)
+ val outputBlocks = if (compressCfg.decompressAnyCpu) {
+ copyAndUncompressBlocksData(filePath, out, blocks, out.getPos, metrics, compressCfg)
+ } else {
+ copyBlocksData(filePath, out, blocks, out.getPos, metrics)
+ }
val footerPos = out.getPos
writeFooter(out, outputBlocks, clippedSchema)
BytesUtils.writeIntLittleEndian(out, (out.getPos - footerPos).toInt)
@@ -1802,7 +2076,7 @@ trait ParquetPartitionReaderBase extends Logging with ScanWithMetrics
block.asInstanceOf[ParquetDataBlock].dataBlock
implicit def toDataBlockBase(blocks: Seq[BlockMetaData]): Seq[DataBlockBase] =
- blocks.map(ParquetDataBlock)
+ blocks.map(b => ParquetDataBlock(b, compressCfg))
implicit def toBlockMetaDataSeq(blocks: Seq[DataBlockBase]): Seq[BlockMetaData] =
blocks.map(_.asInstanceOf[ParquetDataBlock].dataBlock)
@@ -1814,10 +2088,14 @@ private case class ParquetSchemaWrapper(schema: MessageType) extends SchemaBase
}
// Parquet BlockMetaData wrapper
-private case class ParquetDataBlock(dataBlock: BlockMetaData) extends DataBlockBase {
+private case class ParquetDataBlock(
+ dataBlock: BlockMetaData,
+ compressCfg: CpuCompressionConfig) extends DataBlockBase {
override def getRowCount: Long = dataBlock.getRowCount
override def getReadDataSize: Long = dataBlock.getTotalByteSize
- override def getBlockSize: Long = dataBlock.getColumns.asScala.map(_.getTotalSize).sum
+ override def getBlockSize: Long = {
+ ParquetPartitionReader.computeOutputSize(dataBlock, compressCfg)
+ }
}
/** Parquet extra information containing rebase modes and whether there is int96 timestamp */
@@ -1876,6 +2154,7 @@ class MultiFileParquetPartitionReader(
maxGpuColumnSizeBytes: Long,
useChunkedReader: Boolean,
maxChunkedReaderMemoryUsageSizeBytes: Long,
+ override val compressCfg: CpuCompressionConfig,
override val execMetrics: Map[String, GpuMetric],
partitionSchema: StructType,
numThreads: Int,
@@ -1900,7 +2179,8 @@ class MultiFileParquetPartitionReader(
file: Path,
outhmb: HostMemoryBuffer,
blocks: ArrayBuffer[DataBlockBase],
- offset: Long)
+ offset: Long,
+ compressCfg: CpuCompressionConfig)
extends Callable[(Seq[DataBlockBase], Long)] {
override def call(): (Seq[DataBlockBase], Long) = {
@@ -1909,7 +2189,11 @@ class MultiFileParquetPartitionReader(
val startBytesRead = fileSystemBytesRead()
val outputBlocks = withResource(outhmb) { _ =>
withResource(new HostMemoryOutputStream(outhmb)) { out =>
- copyBlocksData(file, out, blocks.toSeq, offset, metrics)
+ if (compressCfg.decompressAnyCpu) {
+ copyAndUncompressBlocksData(file, out, blocks.toSeq, offset, metrics, compressCfg)
+ } else {
+ copyBlocksData(file, out, blocks.toSeq, offset, metrics)
+ }
}
}
val bytesRead = fileSystemBytesRead() - startBytesRead
@@ -1961,7 +2245,7 @@ class MultiFileParquetPartitionReader(
blocks: ArrayBuffer[DataBlockBase],
offset: Long,
batchContext: BatchContext): Callable[(Seq[DataBlockBase], Long)] = {
- new ParquetCopyBlocksRunner(taskContext, file, outhmb, blocks, offset)
+ new ParquetCopyBlocksRunner(taskContext, file, outhmb, blocks, offset, compressCfg)
}
override final def getFileFormatShortName: String = "Parquet"
@@ -2072,6 +2356,7 @@ class MultiFileCloudParquetPartitionReader(
maxGpuColumnSizeBytes: Long,
useChunkedReader: Boolean,
maxChunkedReaderMemoryUsageSizeBytes: Long,
+ override val compressCfg: CpuCompressionConfig,
override val execMetrics: Map[String, GpuMetric],
partitionSchema: StructType,
numThreads: Int,
@@ -2761,6 +3046,7 @@ class ParquetPartitionReader(
targetBatchSizeBytes: Long,
useChunkedReader: Boolean,
maxChunkedReaderMemoryUsageSizeBytes: Long,
+ override val compressCfg: CpuCompressionConfig,
override val execMetrics: Map[String, GpuMetric],
dateRebaseMode: DateTimeRebaseMode,
timestampRebaseMode: DateTimeRebaseMode,
@@ -2873,26 +3159,34 @@ object ParquetPartitionReader {
length: Long,
outputOffset: Long) extends CopyItem
- /**
- * Build a new BlockMetaData
- *
- * @param rowCount the number of rows in this block
- * @param columns the new column chunks to reference in the new BlockMetaData
- * @return the new BlockMetaData
- */
- private[rapids] def newParquetBlock(
- rowCount: Long,
- columns: Seq[ColumnChunkMetaData]): BlockMetaData = {
- val block = new BlockMetaData
- block.setRowCount(rowCount)
+ private[rapids] def computeOutputSize(
+ blocks: Seq[BlockMetaData],
+ compressCfg: CpuCompressionConfig): Long = {
+ blocks.map { block =>
+ computeOutputSize(block, compressCfg)
+ }.sum
+ }
- var totalSize: Long = 0
- columns.foreach { column =>
- block.addColumn(column)
- totalSize += column.getTotalUncompressedSize
+ private[rapids] def computeOutputSize(
+ block: BlockMetaData,
+ compressCfg: CpuCompressionConfig): Long = {
+ if (compressCfg.decompressAnyCpu) {
+ block.getColumns.asScala.map { c =>
+ if ((c.getCodec == CompressionCodecName.SNAPPY && compressCfg.decompressSnappyCpu)
+ || (c.getCodec == CompressionCodecName.ZSTD && compressCfg.decompressZstdCpu)) {
+ // Page headers need to be rewritten when CPU decompresses, and that may
+ // increase the size of the page header. Guess how many pages there may be
+ // and add a fudge factor per page to try to avoid a late realloc+copy.
+ // NOTE: Avoid using block.getTotalByteSize as that is the
+ // uncompressed size rather than the size in the file.
+ val estimatedPageCount = (c.getTotalUncompressedSize / (1024 * 1024)) + 1
+ c.getTotalUncompressedSize + estimatedPageCount * 8
+ } else {
+ c.getTotalSize
+ }
+ }.sum
+ } else {
+ block.getColumns.asScala.map(_.getTotalSize).sum
}
- block.setTotalByteSize(totalSize)
-
- block
}
}
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostMemoryStreams.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostMemoryStreams.scala
index 08fe5be50b2..4be11b13254 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostMemoryStreams.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostMemoryStreams.scala
@@ -54,6 +54,12 @@ class HostMemoryOutputStream(val buffer: HostMemoryBuffer) extends OutputStream
pos += numBytes
}
+ def writeAsByteBuffer(length: Int): ByteBuffer = {
+ val bb = buffer.asByteBuffer(pos, length)
+ pos += length
+ bb
+ }
+
def getPos: Long = pos
def seek(newPos: Long): Unit = {
@@ -132,6 +138,12 @@ trait HostMemoryInputStreamMixIn extends InputStream {
}
}
+ def readByteBuffer(length: Int): ByteBuffer = {
+ val bb = hmb.asByteBuffer(pos, length)
+ pos += length
+ bb
+ }
+
override def skip(count: Long): Long = {
val oldPos = pos
pos = Math.min(pos + count, hmbLength)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
index ab7a788d205..406aeb0365b 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
@@ -1120,6 +1120,31 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern")
.checkValues(RapidsReaderType.values.map(_.toString))
.createWithDefault(RapidsReaderType.AUTO.toString)
+ val PARQUET_DECOMPRESS_CPU =
+ conf("spark.rapids.sql.format.parquet.decompressCpu")
+ .doc("If true then the CPU is eligible to decompress Parquet data rather than the GPU. " +
+ s"See other spark.rapids.sql.format.parquet.decompressCpu.* configuration settings " +
+ "to control this for specific compression codecs.")
+ .internal()
+ .booleanConf
+ .createWithDefault(false)
+
+ val PARQUET_DECOMPRESS_CPU_SNAPPY =
+ conf("spark.rapids.sql.format.parquet.decompressCpu.snappy")
+ .doc(s"If true and $PARQUET_DECOMPRESS_CPU is true then the CPU decompresses " +
+ "Parquet Snappy data rather than the GPU")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARQUET_DECOMPRESS_CPU_ZSTD =
+ conf("spark.rapids.sql.format.parquet.decompressCpu.zstd")
+ .doc(s"If true and $PARQUET_DECOMPRESS_CPU is true then the CPU decompresses " +
+ "Parquet Zstandard data rather than the GPU")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
val READER_MULTITHREADED_COMBINE_THRESHOLD =
conf("spark.rapids.sql.reader.multithreaded.combine.sizeBytes")
.doc("The target size in bytes to combine multiple small files together when using the " +
@@ -2960,6 +2985,12 @@ class RapidsConf(conf: Map[String, String]) extends Logging {
lazy val isParquetMultiThreadReadEnabled: Boolean = isParquetAutoReaderEnabled ||
RapidsReaderType.withName(get(PARQUET_READER_TYPE)) == RapidsReaderType.MULTITHREADED
+ lazy val parquetDecompressCpu: Boolean = get(PARQUET_DECOMPRESS_CPU)
+
+ lazy val parquetDecompressCpuSnappy: Boolean = get(PARQUET_DECOMPRESS_CPU_SNAPPY)
+
+ lazy val parquetDecompressCpuZstd: Boolean = get(PARQUET_DECOMPRESS_CPU_ZSTD)
+
lazy val maxNumParquetFilesParallel: Int = get(PARQUET_MULTITHREAD_READ_MAX_NUM_FILES_PARALLEL)
lazy val isParquetReadEnabled: Boolean = get(ENABLE_PARQUET_READ)
From ed02cfe4f54e3c8531017671fd6ad0388128cb75 Mon Sep 17 00:00:00 2001
From: MithunR
Date: Tue, 26 Nov 2024 14:02:54 -0800
Subject: [PATCH 089/103] Fix `dpp_test.py` failures on [databricks] 14.3
(#11768)
Fixes #11536.
This commit fixes the tests in `dpp_test.py` that were failing on
Databricks 14.3.
The failures were largely a result of an erroneous shim implementation,
that was fixed as part of #11750.
This commit accounts for the remaining failures that result from there
being a `CollectLimitExec` in certain DPP query plans (that include
broadcast joins, for example). The tests have been made more
permissive, in allowing the `CollectLimitExec` to run on the CPU.
The `CollectLimitExec` based plans will be further explored as part of
https://github.com/NVIDIA/spark-rapids/issues/11764.
Signed-off-by: MithunR
---
integration_tests/src/main/python/dpp_test.py | 14 ++++++++++++--
1 file changed, 12 insertions(+), 2 deletions(-)
diff --git a/integration_tests/src/main/python/dpp_test.py b/integration_tests/src/main/python/dpp_test.py
index b362a4175f3..3d5ee1a5afa 100644
--- a/integration_tests/src/main/python/dpp_test.py
+++ b/integration_tests/src/main/python/dpp_test.py
@@ -20,7 +20,7 @@
from conftest import spark_tmp_table_factory
from data_gen import *
from marks import ignore_order, allow_non_gpu, datagen_overrides, disable_ansi_mode
-from spark_session import is_before_spark_320, with_cpu_session, is_before_spark_312, is_databricks_runtime, is_databricks113_or_later
+from spark_session import is_before_spark_320, with_cpu_session, is_before_spark_312, is_databricks_runtime, is_databricks113_or_later, is_databricks_version_or_later
# non-positive values here can produce a degenerative join, so here we ensure that most values are
# positive to ensure the join will produce rows. See https://github.com/NVIDIA/spark-rapids/issues/10147
@@ -167,10 +167,17 @@ def fn(spark):
'''
]
+# On some Databricks versions (>=14.3), some query plans include a `CollectLimitExec`,
+# when filtering partitions. This exec falls back to CPU. These tests allow for `CollectLimit` to
+# run on the CPU, if everything else in the plan execute as expected.
+# Further details are furnished at https://github.com/NVIDIA/spark-rapids/issues/11764.
+dpp_fallback_execs=["CollectLimitExec"] if is_databricks_version_or_later(14,3) else []
+
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
# When BroadcastExchangeExec is available on filtering side, and it can be reused:
# DynamicPruningExpression(InSubqueryExec(value, GpuSubqueryBroadcastExec)))
@ignore_order
+@allow_non_gpu(*dpp_fallback_execs)
@datagen_overrides(seed=0, reason="https://github.com/NVIDIA/spark-rapids/issues/10147")
@pytest.mark.parametrize('store_format', ['parquet', 'orc'], ids=idfn)
@pytest.mark.parametrize('s_index', list(range(len(_statements))), ids=idfn)
@@ -245,6 +252,7 @@ def test_dpp_bypass(spark_tmp_table_factory, store_format, s_index, aqe_enabled)
# then Spark will plan an extra Aggregate to collect filtering values:
# DynamicPruningExpression(InSubqueryExec(value, SubqueryExec(Aggregate(...))))
@ignore_order
+@allow_non_gpu(*dpp_fallback_execs)
@pytest.mark.parametrize('store_format', ['parquet', 'orc'], ids=idfn)
@pytest.mark.parametrize('s_index', list(range(len(_statements))), ids=idfn)
@pytest.mark.parametrize('aqe_enabled', [
@@ -285,10 +293,11 @@ def test_dpp_skip(spark_tmp_table_factory, store_format, s_index, aqe_enabled):
non_exist_classes='DynamicPruningExpression',
conf=dict(_dpp_fallback_conf + [('spark.sql.adaptive.enabled', aqe_enabled)]))
+dpp_like_any_fallback_execs=['FilterExec', 'CollectLimitExec'] if is_databricks_version_or_later(14,3) else ['FilterExec']
# GPU verification on https://issues.apache.org/jira/browse/SPARK-34436
@ignore_order
-@allow_non_gpu('FilterExec')
+@allow_non_gpu(*dpp_like_any_fallback_execs)
@pytest.mark.parametrize('store_format', ['parquet', 'orc'], ids=idfn)
@pytest.mark.parametrize('aqe_enabled', [
'false',
@@ -327,6 +336,7 @@ def create_dim_table_for_like(spark):
@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114
+@allow_non_gpu(*dpp_fallback_execs)
# Test handling DPP expressions from a HashedRelation that rearranges columns
@pytest.mark.parametrize('aqe_enabled', [
'false',
From aa2da410511d8a737e207257769ec662a79174fe Mon Sep 17 00:00:00 2001
From: "Hongbin Ma (Mahone)"
Date: Fri, 29 Nov 2024 23:26:13 +0800
Subject: [PATCH 090/103] fix issue 11790 (#11792)
Signed-off-by: Hongbin Ma (Mahone)
---
.../spark/rapids/GpuAggregateExec.scala | 33 +++++++++++--------
1 file changed, 19 insertions(+), 14 deletions(-)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
index 60f6dd68509..4ba20547e77 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
@@ -219,9 +219,6 @@ object AggregateUtils extends Logging {
): Boolean = {
var repartitionHappened = false
- if (hashSeed > 200) {
- throw new IllegalStateException("Too many times of repartition, may hit a bug?")
- }
def repartitionAndClose(batch: SpillableColumnarBatch): Unit = {
@@ -280,15 +277,23 @@ object AggregateUtils extends Logging {
val newBuckets = batchesByBucket.flatMap(bucket => {
if (needRepartitionAgain(bucket)) {
- val nextLayerBuckets =
- ArrayBuffer.fill(hashBucketNum)(new AutoClosableArrayBuffer[SpillableColumnarBatch]())
- // Recursively merge and repartition the over sized bucket
- repartitionHappened =
- iterateAndRepartition(
- new CloseableBufferedIterator(bucket.iterator), metrics, targetMergeBatchSize,
- helper, hashKeys, hashBucketNum, hashSeed + 7,
- nextLayerBuckets) || repartitionHappened
- nextLayerBuckets
+ if (hashSeed + 7 > 200) {
+ log.warn("Too many times of repartition, may hit a bug? Size for each batch in " +
+ "current bucket: " + bucket.map(_.sizeInBytes).mkString(", ") + " rows: " +
+ bucket.map(_.numRows()).mkString(", ") + " targetMergeBatchSize: "
+ + targetMergeBatchSize)
+ ArrayBuffer.apply(bucket)
+ } else {
+ val nextLayerBuckets =
+ ArrayBuffer.fill(hashBucketNum)(new AutoClosableArrayBuffer[SpillableColumnarBatch]())
+ // Recursively merge and repartition the over sized bucket
+ repartitionHappened =
+ iterateAndRepartition(
+ new CloseableBufferedIterator(bucket.iterator), metrics, targetMergeBatchSize,
+ helper, hashKeys, hashBucketNum, hashSeed + 7,
+ nextLayerBuckets) || repartitionHappened
+ nextLayerBuckets
+ }
} else {
ArrayBuffer.apply(bucket)
}
@@ -1075,8 +1080,8 @@ class GpuMergeAggregateIterator(
closeOnExcept(new ArrayBuffer[AutoClosableArrayBuffer[SpillableColumnarBatch]]) {
toAggregateBuckets =>
var currentSize = 0L
- while (batchesByBucket.nonEmpty &&
- batchesByBucket.last.size() + currentSize < targetMergeBatchSize) {
+ while (batchesByBucket.nonEmpty && (toAggregateBuckets.isEmpty ||
+ batchesByBucket.last.size() + currentSize < targetMergeBatchSize)) {
val bucket = batchesByBucket.remove(batchesByBucket.size - 1)
currentSize += bucket.map(_.sizeInBytes).sum
toAggregateBuckets += bucket
From cb31afb07847ff96b16d70ceec54ee1426fe5e64 Mon Sep 17 00:00:00 2001
From: "Robert (Bobby) Evans"
Date: Mon, 2 Dec 2024 18:19:17 -0600
Subject: [PATCH 091/103] Fall back to CPU for non-UTC months_between (#11802)
Signed-off-by: Robert (Bobby) Evans
---
integration_tests/src/main/python/date_time_test.py | 12 ++++++------
.../spark/sql/rapids/datetimeExpressions.scala | 10 +++++++++-
2 files changed, 15 insertions(+), 7 deletions(-)
diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py
index 5a98e06fadc..1a7024dac85 100644
--- a/integration_tests/src/main/python/date_time_test.py
+++ b/integration_tests/src/main/python/date_time_test.py
@@ -139,34 +139,34 @@ def test_datediff(data_gen):
hms_fallback = ['ProjectExec'] if not is_supported_time_zone() else []
-@allow_non_gpu(*hms_fallback)
+@allow_non_gpu(*non_utc_tz_allow)
def test_months_between():
assert_gpu_and_cpu_are_equal_collect(
lambda spark : binary_op_df(spark, timestamp_gen).selectExpr('months_between(a, b, false)'))
-@allow_non_gpu(*hms_fallback)
+@allow_non_gpu(*non_utc_tz_allow)
def test_months_between_first_day():
assert_gpu_and_cpu_are_equal_collect(
lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('months_between(a, timestamp"2024-01-01", false)'))
-@allow_non_gpu(*hms_fallback)
+@allow_non_gpu(*non_utc_tz_allow)
def test_months_between_last_day():
assert_gpu_and_cpu_are_equal_collect(
lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('months_between(a, timestamp"2023-12-31", false)'))
-@allow_non_gpu(*hms_fallback)
+@allow_non_gpu(*non_utc_tz_allow)
@approximate_float()
def test_months_between_round():
assert_gpu_and_cpu_are_equal_collect(
lambda spark : binary_op_df(spark, timestamp_gen).selectExpr('months_between(a, b, true)'))
-@allow_non_gpu(*hms_fallback)
+@allow_non_gpu(*non_utc_tz_allow)
@approximate_float()
def test_months_between_first_day_round():
assert_gpu_and_cpu_are_equal_collect(
lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('months_between(a, timestamp"2024-01-01", true)'))
-@allow_non_gpu(*hms_fallback)
+@allow_non_gpu(*non_utc_tz_allow)
@approximate_float()
def test_months_between_last_day_round():
assert_gpu_and_cpu_are_equal_collect(
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
index 8ed4c50ac3b..0f382a7b6e6 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
@@ -1217,7 +1217,8 @@ class MonthsBetweenExprMeta(expr: MonthsBetween,
rule: DataFromReplacementRule)
extends ExprMeta[MonthsBetween](expr, conf, parent, rule) {
- override def isTimeZoneSupported = true
+ // See https://github.com/NVIDIA/spark-rapids/issues/11800
+ override def isTimeZoneSupported = false
override def convertToGpu(): GpuExpression = {
val gpuChildren = childExprs.map(_.convertToGpu())
@@ -1287,6 +1288,13 @@ object GpuMonthsBetween {
private def calcSecondsInDay(converted: ColumnVector): ColumnVector = {
// Find the number of seconds that are not counted for in a day
+ // Rounding down to the current day, only works if you are in a time zone with no
+ // transition rules. This is because if a transition happens in between the start
+ // of the day and the timestamp we will be off. As such this will need to change to
+ // support other time zones, and it will need to take the timezone into account when
+ // calculating this.
+ // https://github.com/NVIDIA/spark-rapids/issues/11800
+
// find the micros over by finding the part that is not days
val microsInDay = withResource(converted.dateTimeFloor(DateTimeRoundingFrequency.DAY)) { days =>
// But we cannot subtract timestamps directly. They are both micros
From 738c8e38fc23c1634667443864b80f085f2737ac Mon Sep 17 00:00:00 2001
From: "Hongbin Ma (Mahone)"
Date: Tue, 3 Dec 2024 09:07:10 +0800
Subject: [PATCH 092/103] exclude previous operator's time out of
firstBatchHeuristic (#11794)
Signed-off-by: Hongbin Ma (Mahone)
---
.../main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
index 4ba20547e77..d5bbe15209d 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala
@@ -2091,9 +2091,9 @@ class DynamicGpuPartialAggregateIterator(
helper: AggHelper): (Iterator[ColumnarBatch], Boolean) = {
// we need to decide if we are going to sort the data or not, so the very
// first thing we need to do is get a batch and make a choice.
+ val cb = cbIter.next()
withResource(new NvtxWithMetrics("dynamic sort heuristic", NvtxColor.BLUE,
metrics.opTime, metrics.heuristicTime)) { _ =>
- val cb = cbIter.next()
lazy val estimatedGrowthAfterAgg: Double = closeOnExcept(cb) { cb =>
val numRows = cb.numRows()
val cardinality = estimateCardinality(cb)
From fb2f72df881582855393135d6e574111716ec7bb Mon Sep 17 00:00:00 2001
From: Kuhu Shukla
Date: Fri, 6 Dec 2024 21:52:17 -0600
Subject: [PATCH 093/103] Orc writes don't fully support Booleans with nulls
(#11763)
---
.../main/python/datasourcev2_write_test.py | 8 +--
.../main/python/hive_parquet_write_test.py | 5 +-
.../src/main/python/hive_write_test.py | 10 +++-
integration_tests/src/main/python/orc_test.py | 42 +++++++++++----
.../src/main/python/orc_write_test.py | 51 ++++++++++++++++---
.../src/main/python/schema_evolution_test.py | 11 ++--
.../com/nvidia/spark/rapids/RapidsConf.scala | 10 ++++
.../spark/sql/rapids/GpuOrcFileFormat.scala | 12 ++++-
.../spark/sql/rapids/OrcFilterSuite.scala | 49 +++++++++++++-----
9 files changed, 156 insertions(+), 42 deletions(-)
diff --git a/integration_tests/src/main/python/datasourcev2_write_test.py b/integration_tests/src/main/python/datasourcev2_write_test.py
index 1f4bc133d2a..4fffd10ab44 100644
--- a/integration_tests/src/main/python/datasourcev2_write_test.py
+++ b/integration_tests/src/main/python/datasourcev2_write_test.py
@@ -18,7 +18,7 @@
from data_gen import gen_df, decimal_gens, non_utc_allow
from marks import *
from spark_session import is_hive_available, is_spark_330_or_later, with_cpu_session, with_gpu_session
-from hive_parquet_write_test import _hive_bucket_gens, _hive_array_gens, _hive_struct_gens
+from hive_parquet_write_test import _hive_bucket_gens_sans_bools, _hive_array_gens, _hive_struct_gens
from hive_parquet_write_test import read_single_bucket
_hive_write_conf = {
@@ -33,9 +33,11 @@
@allow_non_gpu(*non_utc_allow)
def test_write_hive_bucketed_table(spark_tmp_table_factory, file_format):
num_rows = 2048
-
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen
def gen_table(spark):
- gen_list = [('_c' + str(i), gen) for i, gen in enumerate(_hive_bucket_gens)]
+ gen_list = [('_c' + str(i), gen) for i, gen in enumerate(_hive_bucket_gens_sans_bools)]
types_sql_str = ','.join('{} {}'.format(
name, gen.data_type.simpleString()) for name, gen in gen_list)
col_names_str = ','.join(name for name, gen in gen_list)
diff --git a/integration_tests/src/main/python/hive_parquet_write_test.py b/integration_tests/src/main/python/hive_parquet_write_test.py
index e66b889a986..540db74a1ad 100644
--- a/integration_tests/src/main/python/hive_parquet_write_test.py
+++ b/integration_tests/src/main/python/hive_parquet_write_test.py
@@ -25,9 +25,10 @@
# "GpuInsertIntoHiveTable" for Parquet write.
_write_to_hive_conf = {"spark.sql.hive.convertMetastoreParquet": False}
-_hive_bucket_gens = [
- boolean_gen, byte_gen, short_gen, int_gen, long_gen, string_gen, float_gen, double_gen,
+_hive_bucket_gens_sans_bools = [
+ byte_gen, short_gen, int_gen, long_gen, string_gen, float_gen, double_gen,
DateGen(start=date(1590, 1, 1)), _restricted_timestamp()]
+_hive_bucket_gens = [boolean_gen] + _hive_bucket_gens_sans_bools
_hive_basic_gens = _hive_bucket_gens + [
DecimalGen(precision=19, scale=1, nullable=True),
diff --git a/integration_tests/src/main/python/hive_write_test.py b/integration_tests/src/main/python/hive_write_test.py
index 945cc4806fb..af825a99810 100644
--- a/integration_tests/src/main/python/hive_write_test.py
+++ b/integration_tests/src/main/python/hive_write_test.py
@@ -29,8 +29,11 @@ def _restricted_timestamp(nullable=True):
end=datetime(2262, 4, 11, tzinfo=timezone.utc),
nullable=nullable)
+# Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+# https://github.com/rapidsai/cudf/issues/6763 .
+# Once the first issue is fixed, add back boolean_gen
_basic_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen,
- string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)),
+ string_gen, DateGen(start=date(1590, 1, 1)),
_restricted_timestamp()
] + decimal_gens
@@ -45,8 +48,11 @@ def _restricted_timestamp(nullable=True):
ArrayGen(ArrayGen(string_gen, max_length=10), max_length=10),
ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]]))]
+# Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+# https://github.com/rapidsai/cudf/issues/6763 .
+# Once the first issue is fixed, add back boolean_gen
_map_gens = [simple_string_to_string_map_gen] + [MapGen(f(nullable=False), f()) for f in [
- BooleanGen, ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen,
+ ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen,
lambda nullable=True: _restricted_timestamp(nullable=nullable),
lambda nullable=True: DateGen(start=date(1590, 1, 1), nullable=nullable),
lambda nullable=True: DecimalGen(precision=15, scale=1, nullable=nullable),
diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py
index 618004ee60d..19894d29aa6 100644
--- a/integration_tests/src/main/python/orc_test.py
+++ b/integration_tests/src/main/python/orc_test.py
@@ -1,4 +1,4 @@
-# Copyright (c) 2020-2023, NVIDIA CORPORATION.
+# Copyright (c) 2020-2024, NVIDIA CORPORATION.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@@ -112,8 +112,11 @@ def test_basic_read(std_input_path, name, read_func, v1_enabled_list, orc_impl,
#E at org.apache.orc.TypeDescription.parseInt(TypeDescription.java:244)
#E at org.apache.orc.TypeDescription.parseType(TypeDescription.java:362)
# ...
+# Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+# https://github.com/rapidsai/cudf/issues/6763 .
+# Once the first issue is fixed, add back boolean_gen
orc_basic_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen,
- string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)),
+ string_gen, DateGen(start=date(1590, 1, 1)),
orc_timestamp_gen] + decimal_gens
orc_basic_struct_gen = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(orc_basic_gens)])
@@ -201,8 +204,11 @@ def test_read_round_trip(spark_tmp_path, orc_gens, read_func, reader_confs, v1_e
read_func(data_path),
conf=all_confs)
+# Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+# https://github.com/rapidsai/cudf/issues/6763 .
+# Once the first issue is fixed, add back boolean_gen
orc_pred_push_gens = [
- byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, boolean_gen,
+ byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen,
string_gen,
# Once https://github.com/NVIDIA/spark-rapids/issues/139 is fixed replace this with
# date_gen
@@ -277,8 +283,11 @@ def test_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list, rea
def test_simple_partitioned_read(spark_tmp_path, v1_enabled_list, reader_confs):
# Once https://github.com/NVIDIA/spark-rapids/issues/131 is fixed
# we should go with a more standard set of generators
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen
orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen,
- string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)),
+ string_gen, DateGen(start=date(1590, 1, 1)),
orc_timestamp_gen]
gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)]
first_data_path = spark_tmp_path + '/ORC_DATA/key=0/key2=20'
@@ -344,8 +353,11 @@ def test_partitioned_read_just_partitions(spark_tmp_path, v1_enabled_list, reade
def test_merge_schema_read(spark_tmp_path, v1_enabled_list, reader_confs):
# Once https://github.com/NVIDIA/spark-rapids/issues/131 is fixed
# we should go with a more standard set of generators
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen
orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen,
- string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)),
+ string_gen, DateGen(start=date(1590, 1, 1)),
orc_timestamp_gen]
first_gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)]
first_data_path = spark_tmp_path + '/ORC_DATA/key=0'
@@ -825,8 +837,11 @@ def test_read_round_trip_for_multithreaded_combining(spark_tmp_path, gens, keep_
@pytest.mark.parametrize('keep_order', [True, pytest.param(False, marks=pytest.mark.ignore_order(local=True))])
@allow_non_gpu(*non_utc_allow_orc_scan)
def test_simple_partitioned_read_for_multithreaded_combining(spark_tmp_path, keep_order):
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen
orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen,
- string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)),
+ string_gen, DateGen(start=date(1590, 1, 1)),
orc_timestamp_gen]
gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)]
first_data_path = spark_tmp_path + '/ORC_DATA/key=0/key2=20'
@@ -927,7 +942,10 @@ def test_orc_column_name_with_dots(spark_tmp_path, reader_confs):
("f.g", int_gen),
("h", string_gen)])),
("i.j", long_gen)])),
- ("k", boolean_gen)]
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen for column k
+ ("k", int_gen)]
with_cpu_session(lambda spark: gen_df(spark, gens).write.orc(data_path))
assert_gpu_and_cpu_are_equal_collect(lambda spark: reader(spark), conf=all_confs)
assert_gpu_and_cpu_are_equal_collect(lambda spark: reader(spark).selectExpr("`a.b`"), conf=all_confs)
@@ -945,7 +963,10 @@ def test_orc_with_null_column(spark_tmp_path, reader_confs):
def gen_null_df(spark):
return spark.createDataFrame(
[(None, None, None, None, None)],
- "c1 int, c2 long, c3 float, c4 double, c5 boolean")
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen
+ "c1 int, c2 long, c3 float, c4 double, c5 int")
assert_gpu_and_cpu_writes_are_equal_collect(
lambda spark, path: gen_null_df(spark).write.orc(path),
@@ -966,7 +987,10 @@ def test_orc_with_null_column_with_1m_rows(spark_tmp_path, reader_confs):
def gen_null_df(spark):
return spark.createDataFrame(
data,
- "c1 int, c2 long, c3 float, c4 double, c5 boolean")
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen
+ "c1 int, c2 long, c3 float, c4 double, c5 int")
assert_gpu_and_cpu_writes_are_equal_collect(
lambda spark, path: gen_null_df(spark).write.orc(path),
lambda spark, path: spark.read.orc(path),
diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py
index ddb69524ac4..7e415c79a46 100644
--- a/integration_tests/src/main/python/orc_write_test.py
+++ b/integration_tests/src/main/python/orc_write_test.py
@@ -24,9 +24,11 @@
from pyspark.sql.types import *
pytestmark = pytest.mark.nightly_resource_consuming_test
-
+# Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+# https://github.com/rapidsai/cudf/issues/6763 .
+# Once the first issue is fixed, add back boolean_gen.
orc_write_basic_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen,
- string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)),
+ string_gen, DateGen(start=date(1590, 1, 1)),
TimestampGen(start=datetime(1970, 1, 1, tzinfo=timezone.utc)) ] + \
decimal_gens
@@ -52,7 +54,8 @@
all_nulls_map_gen,
all_empty_map_gen]
-orc_write_basic_struct_gen = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(orc_write_basic_gens)])
+orc_write_basic_struct_gen = StructGen(
+ [['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(orc_write_basic_gens)])
orc_write_struct_gens_sample = [orc_write_basic_struct_gen,
StructGen([['child0', byte_gen], ['child1', orc_write_basic_struct_gen]]),
@@ -62,15 +65,18 @@
ArrayGen(ArrayGen(short_gen, max_length=10), max_length=10),
ArrayGen(ArrayGen(string_gen, max_length=10), max_length=10),
ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]]))]
-
+# Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+# https://github.com/rapidsai/cudf/issues/6763 .
+# Once the first issue is fixed, add back boolean_gen.
orc_write_basic_map_gens = [simple_string_to_string_map_gen] + [MapGen(f(nullable=False), f()) for f in [
- BooleanGen, ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen,
+ ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen,
# Using timestamps from 1970 to work around a cudf ORC bug
# https://github.com/NVIDIA/spark-rapids/issues/140.
lambda nullable=True: TimestampGen(start=datetime(1970, 1, 1, tzinfo=timezone.utc), nullable=nullable),
lambda nullable=True: DateGen(start=date(1590, 1, 1), nullable=nullable),
lambda nullable=True: DecimalGen(precision=15, scale=1, nullable=nullable),
- lambda nullable=True: DecimalGen(precision=36, scale=5, nullable=nullable)]]
+ lambda nullable=True: DecimalGen(precision=36, scale=5, nullable=nullable)]] + [MapGen(
+ f(nullable=False), f(nullable=False)) for f in [IntegerGen]]
orc_write_gens_list = [orc_write_basic_gens,
orc_write_struct_gens_sample,
@@ -79,6 +85,7 @@
pytest.param([date_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/139')),
pytest.param([timestamp_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/140'))]
+bool_gen = [BooleanGen(nullable=True), BooleanGen(nullable=False)]
@pytest.mark.parametrize('orc_gens', orc_write_gens_list, ids=idfn)
@pytest.mark.parametrize('orc_impl', ["native", "hive"])
@allow_non_gpu(*non_utc_allow)
@@ -91,6 +98,30 @@ def test_write_round_trip(spark_tmp_path, orc_gens, orc_impl):
data_path,
conf={'spark.sql.orc.impl': orc_impl, 'spark.rapids.sql.format.orc.write.enabled': True})
+@pytest.mark.parametrize('orc_gens', [bool_gen], ids=idfn)
+@pytest.mark.parametrize('orc_impl', ["native", "hive"])
+@allow_non_gpu('ExecutedCommandExec', 'DataWritingCommandExec', 'WriteFilesExec')
+def test_write_round_trip_bools_only_fallback(spark_tmp_path, orc_gens, orc_impl):
+ gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)]
+ data_path = spark_tmp_path + '/ORC_DATA'
+ assert_gpu_and_cpu_writes_are_equal_collect(
+ lambda spark, path: gen_df(spark, gen_list).coalesce(1).write.orc(path),
+ lambda spark, path: spark.read.orc(path),
+ data_path,
+ conf={'spark.sql.orc.impl': orc_impl, 'spark.rapids.sql.format.orc.write.enabled': True})
+
+@pytest.mark.parametrize('orc_gens', [bool_gen], ids=idfn)
+@pytest.mark.parametrize('orc_impl', ["native", "hive"])
+def test_write_round_trip_bools_only_no_fallback(spark_tmp_path, orc_gens, orc_impl):
+ gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)]
+ data_path = spark_tmp_path + '/ORC_DATA'
+ assert_gpu_and_cpu_writes_are_equal_collect(
+ lambda spark, path: gen_df(spark, gen_list).coalesce(1).write.orc(path),
+ lambda spark, path: spark.read.orc(path),
+ data_path,
+ conf={'spark.sql.orc.impl': orc_impl, 'spark.rapids.sql.format.orc.write.enabled': True,
+ 'spark.rapids.sql.format.orc.write.boolType.enabled': True})
+
@pytest.mark.parametrize('orc_gen', orc_write_odd_empty_strings_gens_sample, ids=idfn)
@pytest.mark.parametrize('orc_impl', ["native", "hive"])
def test_write_round_trip_corner(spark_tmp_path, orc_gen, orc_impl):
@@ -103,7 +134,8 @@ def test_write_round_trip_corner(spark_tmp_path, orc_gen, orc_impl):
conf={'spark.sql.orc.impl': orc_impl, 'spark.rapids.sql.format.orc.write.enabled': True})
orc_part_write_gens = [
- byte_gen, short_gen, int_gen, long_gen, boolean_gen,
+ # Add back boolean_gen when https://github.com/rapidsai/cudf/issues/6763 is fixed
+ byte_gen, short_gen, int_gen, long_gen,
# Some file systems have issues with UTF8 strings so to help the test pass even there
StringGen('(\\w| ){0,50}'),
# Once https://github.com/NVIDIA/spark-rapids/issues/139 is fixed replace this with
@@ -345,7 +377,10 @@ def test_orc_write_column_name_with_dots(spark_tmp_path):
("f.g", int_gen),
("h", string_gen)])),
("i.j", long_gen)])),
- ("k", boolean_gen)]
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen for column k
+ ("k", int_gen)]
assert_gpu_and_cpu_writes_are_equal_collect(
lambda spark, path: gen_df(spark, gens).coalesce(1).write.orc(path),
lambda spark, path: spark.read.orc(path),
diff --git a/integration_tests/src/main/python/schema_evolution_test.py b/integration_tests/src/main/python/schema_evolution_test.py
index ff501324cc0..57af4a1126e 100644
--- a/integration_tests/src/main/python/schema_evolution_test.py
+++ b/integration_tests/src/main/python/schema_evolution_test.py
@@ -1,4 +1,4 @@
-# Copyright (c) 2023, NVIDIA CORPORATION.
+# Copyright (c) 2023-2024, NVIDIA CORPORATION.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@@ -34,7 +34,9 @@
# List of additional column data generators to use when adding columns
_additional_gens = [
- boolean_gen,
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen
byte_gen,
short_gen,
int_gen,
@@ -49,7 +51,10 @@
# simple_string_to_string_map_gen),
ArrayGen(_custom_date_gen),
struct_gen_decimal128,
- StructGen([("c0", ArrayGen(long_gen)), ("c1", boolean_gen)]),
+ # Use every type except boolean, see https://github.com/NVIDIA/spark-rapids/issues/11762 and
+ # https://github.com/rapidsai/cudf/issues/6763 .
+ # Once the first issue is fixed, add back boolean_gen from int_gen for c1
+ StructGen([("c0", ArrayGen(long_gen)), ("c1", int_gen)]),
]
def get_additional_columns():
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
index 406aeb0365b..e750f5688ce 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
@@ -1268,6 +1268,14 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern")
.booleanConf
.createWithDefault(true)
+ val ENABLE_ORC_BOOL = conf("spark.rapids.sql.format.orc.write.boolType.enabled")
+ .doc("When set to false disables boolean columns for ORC writes. " +
+ "Set to true if you want to experiment. " +
+ "See https://github.com/NVIDIA/spark-rapids/issues/11736.")
+ .internal()
+ .booleanConf
+ .createWithDefault(false)
+
val ENABLE_EXPAND_PREPROJECT = conf("spark.rapids.sql.expandPreproject.enabled")
.doc("When set to false disables the pre-projection for GPU Expand. " +
"Pre-projection leverages the tiered projection to evaluate expressions that " +
@@ -3028,6 +3036,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging {
lazy val maxNumOrcFilesParallel: Int = get(ORC_MULTITHREAD_READ_MAX_NUM_FILES_PARALLEL)
+ lazy val isOrcBoolTypeEnabled: Boolean = get(ENABLE_ORC_BOOL)
+
lazy val isCsvEnabled: Boolean = get(ENABLE_CSV)
lazy val isCsvReadEnabled: Boolean = get(ENABLE_CSV_READ)
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala
index d2f4380646c..1d4bc66a1da 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2020-2023, NVIDIA CORPORATION.
+ * Copyright (c) 2020-2024, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.execution.datasources.FileFormat
import org.apache.spark.sql.execution.datasources.orc.{OrcFileFormat, OrcOptions, OrcUtils}
import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.rapids.execution.TrampolineUtil
import org.apache.spark.sql.types._
object GpuOrcFileFormat extends Logging {
@@ -83,6 +84,11 @@ object GpuOrcFileFormat extends Logging {
// [[org.apache.spark.sql.execution.datasources.DaysWritable]] object
// which is a subclass of [[org.apache.hadoop.hive.serde2.io.DateWritable]].
val types = schema.map(_.dataType).toSet
+ val hasBools = schema.exists { field =>
+ TrampolineUtil.dataTypeExistsRecursively(field.dataType, t =>
+ t.isInstanceOf[BooleanType])
+ }
+
if (types.exists(GpuOverrides.isOrContainsDateOrTimestamp(_))) {
if (!GpuOverrides.isUTCTimezone()) {
meta.willNotWorkOnGpu("Only UTC timezone is supported for ORC. " +
@@ -91,6 +97,10 @@ object GpuOrcFileFormat extends Logging {
}
}
+ if (hasBools && !meta.conf.isOrcBoolTypeEnabled) {
+ meta.willNotWorkOnGpu("Nullable Booleans can not work in certain cases with ORC writer." +
+ "See https://github.com/rapidsai/cudf/issues/6763")
+ }
FileFormatChecks.tag(meta, schema, OrcFormatType, WriteFileOp)
val sqlConf = spark.sessionState.conf
diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala
index fe86900b32f..6d067800dde 100644
--- a/tests/src/test/scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala
+++ b/tests/src/test/scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2023, NVIDIA CORPORATION.
+ * Copyright (c) 2023-2024, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -18,10 +18,11 @@ package org.apache.spark.sql.rapids
import java.sql.Timestamp
-import com.nvidia.spark.rapids.{GpuFilterExec, SparkQueryCompareTestSuite}
+import com.nvidia.spark.rapids.{GpuFilterExec, RapidsConf, SparkQueryCompareTestSuite}
+import org.apache.spark.SparkConf
import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.FilterExec
+import org.apache.spark.sql.execution.{FilterExec, SparkPlan}
class OrcFilterSuite extends SparkQueryCompareTestSuite {
@@ -39,22 +40,42 @@ class OrcFilterSuite extends SparkQueryCompareTestSuite {
test("Support for pushing down filters for boolean types gpu write gpu read") {
withTempPath { file =>
- withGpuSparkSession(spark => {
- val data = (0 until 10).map(i => Tuple1(i == 2))
- val df = spark.createDataFrame(data).toDF("a")
- df.repartition(10).write.orc(file.getCanonicalPath)
- checkPredicatePushDown(spark, file.getCanonicalPath, 10, "a == true")
- })
+ var gpuPlans: Array[SparkPlan] = Array.empty
+ val testConf = new SparkConf().set(
+ RapidsConf.TEST_ALLOWED_NONGPU.key,
+ "DataWritingCommandExec,ShuffleExchangeExec, WriteFilesExec")
+ ExecutionPlanCaptureCallback.startCapture()
+ try {
+ withGpuSparkSession(spark => {
+ val data = (0 until 10).map(i => Tuple1(i == 2))
+ val df = spark.createDataFrame(data).toDF("a")
+ df.repartition(10).write.orc(file.getCanonicalPath)
+ checkPredicatePushDown(spark, file.getCanonicalPath, 10, "a == true")
+ }, testConf)
+ } finally {
+ gpuPlans = ExecutionPlanCaptureCallback.getResultsWithTimeout()
+ }
+ ExecutionPlanCaptureCallback.assertDidFallBack(gpuPlans.head, "DataWritingCommandExec")
}
}
test("Support for pushing down filters for boolean types gpu write cpu read") {
withTempPath { file =>
- withGpuSparkSession(spark => {
- val data = (0 until 10).map(i => Tuple1(i == 2))
- val df = spark.createDataFrame(data).toDF("a")
- df.repartition(10).write.orc(file.getCanonicalPath)
- })
+ var gpuPlans: Array[SparkPlan] = Array.empty
+ val testConf = new SparkConf().set(
+ RapidsConf.TEST_ALLOWED_NONGPU.key,
+ "DataWritingCommandExec,ShuffleExchangeExec, WriteFilesExec")
+ ExecutionPlanCaptureCallback.startCapture()
+ try {
+ withGpuSparkSession(spark => {
+ val data = (0 until 10).map(i => Tuple1(i == 2))
+ val df = spark.createDataFrame(data).toDF("a")
+ df.repartition(10).write.orc(file.getCanonicalPath)
+ }, testConf)
+ } finally {
+ gpuPlans = ExecutionPlanCaptureCallback.getResultsWithTimeout()
+ }
+ ExecutionPlanCaptureCallback.assertDidFallBack(gpuPlans.head, "DataWritingCommandExec")
withCpuSparkSession(spark => {
checkPredicatePushDown(spark, file.getCanonicalPath, 10, "a == true")
})
From 3449c8a772899ff1752ea42efa89f1ddc049cc6a Mon Sep 17 00:00:00 2001
From: Alessandro Bellina
Date: Sun, 8 Dec 2024 15:45:53 -0600
Subject: [PATCH 094/103] Fixes a leak for the empty nlj iterator (#11832)
Signed-off-by: Alessandro Bellina
---
.../GpuBroadcastNestedLoopJoinExecBase.scala | 12 ++++++++----
1 file changed, 8 insertions(+), 4 deletions(-)
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala
index 578c1106eb1..b939a8c4155 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExecBase.scala
@@ -658,11 +658,15 @@ abstract class GpuBroadcastNestedLoopJoinExecBase(
localJoinType match {
case LeftOuter if spillableBuiltBatch.numRows == 0 =>
- new EmptyOuterNestedLoopJoinIterator(streamedIter, spillableBuiltBatch.dataTypes,
- true)
+ withResource(spillableBuiltBatch) { _ =>
+ new EmptyOuterNestedLoopJoinIterator(streamedIter, spillableBuiltBatch.dataTypes,
+ true)
+ }
case RightOuter if spillableBuiltBatch.numRows == 0 =>
- new EmptyOuterNestedLoopJoinIterator(streamedIter, spillableBuiltBatch.dataTypes,
- false)
+ withResource(spillableBuiltBatch) { _ =>
+ new EmptyOuterNestedLoopJoinIterator(streamedIter, spillableBuiltBatch.dataTypes,
+ false)
+ }
case _ =>
new CrossJoinIterator(
spillableBuiltBatch,
From 45cdac34667638b4d29e0ec5aab663d2588e3f26 Mon Sep 17 00:00:00 2001
From: MithunR
Date: Mon, 9 Dec 2024 10:10:03 -0800
Subject: [PATCH 095/103] Fix for lead/lag window test failures. (#11823)
Fixes #11807.
`test_lead_lag_for_structs_with_arrays` in `window_function_test` fails intermittently because of
non-deterministic data ordering.
Window function tests are sensitive to data ordering. With certain values of DATAGEN_SEED,
there are repeated values of partitioning/ordering keys, causing the window function
to return different values on CPU and GPU.
This commit fixes the test so that the ordering is deterministic.
Signed-off-by: MithunR
---
.../src/main/python/window_function_test.py | 10 ++++------
1 file changed, 4 insertions(+), 6 deletions(-)
diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py
index 653eaffa940..7695c1adc9d 100644
--- a/integration_tests/src/main/python/window_function_test.py
+++ b/integration_tests/src/main/python/window_function_test.py
@@ -971,14 +971,12 @@ def do_it(spark):
def test_lead_lag_for_structs_with_arrays(a_b_gen, struct_gen):
data_gen = [
('a', RepeatSeqGen(a_b_gen, length=20)),
- ('b', IntegerGen(nullable=False, special_cases=[])),
+ ('b', UniqueLongGen(nullable=False)),
('c', struct_gen)]
- # By default for many operations a range of unbounded to unbounded is used
- # This will not work until https://github.com/NVIDIA/spark-rapids/issues/216
- # is fixed.
+ # For many operations, a range of unbounded to unbounded is used by default.
- # Ordering needs to include c because with nulls and especially on booleans
- # it is possible to get a different ordering when it is ambiguous.
+ # Ordering needs to include `b` because with nulls and especially on booleans,
+ # it is possible to get a different result when the ordering is ambiguous.
base_window_spec = Window.partitionBy('a').orderBy('b')
def do_it(spark):
From 96a58d121a5af7ef956196a6141fe7777277e95e Mon Sep 17 00:00:00 2001
From: Kuhu Shukla
Date: Mon, 9 Dec 2024 19:47:01 -0600
Subject: [PATCH 096/103] Fix leak in isTimeStamp (#11845)
Signed-off-by: Kuhu Shukla
---
.../org/apache/spark/sql/rapids/datetimeExpressions.scala | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
index 0f382a7b6e6..d08c598cba4 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala
@@ -707,7 +707,9 @@ object GpuToTimestamp {
case _ =>
// this is the incompatibleDateFormats case where we do not guarantee compatibility with
// Spark and assume that all non-null inputs are valid
- ColumnVector.fromScalar(Scalar.fromBool(true), col.getRowCount.toInt)
+ withResource(Scalar.fromBool(true)) { s =>
+ ColumnVector.fromScalar(s, col.getRowCount.toInt)
+ }
}
}
From 1c540c14f895a23399bf36cbe045518706bfa0dc Mon Sep 17 00:00:00 2001
From: nvauto <70000568+nvauto@users.noreply.github.com>
Date: Tue, 10 Dec 2024 07:35:56 +0000
Subject: [PATCH 097/103] Change version to 24.12.0
Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com>
---
CONTRIBUTING.md | 8 ++++----
README.md | 2 +-
aggregator/pom.xml | 4 ++--
api_validation/pom.xml | 4 ++--
datagen/ScaleTest.md | 2 +-
datagen/pom.xml | 4 ++--
delta-lake/delta-20x/pom.xml | 4 ++--
delta-lake/delta-21x/pom.xml | 4 ++--
delta-lake/delta-22x/pom.xml | 4 ++--
delta-lake/delta-23x/pom.xml | 4 ++--
delta-lake/delta-24x/pom.xml | 4 ++--
delta-lake/delta-spark330db/pom.xml | 4 ++--
delta-lake/delta-spark332db/pom.xml | 4 ++--
delta-lake/delta-spark341db/pom.xml | 4 ++--
delta-lake/delta-spark350db143/pom.xml | 4 ++--
delta-lake/delta-stub/pom.xml | 4 ++--
dist/pom.xml | 4 ++--
docs/configs.md | 2 +-
integration_tests/README.md | 6 +++---
integration_tests/ScaleTest.md | 2 +-
integration_tests/pom.xml | 4 ++--
jdk-profiles/pom.xml | 4 ++--
jenkins/databricks/create.py | 2 +-
jenkins/version-def.sh | 4 ++--
pom.xml | 2 +-
scala2.13/aggregator/pom.xml | 4 ++--
scala2.13/api_validation/pom.xml | 4 ++--
scala2.13/datagen/pom.xml | 4 ++--
scala2.13/delta-lake/delta-20x/pom.xml | 4 ++--
scala2.13/delta-lake/delta-21x/pom.xml | 4 ++--
scala2.13/delta-lake/delta-22x/pom.xml | 4 ++--
scala2.13/delta-lake/delta-23x/pom.xml | 4 ++--
scala2.13/delta-lake/delta-24x/pom.xml | 4 ++--
scala2.13/delta-lake/delta-spark330db/pom.xml | 4 ++--
scala2.13/delta-lake/delta-spark332db/pom.xml | 4 ++--
scala2.13/delta-lake/delta-spark341db/pom.xml | 4 ++--
scala2.13/delta-lake/delta-spark350db143/pom.xml | 4 ++--
scala2.13/delta-lake/delta-stub/pom.xml | 4 ++--
scala2.13/dist/pom.xml | 4 ++--
scala2.13/integration_tests/pom.xml | 4 ++--
scala2.13/jdk-profiles/pom.xml | 4 ++--
scala2.13/pom.xml | 2 +-
scala2.13/shim-deps/cloudera/pom.xml | 4 ++--
scala2.13/shim-deps/databricks/pom.xml | 4 ++--
scala2.13/shim-deps/pom.xml | 4 ++--
scala2.13/shuffle-plugin/pom.xml | 4 ++--
scala2.13/sql-plugin-api/pom.xml | 4 ++--
scala2.13/sql-plugin/pom.xml | 4 ++--
scala2.13/tests/pom.xml | 4 ++--
scala2.13/tools/pom.xml | 4 ++--
scala2.13/udf-compiler/pom.xml | 4 ++--
shim-deps/cloudera/pom.xml | 4 ++--
shim-deps/databricks/pom.xml | 4 ++--
shim-deps/pom.xml | 4 ++--
shuffle-plugin/pom.xml | 4 ++--
sql-plugin-api/pom.xml | 4 ++--
sql-plugin/pom.xml | 4 ++--
.../main/scala/com/nvidia/spark/rapids/RapidsConf.scala | 2 +-
tests/pom.xml | 4 ++--
tools/pom.xml | 4 ++--
udf-compiler/pom.xml | 4 ++--
61 files changed, 117 insertions(+), 117 deletions(-)
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index 83b30747abd..21f31ba1498 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -127,15 +127,15 @@ mvn -pl dist -PnoSnapshots package -DskipTests
Verify that shim-specific classes are hidden from a conventional classloader.
```bash
-$ javap -cp dist/target/rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar com.nvidia.spark.rapids.shims.SparkShimImpl
+$ javap -cp dist/target/rapids-4-spark_2.12-24.12.0-cuda11.jar com.nvidia.spark.rapids.shims.SparkShimImpl
Error: class not found: com.nvidia.spark.rapids.shims.SparkShimImpl
```
However, its bytecode can be loaded if prefixed with `spark3XY` not contained in the package name
```bash
-$ javap -cp dist/target/rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar spark320.com.nvidia.spark.rapids.shims.SparkShimImpl | head -2
-Warning: File dist/target/rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar(/spark320/com/nvidia/spark/rapids/shims/SparkShimImpl.class) does not contain class spark320.com.nvidia.spark.rapids.shims.SparkShimImpl
+$ javap -cp dist/target/rapids-4-spark_2.12-24.12.0-cuda11.jar spark320.com.nvidia.spark.rapids.shims.SparkShimImpl | head -2
+Warning: File dist/target/rapids-4-spark_2.12-24.12.0-cuda11.jar(/spark320/com/nvidia/spark/rapids/shims/SparkShimImpl.class) does not contain class spark320.com.nvidia.spark.rapids.shims.SparkShimImpl
Compiled from "SparkShims.scala"
public final class com.nvidia.spark.rapids.shims.SparkShimImpl {
```
@@ -178,7 +178,7 @@ mvn package -pl dist -am -Dbuildver=340 -DallowConventionalDistJar=true
Verify `com.nvidia.spark.rapids.shims.SparkShimImpl` is conventionally loadable:
```bash
-$ javap -cp dist/target/rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar com.nvidia.spark.rapids.shims.SparkShimImpl | head -2
+$ javap -cp dist/target/rapids-4-spark_2.12-24.12.0-cuda11.jar com.nvidia.spark.rapids.shims.SparkShimImpl | head -2
Compiled from "SparkShims.scala"
public final class com.nvidia.spark.rapids.shims.SparkShimImpl {
```
diff --git a/README.md b/README.md
index 94b73565190..65e194de3c2 100644
--- a/README.md
+++ b/README.md
@@ -73,7 +73,7 @@ as a `provided` dependency.
com.nvidia
rapids-4-spark_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
provided
```
diff --git a/aggregator/pom.xml b/aggregator/pom.xml
index c7a6c220247..1ba28e86568 100644
--- a/aggregator/pom.xml
+++ b/aggregator/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../jdk-profiles/pom.xml
rapids-4-spark-aggregator_2.12
RAPIDS Accelerator for Apache Spark Aggregator
Creates an aggregated shaded package of the RAPIDS plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
aggregator
diff --git a/api_validation/pom.xml b/api_validation/pom.xml
index cddcf0c1ce1..697fb4c7759 100644
--- a/api_validation/pom.xml
+++ b/api_validation/pom.xml
@@ -22,11 +22,11 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-api-validation_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
api_validation
diff --git a/datagen/ScaleTest.md b/datagen/ScaleTest.md
index a728ad9a13e..bb5c4a1c988 100644
--- a/datagen/ScaleTest.md
+++ b/datagen/ScaleTest.md
@@ -44,7 +44,7 @@ $SPARK_HOME/bin/spark-submit \
--conf spark.sql.parquet.datetimeRebaseModeInWrite=CORRECTED \
--class com.nvidia.rapids.tests.scaletest.ScaleTestDataGen \ # the main class
--jars $SPARK_HOME/examples/jars/scopt_2.12-3.7.1.jar \ # one dependency jar just shipped with Spark under $SPARK_HOME
-./target/datagen_2.12-24.12.0-SNAPSHOT-spark332.jar \
+./target/datagen_2.12-24.12.0-spark332.jar \
1 \
10 \
parquet \
diff --git a/datagen/pom.xml b/datagen/pom.xml
index 9bdf897cfd7..64e48ff9c85 100644
--- a/datagen/pom.xml
+++ b/datagen/pom.xml
@@ -21,13 +21,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
datagen_2.12
Data Generator
Tools for generating large amounts of data
- 24.12.0-SNAPSHOT
+ 24.12.0
datagen
diff --git a/delta-lake/delta-20x/pom.xml b/delta-lake/delta-20x/pom.xml
index 1d41911c767..a07d81e0f25 100644
--- a/delta-lake/delta-20x/pom.xml
+++ b/delta-lake/delta-20x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-20x_2.12
RAPIDS Accelerator for Apache Spark Delta Lake 2.0.x Support
Delta Lake 2.0.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-20x
diff --git a/delta-lake/delta-21x/pom.xml b/delta-lake/delta-21x/pom.xml
index 7514088ca3a..3ad3e3c83fc 100644
--- a/delta-lake/delta-21x/pom.xml
+++ b/delta-lake/delta-21x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-21x_2.12
RAPIDS Accelerator for Apache Spark Delta Lake 2.1.x Support
Delta Lake 2.1.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-21x
diff --git a/delta-lake/delta-22x/pom.xml b/delta-lake/delta-22x/pom.xml
index 2ed0ea3b159..5d4d389b097 100644
--- a/delta-lake/delta-22x/pom.xml
+++ b/delta-lake/delta-22x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-22x_2.12
RAPIDS Accelerator for Apache Spark Delta Lake 2.2.x Support
Delta Lake 2.2.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-22x
diff --git a/delta-lake/delta-23x/pom.xml b/delta-lake/delta-23x/pom.xml
index 1daebdd0efb..ea394bd26b2 100644
--- a/delta-lake/delta-23x/pom.xml
+++ b/delta-lake/delta-23x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../pom.xml
rapids-4-spark-delta-23x_2.12
RAPIDS Accelerator for Apache Spark Delta Lake 2.3.x Support
Delta Lake 2.3.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-23x
diff --git a/delta-lake/delta-24x/pom.xml b/delta-lake/delta-24x/pom.xml
index 36ec92b70c0..ee1c7926245 100644
--- a/delta-lake/delta-24x/pom.xml
+++ b/delta-lake/delta-24x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-24x_2.12
RAPIDS Accelerator for Apache Spark Delta Lake 2.4.x Support
Delta Lake 2.4.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-24x
diff --git a/delta-lake/delta-spark330db/pom.xml b/delta-lake/delta-spark330db/pom.xml
index 95f54c6807c..e5f60afb125 100644
--- a/delta-lake/delta-spark330db/pom.xml
+++ b/delta-lake/delta-spark330db/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../shim-deps/pom.xml
rapids-4-spark-delta-spark330db_2.12
RAPIDS Accelerator for Apache Spark Databricks 11.3 Delta Lake Support
Databricks 11.3 Delta Lake support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-spark330db
diff --git a/delta-lake/delta-spark332db/pom.xml b/delta-lake/delta-spark332db/pom.xml
index 4d792ee1ca5..102c91daf82 100644
--- a/delta-lake/delta-spark332db/pom.xml
+++ b/delta-lake/delta-spark332db/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../shim-deps/pom.xml
rapids-4-spark-delta-spark332db_2.12
RAPIDS Accelerator for Apache Spark Databricks 12.2 Delta Lake Support
Databricks 12.2 Delta Lake support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-spark332db
diff --git a/delta-lake/delta-spark341db/pom.xml b/delta-lake/delta-spark341db/pom.xml
index 4b229e2e5b5..39e7c0b2dd4 100644
--- a/delta-lake/delta-spark341db/pom.xml
+++ b/delta-lake/delta-spark341db/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../shim-deps/pom.xml
rapids-4-spark-delta-spark341db_2.12
RAPIDS Accelerator for Apache Spark Databricks 13.3 Delta Lake Support
Databricks 13.3 Delta Lake support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
false
diff --git a/delta-lake/delta-spark350db143/pom.xml b/delta-lake/delta-spark350db143/pom.xml
index 1bca394b67c..333a035a680 100644
--- a/delta-lake/delta-spark350db143/pom.xml
+++ b/delta-lake/delta-spark350db143/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../shim-deps/pom.xml
rapids-4-spark-delta-spark350db143_2.12
RAPIDS Accelerator for Apache Spark Databricks 13.3 Delta Lake Support
Databricks 13.3 Delta Lake support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
false
diff --git a/delta-lake/delta-stub/pom.xml b/delta-lake/delta-stub/pom.xml
index 6d0471f9f01..deb2bb7233d 100644
--- a/delta-lake/delta-stub/pom.xml
+++ b/delta-lake/delta-stub/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-stub_2.12
RAPIDS Accelerator for Apache Spark Delta Lake Stub
Delta Lake stub for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-stub
diff --git a/dist/pom.xml b/dist/pom.xml
index d628dd4ba3b..f7d69b761f2 100644
--- a/dist/pom.xml
+++ b/dist/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../jdk-profiles/pom.xml
rapids-4-spark_2.12
RAPIDS Accelerator for Apache Spark Distribution
Creates the distribution package of the RAPIDS plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
com.nvidia
diff --git a/docs/configs.md b/docs/configs.md
index 7f9544496c4..75076bafe7c 100644
--- a/docs/configs.md
+++ b/docs/configs.md
@@ -10,7 +10,7 @@ The following is the list of options that `rapids-plugin-4-spark` supports.
On startup use: `--conf [conf key]=[conf value]`. For example:
```
-${SPARK_HOME}/bin/spark-shell --jars rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar \
+${SPARK_HOME}/bin/spark-shell --jars rapids-4-spark_2.12-24.12.0-cuda11.jar \
--conf spark.plugins=com.nvidia.spark.SQLPlugin \
--conf spark.rapids.sql.concurrentGpuTasks=2
```
diff --git a/integration_tests/README.md b/integration_tests/README.md
index f5237de21a0..f8b6d9510ff 100644
--- a/integration_tests/README.md
+++ b/integration_tests/README.md
@@ -263,7 +263,7 @@ individually, so you don't risk running unit tests along with the integration te
http://www.scalatest.org/user_guide/using_the_scalatest_shell
```shell
-spark-shell --jars rapids-4-spark-tests_2.12-24.12.0-SNAPSHOT-tests.jar,rapids-4-spark-integration-tests_2.12-24.12.0-SNAPSHOT-tests.jar,scalatest_2.12-3.0.5.jar,scalactic_2.12-3.0.5.jar
+spark-shell --jars rapids-4-spark-tests_2.12-24.12.0-tests.jar,rapids-4-spark-integration-tests_2.12-24.12.0-tests.jar,scalatest_2.12-3.0.5.jar,scalactic_2.12-3.0.5.jar
```
First you import the `scalatest_shell` and tell the tests where they can find the test files you
@@ -286,7 +286,7 @@ If you just want to verify the SQL replacement is working you will need to add t
assumes CUDA 11.0 is being used and the Spark distribution is built with Scala 2.12.
```
-$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar" ./runtests.py
+$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-24.12.0-cuda11.jar" ./runtests.py
```
You don't have to enable the plugin for this to work, the test framework will do that for you.
@@ -443,7 +443,7 @@ To run cudf_udf tests, need following configuration changes:
As an example, here is the `spark-submit` command with the cudf_udf parameter on CUDA 11.0:
```
-$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar,rapids-4-spark-tests_2.12-24.12.0-SNAPSHOT.jar" --conf spark.rapids.memory.gpu.allocFraction=0.3 --conf spark.rapids.python.memory.gpu.allocFraction=0.3 --conf spark.rapids.python.concurrentPythonWorkers=2 --py-files "rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar" --conf spark.executorEnv.PYTHONPATH="rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar" ./runtests.py --cudf_udf
+$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-24.12.0-cuda11.jar,rapids-4-spark-tests_2.12-24.12.0.jar" --conf spark.rapids.memory.gpu.allocFraction=0.3 --conf spark.rapids.python.memory.gpu.allocFraction=0.3 --conf spark.rapids.python.concurrentPythonWorkers=2 --py-files "rapids-4-spark_2.12-24.12.0-cuda11.jar" --conf spark.executorEnv.PYTHONPATH="rapids-4-spark_2.12-24.12.0-cuda11.jar" ./runtests.py --cudf_udf
```
### Enabling fuzz tests
diff --git a/integration_tests/ScaleTest.md b/integration_tests/ScaleTest.md
index d9f47fab5cb..bea34954cab 100644
--- a/integration_tests/ScaleTest.md
+++ b/integration_tests/ScaleTest.md
@@ -97,7 +97,7 @@ $SPARK_HOME/bin/spark-submit \
--conf spark.sql.parquet.datetimeRebaseModeInWrite=CORRECTED \
--jars $SPARK_HOME/examples/jars/scopt_2.12-3.7.1.jar \
--class com.nvidia.spark.rapids.tests.scaletest.ScaleTest \
-./target/rapids-4-spark-integration-tests_2.12-24.12.0-SNAPSHOT-spark332.jar \
+./target/rapids-4-spark-integration-tests_2.12-24.12.0-spark332.jar \
10 \
100 \
parquet \
diff --git a/integration_tests/pom.xml b/integration_tests/pom.xml
index aaff3455298..bac78bce0df 100644
--- a/integration_tests/pom.xml
+++ b/integration_tests/pom.xml
@@ -22,11 +22,11 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-integration-tests_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
integration_tests
diff --git a/jdk-profiles/pom.xml b/jdk-profiles/pom.xml
index caaa47245a8..689f3576b89 100644
--- a/jdk-profiles/pom.xml
+++ b/jdk-profiles/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
com.nvidia
rapids-4-spark-jdk-profiles_2.12
pom
Shim JDK Profiles
- 24.12.0-SNAPSHOT
+ 24.12.0
jdk8
diff --git a/jenkins/databricks/create.py b/jenkins/databricks/create.py
index 44c4c856466..4354886e5b7 100644
--- a/jenkins/databricks/create.py
+++ b/jenkins/databricks/create.py
@@ -27,7 +27,7 @@ def main():
workspace = 'https://dbc-9ff9942e-a9c4.cloud.databricks.com'
token = ''
sshkey = ''
- cluster_name = 'CI-GPU-databricks-24.12.0-SNAPSHOT'
+ cluster_name = 'CI-GPU-databricks-24.12.0'
idletime = 240
runtime = '13.3.x-gpu-ml-scala2.12'
num_workers = 1
diff --git a/jenkins/version-def.sh b/jenkins/version-def.sh
index 8600a2f8689..7acdd6204a5 100755
--- a/jenkins/version-def.sh
+++ b/jenkins/version-def.sh
@@ -29,8 +29,8 @@ IFS=$PRE_IFS
CUDA_CLASSIFIER=${CUDA_CLASSIFIER:-"cuda11"}
CLASSIFIER=${CLASSIFIER:-"$CUDA_CLASSIFIER"} # default as CUDA_CLASSIFIER for compatibility
-PROJECT_VER=${PROJECT_VER:-"24.12.0-SNAPSHOT"}
-PROJECT_TEST_VER=${PROJECT_TEST_VER:-"24.12.0-SNAPSHOT"}
+PROJECT_VER=${PROJECT_VER:-"24.12.0"}
+PROJECT_TEST_VER=${PROJECT_TEST_VER:-"24.12.0"}
SPARK_VER=${SPARK_VER:-"3.2.0"}
SPARK_VER_213=${SPARK_VER_213:-"3.3.0"}
# Make a best attempt to set the default value for the shuffle shim.
diff --git a/pom.xml b/pom.xml
index 12828404031..470f198e5fb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -23,7 +23,7 @@
rapids-4-spark-parent_2.12
RAPIDS Accelerator for Apache Spark Root Project
The root project of the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
pom
https://nvidia.github.io/spark-rapids/
diff --git a/scala2.13/aggregator/pom.xml b/scala2.13/aggregator/pom.xml
index 74956108068..053e9370deb 100644
--- a/scala2.13/aggregator/pom.xml
+++ b/scala2.13/aggregator/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../jdk-profiles/pom.xml
rapids-4-spark-aggregator_2.13
RAPIDS Accelerator for Apache Spark Aggregator
Creates an aggregated shaded package of the RAPIDS plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
aggregator
diff --git a/scala2.13/api_validation/pom.xml b/scala2.13/api_validation/pom.xml
index f236345c301..7e73ed7ae72 100644
--- a/scala2.13/api_validation/pom.xml
+++ b/scala2.13/api_validation/pom.xml
@@ -22,11 +22,11 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-api-validation_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
api_validation
diff --git a/scala2.13/datagen/pom.xml b/scala2.13/datagen/pom.xml
index d53ebc014c7..b5f638aa9cf 100644
--- a/scala2.13/datagen/pom.xml
+++ b/scala2.13/datagen/pom.xml
@@ -21,13 +21,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
datagen_2.13
Data Generator
Tools for generating large amounts of data
- 24.12.0-SNAPSHOT
+ 24.12.0
datagen
diff --git a/scala2.13/delta-lake/delta-20x/pom.xml b/scala2.13/delta-lake/delta-20x/pom.xml
index 20c77038f40..0eee940615f 100644
--- a/scala2.13/delta-lake/delta-20x/pom.xml
+++ b/scala2.13/delta-lake/delta-20x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-20x_2.13
RAPIDS Accelerator for Apache Spark Delta Lake 2.0.x Support
Delta Lake 2.0.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-20x
diff --git a/scala2.13/delta-lake/delta-21x/pom.xml b/scala2.13/delta-lake/delta-21x/pom.xml
index 75a41cfa8e0..2b457c80aea 100644
--- a/scala2.13/delta-lake/delta-21x/pom.xml
+++ b/scala2.13/delta-lake/delta-21x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-21x_2.13
RAPIDS Accelerator for Apache Spark Delta Lake 2.1.x Support
Delta Lake 2.1.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-21x
diff --git a/scala2.13/delta-lake/delta-22x/pom.xml b/scala2.13/delta-lake/delta-22x/pom.xml
index c6111eb51a0..42e5685444c 100644
--- a/scala2.13/delta-lake/delta-22x/pom.xml
+++ b/scala2.13/delta-lake/delta-22x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-22x_2.13
RAPIDS Accelerator for Apache Spark Delta Lake 2.2.x Support
Delta Lake 2.2.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-22x
diff --git a/scala2.13/delta-lake/delta-23x/pom.xml b/scala2.13/delta-lake/delta-23x/pom.xml
index 84d1d7275c2..05a38a75ce6 100644
--- a/scala2.13/delta-lake/delta-23x/pom.xml
+++ b/scala2.13/delta-lake/delta-23x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../pom.xml
rapids-4-spark-delta-23x_2.13
RAPIDS Accelerator for Apache Spark Delta Lake 2.3.x Support
Delta Lake 2.3.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-23x
diff --git a/scala2.13/delta-lake/delta-24x/pom.xml b/scala2.13/delta-lake/delta-24x/pom.xml
index 0ffe6c84e10..c1b106d0f55 100644
--- a/scala2.13/delta-lake/delta-24x/pom.xml
+++ b/scala2.13/delta-lake/delta-24x/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-24x_2.13
RAPIDS Accelerator for Apache Spark Delta Lake 2.4.x Support
Delta Lake 2.4.x support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-24x
diff --git a/scala2.13/delta-lake/delta-spark330db/pom.xml b/scala2.13/delta-lake/delta-spark330db/pom.xml
index 3c30b1b0dc8..2fa49fe5847 100644
--- a/scala2.13/delta-lake/delta-spark330db/pom.xml
+++ b/scala2.13/delta-lake/delta-spark330db/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../shim-deps/pom.xml
rapids-4-spark-delta-spark330db_2.13
RAPIDS Accelerator for Apache Spark Databricks 11.3 Delta Lake Support
Databricks 11.3 Delta Lake support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-spark330db
diff --git a/scala2.13/delta-lake/delta-spark332db/pom.xml b/scala2.13/delta-lake/delta-spark332db/pom.xml
index a3501c1003c..a8cae9d2f82 100644
--- a/scala2.13/delta-lake/delta-spark332db/pom.xml
+++ b/scala2.13/delta-lake/delta-spark332db/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../shim-deps/pom.xml
rapids-4-spark-delta-spark332db_2.13
RAPIDS Accelerator for Apache Spark Databricks 12.2 Delta Lake Support
Databricks 12.2 Delta Lake support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-spark332db
diff --git a/scala2.13/delta-lake/delta-spark341db/pom.xml b/scala2.13/delta-lake/delta-spark341db/pom.xml
index c740362b11f..9583419dc7c 100644
--- a/scala2.13/delta-lake/delta-spark341db/pom.xml
+++ b/scala2.13/delta-lake/delta-spark341db/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../shim-deps/pom.xml
rapids-4-spark-delta-spark341db_2.13
RAPIDS Accelerator for Apache Spark Databricks 13.3 Delta Lake Support
Databricks 13.3 Delta Lake support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
false
diff --git a/scala2.13/delta-lake/delta-spark350db143/pom.xml b/scala2.13/delta-lake/delta-spark350db143/pom.xml
index d6046b64578..da47b99455c 100644
--- a/scala2.13/delta-lake/delta-spark350db143/pom.xml
+++ b/scala2.13/delta-lake/delta-spark350db143/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../shim-deps/pom.xml
rapids-4-spark-delta-spark350db143_2.13
RAPIDS Accelerator for Apache Spark Databricks 13.3 Delta Lake Support
Databricks 13.3 Delta Lake support for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
false
diff --git a/scala2.13/delta-lake/delta-stub/pom.xml b/scala2.13/delta-lake/delta-stub/pom.xml
index 2f90b85acd7..989450c3e7e 100644
--- a/scala2.13/delta-lake/delta-stub/pom.xml
+++ b/scala2.13/delta-lake/delta-stub/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../jdk-profiles/pom.xml
rapids-4-spark-delta-stub_2.13
RAPIDS Accelerator for Apache Spark Delta Lake Stub
Delta Lake stub for the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
../delta-lake/delta-stub
diff --git a/scala2.13/dist/pom.xml b/scala2.13/dist/pom.xml
index 15df1ec69f8..d11161e9d7e 100644
--- a/scala2.13/dist/pom.xml
+++ b/scala2.13/dist/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../jdk-profiles/pom.xml
rapids-4-spark_2.13
RAPIDS Accelerator for Apache Spark Distribution
Creates the distribution package of the RAPIDS plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
com.nvidia
diff --git a/scala2.13/integration_tests/pom.xml b/scala2.13/integration_tests/pom.xml
index 88ab2531235..0f82e0b9186 100644
--- a/scala2.13/integration_tests/pom.xml
+++ b/scala2.13/integration_tests/pom.xml
@@ -22,11 +22,11 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-integration-tests_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
integration_tests
diff --git a/scala2.13/jdk-profiles/pom.xml b/scala2.13/jdk-profiles/pom.xml
index 793bf0fb327..808031d488b 100644
--- a/scala2.13/jdk-profiles/pom.xml
+++ b/scala2.13/jdk-profiles/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
com.nvidia
rapids-4-spark-jdk-profiles_2.13
pom
Shim JDK Profiles
- 24.12.0-SNAPSHOT
+ 24.12.0
jdk8
diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml
index d52c8658423..baebe599dc1 100644
--- a/scala2.13/pom.xml
+++ b/scala2.13/pom.xml
@@ -23,7 +23,7 @@
rapids-4-spark-parent_2.13
RAPIDS Accelerator for Apache Spark Root Project
The root project of the RAPIDS Accelerator for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
pom
https://nvidia.github.io/spark-rapids/
diff --git a/scala2.13/shim-deps/cloudera/pom.xml b/scala2.13/shim-deps/cloudera/pom.xml
index 95c49a2b1ca..bf89e7c373c 100644
--- a/scala2.13/shim-deps/cloudera/pom.xml
+++ b/scala2.13/shim-deps/cloudera/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../pom.xml
rapids-4-spark-cdh-bom
pom
CDH Shim Dependencies
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/cloudera
diff --git a/scala2.13/shim-deps/databricks/pom.xml b/scala2.13/shim-deps/databricks/pom.xml
index 484e2896f61..1d7c047f407 100644
--- a/scala2.13/shim-deps/databricks/pom.xml
+++ b/scala2.13/shim-deps/databricks/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../../pom.xml
rapids-4-spark-db-bom
pom
Databricks Shim Dependencies
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/databricks
diff --git a/scala2.13/shim-deps/pom.xml b/scala2.13/shim-deps/pom.xml
index 66cfa22afea..318909784ca 100644
--- a/scala2.13/shim-deps/pom.xml
+++ b/scala2.13/shim-deps/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../jdk-profiles/pom.xml
rapids-4-spark-shim-deps-parent_2.13
pom
Shim Dependencies Profiles
- 24.12.0-SNAPSHOT
+ 24.12.0
release321cdh
diff --git a/scala2.13/shuffle-plugin/pom.xml b/scala2.13/shuffle-plugin/pom.xml
index b9e76b2f068..d8a61620d6f 100644
--- a/scala2.13/shuffle-plugin/pom.xml
+++ b/scala2.13/shuffle-plugin/pom.xml
@@ -21,13 +21,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-shuffle_2.13
RAPIDS Accelerator for Apache Spark Shuffle Plugin
Accelerated shuffle plugin for the RAPIDS plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
shuffle-plugin
diff --git a/scala2.13/sql-plugin-api/pom.xml b/scala2.13/sql-plugin-api/pom.xml
index 3c48d7c13f2..dbe2d8f0222 100644
--- a/scala2.13/sql-plugin-api/pom.xml
+++ b/scala2.13/sql-plugin-api/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-sql-plugin-api_2.13
Module for Non-Shimmable API
- 24.12.0-SNAPSHOT
+ 24.12.0
sql-plugin-api
false
diff --git a/scala2.13/sql-plugin/pom.xml b/scala2.13/sql-plugin/pom.xml
index b96e1517690..a58ae69c1a2 100644
--- a/scala2.13/sql-plugin/pom.xml
+++ b/scala2.13/sql-plugin/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-sql_2.13
RAPIDS Accelerator for Apache Spark SQL Plugin
The RAPIDS SQL plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
sql-plugin
diff --git a/scala2.13/tests/pom.xml b/scala2.13/tests/pom.xml
index 377dc4671fb..33d1e1a761c 100644
--- a/scala2.13/tests/pom.xml
+++ b/scala2.13/tests/pom.xml
@@ -21,13 +21,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-tests_2.13
RAPIDS Accelerator for Apache Spark Tests
RAPIDS plugin for Apache Spark integration tests
- 24.12.0-SNAPSHOT
+ 24.12.0
tests
diff --git a/scala2.13/tools/pom.xml b/scala2.13/tools/pom.xml
index 0c3179e09ff..257d4aaf432 100644
--- a/scala2.13/tools/pom.xml
+++ b/scala2.13/tools/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../jdk-profiles/pom.xml
rapids-4-spark-tools-support
pom
RAPIDS Accelerator for Apache Spark Tools Support
Supporting code for RAPIDS Accelerator tools
- 24.12.0-SNAPSHOT
+ 24.12.0
com.nvidia
diff --git a/scala2.13/udf-compiler/pom.xml b/scala2.13/udf-compiler/pom.xml
index 10ad46a48aa..b52f3afe9f2 100644
--- a/scala2.13/udf-compiler/pom.xml
+++ b/scala2.13/udf-compiler/pom.xml
@@ -21,13 +21,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.13
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-udf_2.13
RAPIDS Accelerator for Apache Spark Scala UDF Plugin
The RAPIDS Scala UDF plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
udf-compiler
diff --git a/shim-deps/cloudera/pom.xml b/shim-deps/cloudera/pom.xml
index a9b71366927..fedc8d0931f 100644
--- a/shim-deps/cloudera/pom.xml
+++ b/shim-deps/cloudera/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../pom.xml
rapids-4-spark-cdh-bom
pom
CDH Shim Dependencies
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/cloudera
diff --git a/shim-deps/databricks/pom.xml b/shim-deps/databricks/pom.xml
index 5f36e529aa7..fdc1cceccd4 100644
--- a/shim-deps/databricks/pom.xml
+++ b/shim-deps/databricks/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../../pom.xml
rapids-4-spark-db-bom
pom
Databricks Shim Dependencies
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/databricks
diff --git a/shim-deps/pom.xml b/shim-deps/pom.xml
index d90dfc34190..580ae542eb5 100644
--- a/shim-deps/pom.xml
+++ b/shim-deps/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../jdk-profiles/pom.xml
rapids-4-spark-shim-deps-parent_2.12
pom
Shim Dependencies Profiles
- 24.12.0-SNAPSHOT
+ 24.12.0
release321cdh
diff --git a/shuffle-plugin/pom.xml b/shuffle-plugin/pom.xml
index 69d8f1b765b..1f3a7dd09d1 100644
--- a/shuffle-plugin/pom.xml
+++ b/shuffle-plugin/pom.xml
@@ -21,13 +21,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-shuffle_2.12
RAPIDS Accelerator for Apache Spark Shuffle Plugin
Accelerated shuffle plugin for the RAPIDS plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
shuffle-plugin
diff --git a/sql-plugin-api/pom.xml b/sql-plugin-api/pom.xml
index 090a809fc05..1bff34c1d61 100644
--- a/sql-plugin-api/pom.xml
+++ b/sql-plugin-api/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-sql-plugin-api_2.12
Module for Non-Shimmable API
- 24.12.0-SNAPSHOT
+ 24.12.0
sql-plugin-api
false
diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml
index c9cfb8ce99f..de1b9128481 100644
--- a/sql-plugin/pom.xml
+++ b/sql-plugin/pom.xml
@@ -22,13 +22,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-sql_2.12
RAPIDS Accelerator for Apache Spark SQL Plugin
The RAPIDS SQL plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
sql-plugin
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
index e750f5688ce..663a48f5a7e 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala
@@ -2504,7 +2504,7 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression.
|On startup use: `--conf [conf key]=[conf value]`. For example:
|
|```
- |${SPARK_HOME}/bin/spark-shell --jars rapids-4-spark_2.12-24.12.0-SNAPSHOT-cuda11.jar \
+ |${SPARK_HOME}/bin/spark-shell --jars rapids-4-spark_2.12-24.12.0-cuda11.jar \
|--conf spark.plugins=com.nvidia.spark.SQLPlugin \
|--conf spark.rapids.sql.concurrentGpuTasks=2
|```
diff --git a/tests/pom.xml b/tests/pom.xml
index a8fef6b7930..1efb8bdbaa2 100644
--- a/tests/pom.xml
+++ b/tests/pom.xml
@@ -21,13 +21,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-tests_2.12
RAPIDS Accelerator for Apache Spark Tests
RAPIDS plugin for Apache Spark integration tests
- 24.12.0-SNAPSHOT
+ 24.12.0
tests
diff --git a/tools/pom.xml b/tools/pom.xml
index df919f112ef..8725aca6cbe 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -22,14 +22,14 @@
com.nvidia
rapids-4-spark-jdk-profiles_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../jdk-profiles/pom.xml
rapids-4-spark-tools-support
pom
RAPIDS Accelerator for Apache Spark Tools Support
Supporting code for RAPIDS Accelerator tools
- 24.12.0-SNAPSHOT
+ 24.12.0
com.nvidia
diff --git a/udf-compiler/pom.xml b/udf-compiler/pom.xml
index afe827baf78..c4c2ce587bb 100644
--- a/udf-compiler/pom.xml
+++ b/udf-compiler/pom.xml
@@ -21,13 +21,13 @@
com.nvidia
rapids-4-spark-shim-deps-parent_2.12
- 24.12.0-SNAPSHOT
+ 24.12.0
../shim-deps/pom.xml
rapids-4-spark-udf_2.12
RAPIDS Accelerator for Apache Spark Scala UDF Plugin
The RAPIDS Scala UDF plugin for Apache Spark
- 24.12.0-SNAPSHOT
+ 24.12.0
udf-compiler
From 81b0b98180cbc88fbe4ce4b810950d9d966e9f26 Mon Sep 17 00:00:00 2001
From: Tim Liu
Date: Wed, 11 Dec 2024 13:21:42 +0800
Subject: [PATCH 098/103] Increase the pre-merge CI timeout to 6 hours (#11857)
I've seen several cases of PRs timing out after 4 hours though we've done a re-balance for 25.02 recently
https://github.com/NVIDIA/spark-rapids/pull/11826
We'll make additional efforts to balance the pre-merge CI's duration.
Let's increase the timeout to 6 hours first.
We'll continue to work on balancing the pre-merge CI's duration
Signed-off-by: Tim Liu
---
jenkins/Jenkinsfile-blossom.premerge | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/jenkins/Jenkinsfile-blossom.premerge b/jenkins/Jenkinsfile-blossom.premerge
index d61638d901a..474b14e66c4 100755
--- a/jenkins/Jenkinsfile-blossom.premerge
+++ b/jenkins/Jenkinsfile-blossom.premerge
@@ -190,7 +190,7 @@ git --no-pager diff --name-only HEAD \$BASE -- ${PREMERGE_DOCKERFILE} || true"""
steps {
script {
container('gpu') {
- timeout(time: 4, unit: 'HOURS') { // step only timeout for test run
+ timeout(time: 6, unit: 'HOURS') { // step only timeout for test run
try {
sh "$PREMERGE_SCRIPT mvn_verify"
step([$class : 'JacocoPublisher',
@@ -228,7 +228,7 @@ git --no-pager diff --name-only HEAD \$BASE -- ${PREMERGE_DOCKERFILE} || true"""
script {
unstash "source_tree"
container('gpu') {
- timeout(time: 4, unit: 'HOURS') {
+ timeout(time: 6, unit: 'HOURS') {
try {
sh "$PREMERGE_SCRIPT ci_2"
} finally {
@@ -260,7 +260,7 @@ git --no-pager diff --name-only HEAD \$BASE -- ${PREMERGE_DOCKERFILE} || true"""
script {
unstash "source_tree"
container('gpu') {
- timeout(time: 4, unit: 'HOURS') {
+ timeout(time: 6, unit: 'HOURS') {
try {
sh "$PREMERGE_SCRIPT ci_scala213"
} finally {
From 4b9bb2330ca646f44d2c0bc2845f197c2e34d83d Mon Sep 17 00:00:00 2001
From: liyuan <84758614+nvliyuan@users.noreply.github.com>
Date: Fri, 13 Dec 2024 09:41:45 +0800
Subject: [PATCH 099/103] [DOC] update doc for 24.12 release [skip ci] (#11841)
* update download page
Signed-off-by: liyuan
* update download page
Signed-off-by: liyuan
* update download page
Signed-off-by: liyuan
* update download page
Signed-off-by: liyuan
* update download page
Signed-off-by: liyuan
---------
Signed-off-by: liyuan
---
docs/archive.md | 90 ++++++++++++++++++++++++++++++++++++++++++++++++
docs/download.md | 33 ++++++++++--------
2 files changed, 108 insertions(+), 15 deletions(-)
diff --git a/docs/archive.md b/docs/archive.md
index 2d9a78ca3d4..4f9c5ed49fc 100644
--- a/docs/archive.md
+++ b/docs/archive.md
@@ -5,6 +5,96 @@ nav_order: 15
---
Below are archived releases for RAPIDS Accelerator for Apache Spark.
+## Release v24.10.1
+### Hardware Requirements:
+
+The plugin is tested on the following architectures:
+
+ GPU Models: NVIDIA V100, T4, A10/A100, L4 and H100 GPUs
+
+### Software Requirements:
+
+ OS: Spark RAPIDS is compatible with any Linux distribution with glibc >= 2.28 (Please check ldd --version output). glibc 2.28 was released August 1, 2018.
+ Tested on Ubuntu 20.04, Ubuntu 22.04, Rocky Linux 8 and Rocky Linux 9
+
+ NVIDIA Driver*: R470+
+
+ Runtime:
+ Scala 2.12, 2.13
+ Python, Java Virtual Machine (JVM) compatible with your spark-version.
+
+ * Check the Spark documentation for Python and Java version compatibility with your specific
+ Spark version. For instance, visit `https://spark.apache.org/docs/3.4.1` for Spark 3.4.1.
+
+ Supported Spark versions:
+ Apache Spark 3.2.0, 3.2.1, 3.2.2, 3.2.3, 3.2.4
+ Apache Spark 3.3.0, 3.3.1, 3.3.2, 3.3.3, 3.3.4
+ Apache Spark 3.4.0, 3.4.1, 3.4.2, 3.4.3
+ Apache Spark 3.5.0, 3.5.1, 3.5.2
+
+ Supported Databricks runtime versions for Azure and AWS:
+ Databricks 11.3 ML LTS (GPU, Scala 2.12, Spark 3.3.0)
+ Databricks 12.2 ML LTS (GPU, Scala 2.12, Spark 3.3.2)
+ Databricks 13.3 ML LTS (GPU, Scala 2.12, Spark 3.4.1)
+
+ Supported Dataproc versions (Debian/Ubuntu/Rocky):
+ GCP Dataproc 2.1
+ GCP Dataproc 2.2
+
+ Supported Dataproc Serverless versions:
+ Spark runtime 1.1 LTS
+ Spark runtime 2.0
+ Spark runtime 2.1
+ Spark runtime 2.2
+
+*Some hardware may have a minimum driver version greater than R470. Check the GPU spec sheet
+for your hardware's minimum driver version.
+
+*For Cloudera and EMR support, please refer to the
+[Distributions](https://docs.nvidia.com/spark-rapids/user-guide/latest/faq.html#which-distributions-are-supported) section of the FAQ.
+
+### RAPIDS Accelerator's Support Policy for Apache Spark
+The RAPIDS Accelerator maintains support for Apache Spark versions available for download from [Apache Spark](https://spark.apache.org/downloads.html)
+
+### Download RAPIDS Accelerator for Apache Spark v24.10.1
+
+| Processor | Scala Version | Download Jar | Download Signature |
+|-----------|---------------|--------------|--------------------|
+| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1.jar.asc) |
+| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1.jar.asc) |
+| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1-cuda11-arm64.jar.asc) |
+| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1-cuda11-arm64.jar.asc) |
+
+This package is built against CUDA 11.8. It is tested on V100, T4, A10, A100, L4 and H100 GPUs with
+CUDA 11.8 through CUDA 12.0.
+
+### Verify signature
+* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com).
+* Import the public key: `gpg --import PUB_KEY`
+* Verify the signature for Scala 2.12 jar:
+ `gpg --verify rapids-4-spark_2.12-24.10.1.jar.asc rapids-4-spark_2.12-24.10.1.jar`
+* Verify the signature for Scala 2.13 jar:
+ `gpg --verify rapids-4-spark_2.13-24.10.1.jar.asc rapids-4-spark_2.13-24.10.1.jar`
+
+The output of signature verify:
+
+ gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) "
+
+### Release Notes
+* Optimize scheduling policy for GPU Semaphore
+* Support distinct join for right outer joins
+* Support MinBy and MaxBy for non-float ordering
+* Support ArrayJoin expression
+* Optimize Expand and Aggregate expression performance
+* Improve JSON related expressions
+* For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases)
+
+Note: There is a known issue in the 24.10.1 release when decompressing gzip files on H100 GPUs.
+Please find more details in [issue-16661](https://github.com/rapidsai/cudf/issues/16661).
+
+For a detailed list of changes, please refer to the
+[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md).
+
## Release v24.10.0
### Hardware Requirements:
diff --git a/docs/download.md b/docs/download.md
index 60c62071f8b..e16c94e5b90 100644
--- a/docs/download.md
+++ b/docs/download.md
@@ -18,7 +18,7 @@ cuDF jar, that is either preinstalled in the Spark classpath on all nodes or sub
that uses the RAPIDS Accelerator For Apache Spark. See the [getting-started
guide](https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html) for more details.
-## Release v24.10.1
+## Release v24.12.0
### Hardware Requirements:
The plugin is tested on the following architectures:
@@ -69,14 +69,14 @@ for your hardware's minimum driver version.
### RAPIDS Accelerator's Support Policy for Apache Spark
The RAPIDS Accelerator maintains support for Apache Spark versions available for download from [Apache Spark](https://spark.apache.org/downloads.html)
-### Download RAPIDS Accelerator for Apache Spark v24.10.1
+### Download RAPIDS Accelerator for Apache Spark v24.12.0
| Processor | Scala Version | Download Jar | Download Signature |
|-----------|---------------|--------------|--------------------|
-| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1.jar.asc) |
-| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1.jar.asc) |
-| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1-cuda11-arm64.jar.asc) |
-| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1-cuda11-arm64.jar.asc) |
+| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0.jar.asc) |
+| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0.jar.asc) |
+| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0-cuda11-arm64.jar.asc) |
+| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0-cuda11-arm64.jar.asc) |
This package is built against CUDA 11.8. It is tested on V100, T4, A10, A100, L4 and H100 GPUs with
CUDA 11.8 through CUDA 12.0.
@@ -85,24 +85,27 @@ CUDA 11.8 through CUDA 12.0.
* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com).
* Import the public key: `gpg --import PUB_KEY`
* Verify the signature for Scala 2.12 jar:
- `gpg --verify rapids-4-spark_2.12-24.10.1.jar.asc rapids-4-spark_2.12-24.10.1.jar`
+ `gpg --verify rapids-4-spark_2.12-24.12.0.jar.asc rapids-4-spark_2.12-24.12.0.jar`
* Verify the signature for Scala 2.13 jar:
- `gpg --verify rapids-4-spark_2.13-24.10.1.jar.asc rapids-4-spark_2.13-24.10.1.jar`
+ `gpg --verify rapids-4-spark_2.13-24.12.0.jar.asc rapids-4-spark_2.13-24.12.0.jar`
The output of signature verify:
gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) "
### Release Notes
-* Optimize scheduling policy for GPU Semaphore
-* Support distinct join for right outer joins
-* Support MinBy and MaxBy for non-float ordering
-* Support ArrayJoin expression
-* Optimize Expand and Aggregate expression performance
-* Improve JSON related expressions
+* Add repartition-based algorithm fallback in hash aggregate
+* Support Spark function months_between
+* Support asynchronous writing for Parquet files
+* Add retry support to improve sub hash-join stability
+* Improve JSON scan and from_json
+* Improved performance for CASE WHEN statements comparing a string column against multiple values
+* Falling back to the CPU for ORC boolean writes by the GPU due to a bug in cudf's ORC writer
+* Fix a device memory leak in timestamp operator in `incompatibleDateFormats` case
+* Fix a host memory leak in GpuBroadcastNestedLoopJoinExecBase when `spillableBuiltBatch` is 0
* For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases)
-Note: There is a known issue in the 24.10.1 release when decompressing gzip files on H100 GPUs.
+Note: There is a known issue in the 24.12.0 release when decompressing gzip files on H100 GPUs.
Please find more details in [issue-16661](https://github.com/rapidsai/cudf/issues/16661).
For a detailed list of changes, please refer to the
From 4d7373b2c28f34eba3044b1ccc28aca91b968905 Mon Sep 17 00:00:00 2001
From: Jenkins Automation <70000568+nvauto@users.noreply.github.com>
Date: Mon, 16 Dec 2024 09:05:08 +0800
Subject: [PATCH 100/103] Update rapids JNI and private dependency to 24.12.0
(#11849)
\nWait for the pre-merge CI job to SUCCEED
Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com>
---
pom.xml | 4 ++--
scala2.13/pom.xml | 4 ++--
2 files changed, 4 insertions(+), 4 deletions(-)
diff --git a/pom.xml b/pom.xml
index 12828404031..00d2777a58e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -829,8 +829,8 @@
spark${buildver}
cuda11
${cuda.version}
- 24.12.0-SNAPSHOT
- 24.12.0-SNAPSHOT
+ 24.12.0
+ 24.12.0
2.12
2.8.0
incremental
diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml
index d52c8658423..b9b97830d8b 100644
--- a/scala2.13/pom.xml
+++ b/scala2.13/pom.xml
@@ -829,8 +829,8 @@
spark${buildver}
cuda11
${cuda.version}
- 24.12.0-SNAPSHOT
- 24.12.0-SNAPSHOT
+ 24.12.0
+ 24.12.0
2.13
2.8.0
incremental
From 22680f57119012a2fcc125cc93b71dcd1f6473a3 Mon Sep 17 00:00:00 2001
From: Jenkins Automation <70000568+nvauto@users.noreply.github.com>
Date: Mon, 16 Dec 2024 09:54:46 +0800
Subject: [PATCH 101/103] Update latest changelog [skip ci] (#11851)
* Update latest changelog [skip ci]
Update change log with CLI: \n\n scripts/generate-changelog --token= --releases=24.10,24.12
Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com>
* Update changelog
Signed-off-by: Tim Liu
* update changelog to involve new changes.
Signed-off-by: Yanxuan Liu
---------
Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com>
Signed-off-by: Tim Liu
Signed-off-by: Yanxuan Liu
Co-authored-by: Tim Liu
Co-authored-by: Yanxuan Liu
---
CHANGELOG.md | 387 ++++++++----------
...o-24.06.md => CHANGELOG_24.02-to-24.08.md} | 209 +++++++++-
2 files changed, 385 insertions(+), 211 deletions(-)
rename docs/archives/{CHANGELOG_24.02-to-24.06.md => CHANGELOG_24.02-to-24.08.md} (74%)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 2510eba5dfe..5dd6c7b19ba 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,175 @@
# Change log
-Generated on 2024-10-31
+Generated on 2024-12-16
+
+## Release 24.12
+
+### Features
+|||
+|:---|:---|
+|[#11630](https://github.com/NVIDIA/spark-rapids/issues/11630)|[FEA] enable from_json and json scan by default|
+|[#11709](https://github.com/NVIDIA/spark-rapids/issues/11709)|[FEA] Add support for `MonthsBetween`|
+|[#11666](https://github.com/NVIDIA/spark-rapids/issues/11666)|[FEA] support task limit profiling for specified stages|
+|[#11662](https://github.com/NVIDIA/spark-rapids/issues/11662)|[FEA] Support Apache Spark 3.4.4|
+|[#11657](https://github.com/NVIDIA/spark-rapids/issues/11657)|[FEA] Support format 'yyyyMMdd HH:mm:ss' for legacy mode|
+|[#11419](https://github.com/NVIDIA/spark-rapids/issues/11419)|[FEA] Support Spark 3.5.3 release|
+|[#11505](https://github.com/NVIDIA/spark-rapids/issues/11505)|[FEA] Support yyyymmdd format for GetTimestamp for LEGACY mode.|
+
+### Performance
+|||
+|:---|:---|
+|[#8391](https://github.com/NVIDIA/spark-rapids/issues/8391)|[FEA] Do a hash based re-partition instead of a sort based fallback for hash aggregate|
+|[#11560](https://github.com/NVIDIA/spark-rapids/issues/11560)|[FEA] Improve `GpuJsonToStructs` performance|
+|[#11458](https://github.com/NVIDIA/spark-rapids/issues/11458)|[FEA] enable prune_columns for from_json|
+
+### Bugs Fixed
+|||
+|:---|:---|
+|[#10907](https://github.com/NVIDIA/spark-rapids/issues/10907)|from_json function parses a column containing an empty array, throws an exception.|
+|[#11793](https://github.com/NVIDIA/spark-rapids/issues/11793)|[BUG] "Time in Heuristic" should not include previous operator's compute time|
+|[#11798](https://github.com/NVIDIA/spark-rapids/issues/11798)|[BUG] mismatch CPU and GPU result in test_months_between_first_day[DATAGEN_SEED=1733006411, TZ=Africa/Casablanca]|
+|[#11790](https://github.com/NVIDIA/spark-rapids/issues/11790)|[BUG] test_hash_* failed "java.util.NoSuchElementException: head of empty list" or "Too many times of repartition, may hit a bug?"|
+|[#11643](https://github.com/NVIDIA/spark-rapids/issues/11643)|[BUG] Support AQE with Broadcast Hash Join and DPP on Databricks 14.3|
+|[#10910](https://github.com/NVIDIA/spark-rapids/issues/10910)|from_json, when input = empty object, rapids throws an exception.|
+|[#10891](https://github.com/NVIDIA/spark-rapids/issues/10891)|Parsing a column containing invalid json into StructureType with schema throws an Exception.|
+|[#11741](https://github.com/NVIDIA/spark-rapids/issues/11741)|[BUG] Fix spark400 build due to writeWithV1 return value change|
+|[#11533](https://github.com/NVIDIA/spark-rapids/issues/11533)|Fix JSON Matrix tests on Databricks 14.3|
+|[#11722](https://github.com/NVIDIA/spark-rapids/issues/11722)|[BUG] Spark 4.0.0 has moved `NullIntolerant` and builds are breaking because they are unable to find it.|
+|[#11726](https://github.com/NVIDIA/spark-rapids/issues/11726)|[BUG] Databricks 14.3 nightly deploy fails due to incorrect DB_SHIM_NAME|
+|[#11293](https://github.com/NVIDIA/spark-rapids/issues/11293)|[BUG] A user query with from_json failed with "JSON Parser encountered an invalid format at location"|
+|[#9592](https://github.com/NVIDIA/spark-rapids/issues/9592)|[BUG][JSON] `from_json` to Map type should produce null for invalid entries|
+|[#11715](https://github.com/NVIDIA/spark-rapids/issues/11715)|[BUG] parquet_testing_test.py failed on "AssertionError: GPU and CPU boolean values are different"|
+|[#11716](https://github.com/NVIDIA/spark-rapids/issues/11716)|[BUG] delta_lake_write_test.py failed on "AssertionError: GPU and CPU boolean values are different"|
+|[#11684](https://github.com/NVIDIA/spark-rapids/issues/11684)|[BUG] 24.12 Precommit fails with wrong number of arguments in `GpuDataSource`|
+|[#11168](https://github.com/NVIDIA/spark-rapids/issues/11168)|[BUG] reserve allocation should be displayed when erroring due to lack of memory on startup|
+|[#7585](https://github.com/NVIDIA/spark-rapids/issues/7585)|[BUG] [Regexp] Line anchor '$' incorrect matching of unicode line terminators|
+|[#11622](https://github.com/NVIDIA/spark-rapids/issues/11622)|[BUG] GPU Parquet scan filter pushdown fails with timestamp/INT96 column|
+|[#11646](https://github.com/NVIDIA/spark-rapids/issues/11646)|[BUG] NullPointerException in GpuRand|
+|[#10498](https://github.com/NVIDIA/spark-rapids/issues/10498)|[BUG] Unit tests failed: [INTERVAL_ARITHMETIC_OVERFLOW] integer overflow. Use 'try_add' to tolerate overflow and return NULL instead|
+|[#11659](https://github.com/NVIDIA/spark-rapids/issues/11659)|[BUG] parse_url throws exception if partToExtract is invalid while Spark returns null|
+|[#10894](https://github.com/NVIDIA/spark-rapids/issues/10894)|Parsing a column containing a nested structure to json thows an exception|
+|[#10895](https://github.com/NVIDIA/spark-rapids/issues/10895)|Converting a column containing a map into json throws an exception|
+|[#10896](https://github.com/NVIDIA/spark-rapids/issues/10896)|Converting an column containing an array into json throws an exception|
+|[#10915](https://github.com/NVIDIA/spark-rapids/issues/10915)|to_json when converts an array will throw an exception:|
+|[#10916](https://github.com/NVIDIA/spark-rapids/issues/10916)|to_json function doesn't support map[string, struct] to json conversion.|
+|[#10919](https://github.com/NVIDIA/spark-rapids/issues/10919)|to_json converting map[string, integer] to json, throws an exception|
+|[#10920](https://github.com/NVIDIA/spark-rapids/issues/10920)|to_json converting an array with maps throws an exception.|
+|[#10921](https://github.com/NVIDIA/spark-rapids/issues/10921)|to_json - array with single map|
+|[#10923](https://github.com/NVIDIA/spark-rapids/issues/10923)|[BUG] Spark UT framework: to_json function to convert the array with a single empty row to a JSON string throws an exception.|
+|[#10924](https://github.com/NVIDIA/spark-rapids/issues/10924)|[BUG] Spark UT framework: to_json when converts an empty array into json throws an exception. |
+|[#11024](https://github.com/NVIDIA/spark-rapids/issues/11024)|Fix tests failures in parquet_write_test.py|
+|[#11174](https://github.com/NVIDIA/spark-rapids/issues/11174)|Opcode Suite fails for Scala 2.13.8+ |
+|[#10483](https://github.com/NVIDIA/spark-rapids/issues/10483)|[BUG] JsonToStructs fails to parse all empty dicts and invalid lines|
+|[#10489](https://github.com/NVIDIA/spark-rapids/issues/10489)|[BUG] from_json does not support input with \n in it.|
+|[#10347](https://github.com/NVIDIA/spark-rapids/issues/10347)|[BUG] Failures in Integration Tests on Dataproc Serverless|
+|[#11021](https://github.com/NVIDIA/spark-rapids/issues/11021)|Fix tests failures in orc_cast_test.py|
+|[#11609](https://github.com/NVIDIA/spark-rapids/issues/11609)|[BUG] test_hash_repartition_long_overflow_ansi_exception failed on 341DB|
+|[#11600](https://github.com/NVIDIA/spark-rapids/issues/11600)|[BUG] regex_test failed mismatched cpu and gpu values in UT and IT|
+|[#11611](https://github.com/NVIDIA/spark-rapids/issues/11611)|[BUG] Spark 4.0 build failure - value cannotSaveIntervalIntoExternalStorageError is not a member of object org.apache.spark.sql.errors.QueryCompilationErrors|
+|[#10922](https://github.com/NVIDIA/spark-rapids/issues/10922)|from_json cannot support line separator in the input string.|
+|[#11009](https://github.com/NVIDIA/spark-rapids/issues/11009)|Fix tests failures in cast_test.py|
+|[#11572](https://github.com/NVIDIA/spark-rapids/issues/11572)|[BUG] MultiFileReaderThreadPool may flood the console with log messages|
+
+### PRs
+|||
+|:---|:---|
+|[#11849](https://github.com/NVIDIA/spark-rapids/pull/11849)|Update rapids JNI and private dependency to 24.12.0|
+|[#11857](https://github.com/NVIDIA/spark-rapids/pull/11857)|Increase the pre-merge CI timeout to 6 hours|
+|[#11845](https://github.com/NVIDIA/spark-rapids/pull/11845)|Fix leak in isTimeStamp|
+|[#11823](https://github.com/NVIDIA/spark-rapids/pull/11823)|Fix for `LEAD/LAG` window function test failures.|
+|[#11832](https://github.com/NVIDIA/spark-rapids/pull/11832)|Fix leak in GpuBroadcastNestedLoopJoinExecBase|
+|[#11763](https://github.com/NVIDIA/spark-rapids/pull/11763)|Orc writes don't fully support Booleans with nulls |
+|[#11794](https://github.com/NVIDIA/spark-rapids/pull/11794)|exclude previous operator's time out of firstBatchHeuristic|
+|[#11802](https://github.com/NVIDIA/spark-rapids/pull/11802)|Fall back to CPU for non-UTC months_between|
+|[#11792](https://github.com/NVIDIA/spark-rapids/pull/11792)|[BUG] Fix issue 11790|
+|[#11768](https://github.com/NVIDIA/spark-rapids/pull/11768)|Fix `dpp_test.py` failures on 14.3|
+|[#11752](https://github.com/NVIDIA/spark-rapids/pull/11752)|Ability to decompress snappy and zstd Parquet files via CPU|
+|[#11777](https://github.com/NVIDIA/spark-rapids/pull/11777)|Append knoguchi22 to blossom-ci whitelist [skip ci]|
+|[#11712](https://github.com/NVIDIA/spark-rapids/pull/11712)|repartition-based fallback for hash aggregate v3|
+|[#11771](https://github.com/NVIDIA/spark-rapids/pull/11771)|Fix query hang when using rapids multithread shuffle manager with kudo|
+|[#11759](https://github.com/NVIDIA/spark-rapids/pull/11759)|Avoid using StringBuffer in single-threaded methods.|
+|[#11766](https://github.com/NVIDIA/spark-rapids/pull/11766)|Fix Kudo batch serializer to only read header in hasNext|
+|[#11730](https://github.com/NVIDIA/spark-rapids/pull/11730)|Add support for asynchronous writing for parquet|
+|[#11750](https://github.com/NVIDIA/spark-rapids/pull/11750)|Fix aqe_test failures on 14.3.|
+|[#11753](https://github.com/NVIDIA/spark-rapids/pull/11753)|Enable JSON Scan and from_json by default|
+|[#11733](https://github.com/NVIDIA/spark-rapids/pull/11733)|Print out the current attempt object when OOM inside a retry block|
+|[#11618](https://github.com/NVIDIA/spark-rapids/pull/11618)|Execute `from_json` with struct schema using `JSONUtils.fromJSONToStructs`|
+|[#11725](https://github.com/NVIDIA/spark-rapids/pull/11725)|host watermark metric|
+|[#11746](https://github.com/NVIDIA/spark-rapids/pull/11746)|Remove batch size bytes limits|
+|[#11723](https://github.com/NVIDIA/spark-rapids/pull/11723)|Add NVIDIA Copyright|
+|[#11721](https://github.com/NVIDIA/spark-rapids/pull/11721)|Add a few more JSON tests for MAP|
+|[#11744](https://github.com/NVIDIA/spark-rapids/pull/11744)|Do not package the Databricks 14.3 shim into the dist jar [skip ci]|
+|[#11724](https://github.com/NVIDIA/spark-rapids/pull/11724)|Integrate with kudo|
+|[#11739](https://github.com/NVIDIA/spark-rapids/pull/11739)|Update to Spark 4.0 changing signature of SupportsV1Write.writeWithV1|
+|[#11737](https://github.com/NVIDIA/spark-rapids/pull/11737)|Add in support for months_between|
+|[#11700](https://github.com/NVIDIA/spark-rapids/pull/11700)|Fix leak with RapidsHostColumnBuilder in GpuUserDefinedFunction|
+|[#11727](https://github.com/NVIDIA/spark-rapids/pull/11727)|Widen type promotion for decimals with larger scale in Parquet Read|
+|[#11719](https://github.com/NVIDIA/spark-rapids/pull/11719)|Skip `from_json` overflow tests for 14.3|
+|[#11708](https://github.com/NVIDIA/spark-rapids/pull/11708)|Support profiling for specific stages on a limited number of tasks|
+|[#11731](https://github.com/NVIDIA/spark-rapids/pull/11731)|Add NullIntolerantShim to adapt to Spark 4.0 removing NullIntolerant|
+|[#11413](https://github.com/NVIDIA/spark-rapids/pull/11413)|Support multi string contains|
+|[#11728](https://github.com/NVIDIA/spark-rapids/pull/11728)|Change Databricks 14.3 shim name to spark350db143 [skip ci]|
+|[#11702](https://github.com/NVIDIA/spark-rapids/pull/11702)|Improve JSON scan and `from_json`|
+|[#11635](https://github.com/NVIDIA/spark-rapids/pull/11635)|Added Shims for adding Databricks 14.3 Support|
+|[#11714](https://github.com/NVIDIA/spark-rapids/pull/11714)|Let AWS Databricks automatically choose an Availability Zone|
+|[#11703](https://github.com/NVIDIA/spark-rapids/pull/11703)|Simplify $ transpiling and fix newline character bug|
+|[#11707](https://github.com/NVIDIA/spark-rapids/pull/11707)|impalaFile cannot be found by UT framework. |
+|[#11697](https://github.com/NVIDIA/spark-rapids/pull/11697)|Make delta-lake shim dependencies parametrizable|
+|[#11710](https://github.com/NVIDIA/spark-rapids/pull/11710)|Add shim version 344 to LogicalPlanShims.scala|
+|[#11706](https://github.com/NVIDIA/spark-rapids/pull/11706)|Add retry support in sub hash join|
+|[#11673](https://github.com/NVIDIA/spark-rapids/pull/11673)|Fix Parquet Writer tests on 14.3|
+|[#11669](https://github.com/NVIDIA/spark-rapids/pull/11669)|Fix `string_test` for 14.3|
+|[#11692](https://github.com/NVIDIA/spark-rapids/pull/11692)|Add Spark 3.4.4 Shim |
+|[#11695](https://github.com/NVIDIA/spark-rapids/pull/11695)|Fix spark400 build due to LogicalRelation signature changes|
+|[#11689](https://github.com/NVIDIA/spark-rapids/pull/11689)|Update the Maven repository to download Spark JAR files [skip ci]|
+|[#11670](https://github.com/NVIDIA/spark-rapids/pull/11670)|Fix `misc_expr_test` for 14.3|
+|[#11652](https://github.com/NVIDIA/spark-rapids/pull/11652)|Fix skipping fixed_length_char ORC tests on > 13.3|
+|[#11644](https://github.com/NVIDIA/spark-rapids/pull/11644)|Skip AQE-join-DPP tests for 14.3|
+|[#11667](https://github.com/NVIDIA/spark-rapids/pull/11667)|Preparation for the coming Kudo support|
+|[#11685](https://github.com/NVIDIA/spark-rapids/pull/11685)|Exclude shimplify-generated files from scalastyle|
+|[#11282](https://github.com/NVIDIA/spark-rapids/pull/11282)|Reserve allocation should be displayed when erroring due to lack of memory on startup|
+|[#11671](https://github.com/NVIDIA/spark-rapids/pull/11671)|Use the new host memory allocation API|
+|[#11682](https://github.com/NVIDIA/spark-rapids/pull/11682)|Fix auto merge conflict 11679 [skip ci]|
+|[#11663](https://github.com/NVIDIA/spark-rapids/pull/11663)|Simplify Transpilation of $ with Extended Line Separator Support in cuDF Regex|
+|[#11672](https://github.com/NVIDIA/spark-rapids/pull/11672)|Fix race condition with Parquet filter pushdown modifying shared hadoop Configuration|
+|[#11596](https://github.com/NVIDIA/spark-rapids/pull/11596)|Add a new NVTX range for task GPU ownership|
+|[#11664](https://github.com/NVIDIA/spark-rapids/pull/11664)|Fix `orc_write_test.py` for 14.3|
+|[#11656](https://github.com/NVIDIA/spark-rapids/pull/11656)|[DOC] update the supported OS in download page [skip ci]|
+|[#11665](https://github.com/NVIDIA/spark-rapids/pull/11665)|Generate classes identical up to the shim package name|
+|[#11647](https://github.com/NVIDIA/spark-rapids/pull/11647)|Fix a NPE issue in GpuRand|
+|[#11658](https://github.com/NVIDIA/spark-rapids/pull/11658)|Support format 'yyyyMMdd HH:mm:ss' for legacy mode|
+|[#11661](https://github.com/NVIDIA/spark-rapids/pull/11661)|Support invalid partToExtract for parse_url|
+|[#11520](https://github.com/NVIDIA/spark-rapids/pull/11520)|UT adjust override checkScanSchemata & enabling ut of exclude_by_suffix fea.|
+|[#11634](https://github.com/NVIDIA/spark-rapids/pull/11634)|Put DF_UDF plugin code into the main uber jar.|
+|[#11522](https://github.com/NVIDIA/spark-rapids/pull/11522)|UT adjust test SPARK-26677: negated null-safe equality comparison|
+|[#11521](https://github.com/NVIDIA/spark-rapids/pull/11521)|Datetime rebasing issue fixed|
+|[#11642](https://github.com/NVIDIA/spark-rapids/pull/11642)|Update to_json to be more generic and fix some bugs|
+|[#11615](https://github.com/NVIDIA/spark-rapids/pull/11615)|Spark 4 parquet_writer_test.py fixes|
+|[#11623](https://github.com/NVIDIA/spark-rapids/pull/11623)|Fix `collection_ops_test` for 14.3|
+|[#11553](https://github.com/NVIDIA/spark-rapids/pull/11553)|Fix udf-compiler scala2.13 internal return statements|
+|[#11640](https://github.com/NVIDIA/spark-rapids/pull/11640)|Disable date/timestamp types by default when parsing JSON|
+|[#11570](https://github.com/NVIDIA/spark-rapids/pull/11570)|Add support for Spark 3.5.3|
+|[#11591](https://github.com/NVIDIA/spark-rapids/pull/11591)|Spark UT framework: Read Parquet file generated by parquet-thrift Rapids, UT case adjust.|
+|[#11631](https://github.com/NVIDIA/spark-rapids/pull/11631)|Update JSON tests based on a closed/fixed issues|
+|[#11617](https://github.com/NVIDIA/spark-rapids/pull/11617)|Quick fix for the build script failure of Scala 2.13 jars [skip ci]|
+|[#11614](https://github.com/NVIDIA/spark-rapids/pull/11614)|Ensure repartition overflow test always overflows|
+|[#11612](https://github.com/NVIDIA/spark-rapids/pull/11612)|Revert "Disable regex tests to unblock CI (#11606)"|
+|[#11597](https://github.com/NVIDIA/spark-rapids/pull/11597)|`install_deps` changes for Databricks 14.3|
+|[#11608](https://github.com/NVIDIA/spark-rapids/pull/11608)|Use mvn -f scala2.13/ in the build scripts to build the 2.13 jars|
+|[#11610](https://github.com/NVIDIA/spark-rapids/pull/11610)|Change DataSource calendar interval error to fix spark400 build|
+|[#11549](https://github.com/NVIDIA/spark-rapids/pull/11549)|Adopt `JSONUtils.concatenateJsonStrings` for concatenating JSON strings|
+|[#11595](https://github.com/NVIDIA/spark-rapids/pull/11595)|Remove an unused config shuffle.spillThreads|
+|[#11606](https://github.com/NVIDIA/spark-rapids/pull/11606)|Disable regex tests to unblock CI|
+|[#11605](https://github.com/NVIDIA/spark-rapids/pull/11605)|Fix auto merge conflict 11604 [skip ci]|
+|[#11587](https://github.com/NVIDIA/spark-rapids/pull/11587)|avoid long tail tasks due to PrioritySemaphore, remaing part|
+|[#11574](https://github.com/NVIDIA/spark-rapids/pull/11574)|avoid long tail tasks due to PrioritySemaphore|
+|[#11559](https://github.com/NVIDIA/spark-rapids/pull/11559)|[Spark 4.0] Address test failures in cast_test.py|
+|[#11579](https://github.com/NVIDIA/spark-rapids/pull/11579)|Fix merge conflict with branch-24.10|
+|[#11571](https://github.com/NVIDIA/spark-rapids/pull/11571)|Log reconfigure multi-file thread pool only once|
+|[#11564](https://github.com/NVIDIA/spark-rapids/pull/11564)|Disk spill metric|
+|[#11561](https://github.com/NVIDIA/spark-rapids/pull/11561)|Add in a basic plugin for dataframe UDF support in Apache Spark|
+|[#11563](https://github.com/NVIDIA/spark-rapids/pull/11563)|Fix the latest merge conflict in integration tests|
+|[#11542](https://github.com/NVIDIA/spark-rapids/pull/11542)|Update rapids JNI and private dependency to 24.12.0-SNAPSHOT [skip ci]|
+|[#11493](https://github.com/NVIDIA/spark-rapids/pull/11493)|Support legacy mode for yyyymmdd format|
## Release 24.10
@@ -69,15 +239,21 @@ Generated on 2024-10-31
### PRs
|||
|:---|:---|
+|[#11683](https://github.com/NVIDIA/spark-rapids/pull/11683)|[DOC] update download page for 2410 hot fix release [skip ci]|
+|[#11680](https://github.com/NVIDIA/spark-rapids/pull/11680)|Update latest changelog [skip ci]|
+|[#11678](https://github.com/NVIDIA/spark-rapids/pull/11678)|Update version to 24.10.1-SNAPSHOT [skip ci]|
|[#11676](https://github.com/NVIDIA/spark-rapids/pull/11676)| Fix race condition with Parquet filter pushdown modifying shared hadoop Configuration|
|[#11626](https://github.com/NVIDIA/spark-rapids/pull/11626)|Update latest changelog [skip ci]|
|[#11624](https://github.com/NVIDIA/spark-rapids/pull/11624)|Update the download link [skip ci]|
|[#11577](https://github.com/NVIDIA/spark-rapids/pull/11577)|Update latest changelog [skip ci]|
|[#11576](https://github.com/NVIDIA/spark-rapids/pull/11576)|Update rapids JNI and private dependency to 24.10.0|
|[#11582](https://github.com/NVIDIA/spark-rapids/pull/11582)|[DOC] update doc for 24.10 release [skip ci]|
+|[#11414](https://github.com/NVIDIA/spark-rapids/pull/11414)|Fix `collection_ops_tests` for Spark 4.0|
|[#11588](https://github.com/NVIDIA/spark-rapids/pull/11588)|backport fixes of #11573 to branch 24.10|
|[#11569](https://github.com/NVIDIA/spark-rapids/pull/11569)|Have "dump always" dump input files before trying to decode them|
+|[#11544](https://github.com/NVIDIA/spark-rapids/pull/11544)|Update test case related to LEACY datetime format to unblock nightly CI|
|[#11567](https://github.com/NVIDIA/spark-rapids/pull/11567)|Fix test case unix_timestamp(col, 'yyyyMMdd') failed for Africa/Casablanca timezone and LEGACY mode|
+|[#11519](https://github.com/NVIDIA/spark-rapids/pull/11519)|Spark 4: Fix parquet_test.py|
|[#11496](https://github.com/NVIDIA/spark-rapids/pull/11496)|Update test now that code is fixed|
|[#11548](https://github.com/NVIDIA/spark-rapids/pull/11548)|Fix negative rs. shuffle write time|
|[#11545](https://github.com/NVIDIA/spark-rapids/pull/11545)|Update test case related to LEACY datetime format to unblock nightly CI|
@@ -157,215 +333,6 @@ Generated on 2024-10-31
|[#11280](https://github.com/NVIDIA/spark-rapids/pull/11280)|Asynchronously copy table data to the host during shuffle|
|[#11258](https://github.com/NVIDIA/spark-rapids/pull/11258)|Explicitly disable ANSI mode for ast_test.py|
|[#11267](https://github.com/NVIDIA/spark-rapids/pull/11267)|Update the rapids JNI and private dependency version to 24.10.0-SNAPSHOT|
-|[#11241](https://github.com/NVIDIA/spark-rapids/pull/11241)|Auto merge PRs to branch-24.10 from branch-24.08 [skip ci]|
-|[#11231](https://github.com/NVIDIA/spark-rapids/pull/11231)|Cache dependencies for scala 2.13 [skip ci]|
-
-## Release 24.08
-
-### Features
-|||
-|:---|:---|
-|[#9259](https://github.com/NVIDIA/spark-rapids/issues/9259)|[FEA] Create Spark 4.0.0 shim and build env|
-|[#10366](https://github.com/NVIDIA/spark-rapids/issues/10366)|[FEA] It would be nice if we could support Hive-style write bucketing table|
-|[#10987](https://github.com/NVIDIA/spark-rapids/issues/10987)|[FEA] Implement lore framework to support all operators.|
-|[#11087](https://github.com/NVIDIA/spark-rapids/issues/11087)|[FEA] Support regex pattern with brackets when rewrite to PrefixRange patten in rlike|
-|[#22](https://github.com/NVIDIA/spark-rapids/issues/22)|[FEA] Add support for bucketed writes|
-|[#9939](https://github.com/NVIDIA/spark-rapids/issues/9939)|[FEA] `GpuInsertIntoHiveTable` supports parquet format|
-
-### Performance
-|||
-|:---|:---|
-|[#8750](https://github.com/NVIDIA/spark-rapids/issues/8750)|[FEA] Rework GpuSubstringIndex to use cudf::slice_strings|
-|[#7404](https://github.com/NVIDIA/spark-rapids/issues/7404)|[FEA] explore a hash agg passthrough on partial aggregates|
-|[#10976](https://github.com/NVIDIA/spark-rapids/issues/10976)|Rewrite `pattern1|pattern2|pattern3` to multiple contains in `rlike`|
-
-### Bugs Fixed
-|||
-|:---|:---|
-|[#11287](https://github.com/NVIDIA/spark-rapids/issues/11287)|[BUG] String split APIs on empty string produce incorrect result|
-|[#11270](https://github.com/NVIDIA/spark-rapids/issues/11270)|[BUG] test_regexp_replace[DATAGEN_SEED=1722297411, TZ=UTC] hanging there forever in pre-merge CI intermittently|
-|[#9682](https://github.com/NVIDIA/spark-rapids/issues/9682)|[BUG] Casting FLOAT64 to DECIMAL(12,7) produces different rows from Apache Spark CPU|
-|[#10809](https://github.com/NVIDIA/spark-rapids/issues/10809)|[BUG] cast(9.95 as decimal(3,1)), actual: 9.9, expected: 10.0|
-|[#11266](https://github.com/NVIDIA/spark-rapids/issues/11266)|[BUG] test_broadcast_hash_join_constant_keys failed in databricks runtimes|
-|[#11243](https://github.com/NVIDIA/spark-rapids/issues/11243)|[BUG] ArrayIndexOutOfBoundsException on a left outer join|
-|[#11030](https://github.com/NVIDIA/spark-rapids/issues/11030)|Fix tests failures in string_test.py|
-|[#11245](https://github.com/NVIDIA/spark-rapids/issues/11245)|[BUG] mvn verify for the source-javadoc fails and no pre-merge check catches it|
-|[#11223](https://github.com/NVIDIA/spark-rapids/issues/11223)|[BUG] Remove unreferenced `CUDF_VER=xxx` in the CI script|
-|[#11114](https://github.com/NVIDIA/spark-rapids/issues/11114)|[BUG] Update nightly tests for Scala 2.13 to use JDK 17 only|
-|[#11229](https://github.com/NVIDIA/spark-rapids/issues/11229)|[BUG] test_delta_name_column_mapping_no_field_ids fails on Spark |
-|[#11031](https://github.com/NVIDIA/spark-rapids/issues/11031)|Fix tests failures in multiple files |
-|[#10948](https://github.com/NVIDIA/spark-rapids/issues/10948)|Figure out why `MapFromArrays ` appears in the tests for hive parquet write|
-|[#11018](https://github.com/NVIDIA/spark-rapids/issues/11018)|Fix tests failures in hash_aggregate_test.py|
-|[#11173](https://github.com/NVIDIA/spark-rapids/issues/11173)|[BUG] The `rs. serialization time` metric is misleading|
-|[#11017](https://github.com/NVIDIA/spark-rapids/issues/11017)|Fix tests failures in url_test.py|
-|[#11201](https://github.com/NVIDIA/spark-rapids/issues/11201)|[BUG] Delta Lake tables with name mapping can throw exceptions on read|
-|[#11175](https://github.com/NVIDIA/spark-rapids/issues/11175)|[BUG] Clean up unused and duplicated 'org/roaringbitmap' folder in the spark3xx shims|
-|[#11196](https://github.com/NVIDIA/spark-rapids/issues/11196)|[BUG] pipeline failed due to class not found exception: NoClassDefFoundError: com/nvidia/spark/rapids/GpuScalar|
-|[#11189](https://github.com/NVIDIA/spark-rapids/issues/11189)|[BUG] regression in NDS after PR #11170|
-|[#11167](https://github.com/NVIDIA/spark-rapids/issues/11167)|[BUG] UnsupportedOperationException during delta write with `optimize()`|
-|[#11172](https://github.com/NVIDIA/spark-rapids/issues/11172)|[BUG] `get_json_object` returns wrong output with wildcard path|
-|[#11148](https://github.com/NVIDIA/spark-rapids/issues/11148)|[BUG] Integration test `test_write_hive_bucketed_table` fails|
-|[#11155](https://github.com/NVIDIA/spark-rapids/issues/11155)|[BUG] ArrayIndexOutOfBoundsException in BatchWithPartitionData.splitColumnarBatch|
-|[#11152](https://github.com/NVIDIA/spark-rapids/issues/11152)|[BUG] LORE dumping consumes too much memory.|
-|[#11029](https://github.com/NVIDIA/spark-rapids/issues/11029)|Fix tests failures in subquery_test.py|
-|[#11150](https://github.com/NVIDIA/spark-rapids/issues/11150)|[BUG] hive_parquet_write_test.py::test_insert_hive_bucketed_table failure|
-|[#11070](https://github.com/NVIDIA/spark-rapids/issues/11070)|[BUG] numpy2 fail fastparquet cases: numpy.dtype size changed|
-|[#11136](https://github.com/NVIDIA/spark-rapids/issues/11136)|UnaryPositive expression doesn't extend UnaryExpression|
-|[#11122](https://github.com/NVIDIA/spark-rapids/issues/11122)|[BUG] UT MetricRange failed 651070526 was not less than 1.5E8 in spark313|
-|[#11119](https://github.com/NVIDIA/spark-rapids/issues/11119)|[BUG] window_function_test.py::test_window_group_limits_fallback_for_row_number fails in a distributed environment|
-|[#11023](https://github.com/NVIDIA/spark-rapids/issues/11023)|Fix tests failures in dpp_test.py|
-|[#11026](https://github.com/NVIDIA/spark-rapids/issues/11026)|Fix tests failures in map_test.py|
-|[#11020](https://github.com/NVIDIA/spark-rapids/issues/11020)|Fix tests failures in grouping_sets_test.py|
-|[#11113](https://github.com/NVIDIA/spark-rapids/issues/11113)|[BUG] Update premerge tests for Scala 2.13 to use JDK 17 only|
-|[#11027](https://github.com/NVIDIA/spark-rapids/issues/11027)|Fix tests failures in sort_test.py|
-|[#10775](https://github.com/NVIDIA/spark-rapids/issues/10775)|[BUG] Issues found by Spark UT Framework on RapidsStringExpressionsSuite|
-|[#11033](https://github.com/NVIDIA/spark-rapids/issues/11033)|[BUG] CICD failed a case: cmp_test.py::test_empty_filter[>]|
-|[#11103](https://github.com/NVIDIA/spark-rapids/issues/11103)|[BUG] UCX Shuffle With scala.MatchError |
-|[#11007](https://github.com/NVIDIA/spark-rapids/issues/11007)|Fix tests failures in array_test.py|
-|[#10801](https://github.com/NVIDIA/spark-rapids/issues/10801)|[BUG] JDK17 nightly build after Spark UT Framework is merged|
-|[#11019](https://github.com/NVIDIA/spark-rapids/issues/11019)|Fix tests failures in window_function_test.py|
-|[#11063](https://github.com/NVIDIA/spark-rapids/issues/11063)|[BUG] op time for GpuCoalesceBatches is more than actual|
-|[#11006](https://github.com/NVIDIA/spark-rapids/issues/11006)|Fix test failures in arithmetic_ops_test.py|
-|[#10995](https://github.com/NVIDIA/spark-rapids/issues/10995)|Fallback TimeZoneAwareExpression that only support UTC with zoneId instead of timeZone config|
-|[#8652](https://github.com/NVIDIA/spark-rapids/issues/8652)|[BUG] array_item test failures on Spark 3.3.x|
-|[#11053](https://github.com/NVIDIA/spark-rapids/issues/11053)|[BUG] Build on Databricks 330 fails|
-|[#10925](https://github.com/NVIDIA/spark-rapids/issues/10925)| Concat cannot accept no parameter|
-|[#10975](https://github.com/NVIDIA/spark-rapids/issues/10975)|[BUG] regex `^.*literal` cannot be rewritten as `contains(literal)` for multiline strings|
-|[#10956](https://github.com/NVIDIA/spark-rapids/issues/10956)|[BUG] hive_parquet_write_test.py: test_write_compressed_parquet_into_hive_table integration test failures|
-|[#10772](https://github.com/NVIDIA/spark-rapids/issues/10772)|[BUG] Issues found by Spark UT Framework on RapidsDataFrameAggregateSuite|
-|[#10986](https://github.com/NVIDIA/spark-rapids/issues/10986)|[BUG]Cast from string to float using hand-picked values failed in CastOpSuite|
-|[#10972](https://github.com/NVIDIA/spark-rapids/issues/10972)|Spark 4.0 compile errors |
-|[#10794](https://github.com/NVIDIA/spark-rapids/issues/10794)|[BUG] Incorrect cast of string columns containing various infinity notations with trailing spaces |
-|[#10964](https://github.com/NVIDIA/spark-rapids/issues/10964)|[BUG] Improve stability of pre-merge jenkinsfile|
-|[#10714](https://github.com/NVIDIA/spark-rapids/issues/10714)|Signature changed for `PythonUDFRunner.writeUDFs` |
-|[#10712](https://github.com/NVIDIA/spark-rapids/issues/10712)|[AUDIT] BatchScanExec/DataSourceV2Relation to group splits by join keys if they differ from partition keys|
-|[#10673](https://github.com/NVIDIA/spark-rapids/issues/10673)|[AUDIT] Rename plan nodes for PythonMapInArrowExec|
-|[#10710](https://github.com/NVIDIA/spark-rapids/issues/10710)|[AUDIT] `uncacheTableOrView` changed in CommandUtils |
-|[#10711](https://github.com/NVIDIA/spark-rapids/issues/10711)|[AUDIT] Match DataSourceV2ScanExecBase changes to groupPartitions method |
-|[#10669](https://github.com/NVIDIA/spark-rapids/issues/10669)|Supporting broadcast of multiple filtering keys in DynamicPruning |
-
-### PRs
-|||
-|:---|:---|
-|[#11400](https://github.com/NVIDIA/spark-rapids/pull/11400)|[DOC] update notes in download page for the decompressing gzip issue [skip ci]|
-|[#11355](https://github.com/NVIDIA/spark-rapids/pull/11355)|Update changelog for the v24.08 release [skip ci]|
-|[#11353](https://github.com/NVIDIA/spark-rapids/pull/11353)|Update download doc for v24.08.1 [skip ci]|
-|[#11352](https://github.com/NVIDIA/spark-rapids/pull/11352)|Update version to 24.08.1-SNAPSHOT [skip ci]|
-|[#11337](https://github.com/NVIDIA/spark-rapids/pull/11337)|Update changelog for the v24.08 release [skip ci]|
-|[#11335](https://github.com/NVIDIA/spark-rapids/pull/11335)|Fix Delta Lake truncation of min/max string values|
-|[#11304](https://github.com/NVIDIA/spark-rapids/pull/11304)|Update changelog for v24.08.0 release [skip ci]|
-|[#11303](https://github.com/NVIDIA/spark-rapids/pull/11303)|Update rapids JNI and private dependency to 24.08.0|
-|[#11296](https://github.com/NVIDIA/spark-rapids/pull/11296)|[DOC] update doc for 2408 release [skip CI]|
-|[#11309](https://github.com/NVIDIA/spark-rapids/pull/11309)|[Doc ]Update lore doc about the range [skip ci]|
-|[#11292](https://github.com/NVIDIA/spark-rapids/pull/11292)|Add work around for string split with empty input.|
-|[#11278](https://github.com/NVIDIA/spark-rapids/pull/11278)|Fix formatting of advanced configs doc|
-|[#10917](https://github.com/NVIDIA/spark-rapids/pull/10917)|Adopt changes from JNI for casting from float to decimal|
-|[#11269](https://github.com/NVIDIA/spark-rapids/pull/11269)|Revert "upgrade ucx to 1.17.0"|
-|[#11260](https://github.com/NVIDIA/spark-rapids/pull/11260)|Mitigate intermittent test_buckets and shuffle_smoke_test OOM issue|
-|[#11268](https://github.com/NVIDIA/spark-rapids/pull/11268)|Fix degenerate conditional nested loop join detection|
-|[#11244](https://github.com/NVIDIA/spark-rapids/pull/11244)|Fix ArrayIndexOutOfBoundsException on join counts with constant join keys|
-|[#11259](https://github.com/NVIDIA/spark-rapids/pull/11259)|CI Docker to support integration tests with Rocky OS + jdk17 [skip ci]|
-|[#11247](https://github.com/NVIDIA/spark-rapids/pull/11247)|Fix `string_test.py` errors on Spark 4.0|
-|[#11246](https://github.com/NVIDIA/spark-rapids/pull/11246)|Rework Maven Source Plugin Skip|
-|[#11149](https://github.com/NVIDIA/spark-rapids/pull/11149)|Rework on substring index|
-|[#11236](https://github.com/NVIDIA/spark-rapids/pull/11236)|Remove the unused vars from the version-def CI script|
-|[#11237](https://github.com/NVIDIA/spark-rapids/pull/11237)|Fork jvm for maven-source-plugin|
-|[#11200](https://github.com/NVIDIA/spark-rapids/pull/11200)|Multi-get_json_object|
-|[#11230](https://github.com/NVIDIA/spark-rapids/pull/11230)|Skip test where Delta Lake may not be fully compatible with Spark|
-|[#11220](https://github.com/NVIDIA/spark-rapids/pull/11220)|Avoid failing spark bug SPARK-44242 while generate run_dir|
-|[#11226](https://github.com/NVIDIA/spark-rapids/pull/11226)|Fix auto merge conflict 11212|
-|[#11129](https://github.com/NVIDIA/spark-rapids/pull/11129)|Spark 4: Fix miscellaneous tests including logic, repart, hive_delimited.|
-|[#11163](https://github.com/NVIDIA/spark-rapids/pull/11163)|Support `MapFromArrays` on GPU|
-|[#11219](https://github.com/NVIDIA/spark-rapids/pull/11219)|Fix hash_aggregate_test.py to run with ANSI enabled|
-|[#11186](https://github.com/NVIDIA/spark-rapids/pull/11186)|from_json Json to Struct Exception Logging|
-|[#11180](https://github.com/NVIDIA/spark-rapids/pull/11180)|More accurate estimation for the result serialization time in RapidsShuffleThreadedWriterBase|
-|[#11194](https://github.com/NVIDIA/spark-rapids/pull/11194)|Fix ANSI mode test failures in url_test.py|
-|[#11202](https://github.com/NVIDIA/spark-rapids/pull/11202)|Fix read from Delta Lake table with name column mapping and missing Parquet IDs|
-|[#11185](https://github.com/NVIDIA/spark-rapids/pull/11185)|Fix multi-release jar problem|
-|[#11144](https://github.com/NVIDIA/spark-rapids/pull/11144)|Build the Scala2.13 dist jar with JDK17|
-|[#11197](https://github.com/NVIDIA/spark-rapids/pull/11197)|Fix class not found error: com/nvidia/spark/rapids/GpuScalar|
-|[#11191](https://github.com/NVIDIA/spark-rapids/pull/11191)|Fix dynamic pruning regression in GpuFileSourceScanExec|
-|[#10994](https://github.com/NVIDIA/spark-rapids/pull/10994)|Add Spark 4.0.0 Build Profile and Other Supporting Changes|
-|[#11192](https://github.com/NVIDIA/spark-rapids/pull/11192)|Append new authorized user to blossom-ci whitelist [skip ci]|
-|[#11179](https://github.com/NVIDIA/spark-rapids/pull/11179)|Allow more expressions to be tiered|
-|[#11141](https://github.com/NVIDIA/spark-rapids/pull/11141)|Enable some Rapids config in RapidsSQLTestsBaseTrait for Spark UT|
-|[#11170](https://github.com/NVIDIA/spark-rapids/pull/11170)|Avoid listFiles or inputFiles on relations with static partitioning|
-|[#11159](https://github.com/NVIDIA/spark-rapids/pull/11159)|Drop spark31x shims|
-|[#10951](https://github.com/NVIDIA/spark-rapids/pull/10951)|Case when performance improvement: reduce the `copy_if_else`|
-|[#11165](https://github.com/NVIDIA/spark-rapids/pull/11165)|Fix some GpuBroadcastToRowExec by not dropping columns|
-|[#11126](https://github.com/NVIDIA/spark-rapids/pull/11126)|Coalesce batches after a logical coalesce operation|
-|[#11164](https://github.com/NVIDIA/spark-rapids/pull/11164)|fix the bucketed write error for non-utc cases|
-|[#11132](https://github.com/NVIDIA/spark-rapids/pull/11132)|Add deletion vector metrics for low shuffle merge.|
-|[#11156](https://github.com/NVIDIA/spark-rapids/pull/11156)|Fix batch splitting for partition column size on row-count-only batches|
-|[#11153](https://github.com/NVIDIA/spark-rapids/pull/11153)|Fix LORE dump oom.|
-|[#11102](https://github.com/NVIDIA/spark-rapids/pull/11102)|Fix ANSI mode failures in subquery_test.py|
-|[#11151](https://github.com/NVIDIA/spark-rapids/pull/11151)|Fix the test error of the bucketed write for the non-utc case|
-|[#11147](https://github.com/NVIDIA/spark-rapids/pull/11147)|upgrade ucx to 1.17.0|
-|[#11138](https://github.com/NVIDIA/spark-rapids/pull/11138)|Update fastparquet to 2024.5.0 for numpy2 compatibility|
-|[#11137](https://github.com/NVIDIA/spark-rapids/pull/11137)|Handle the change for UnaryPositive now extending RuntimeReplaceable|
-|[#11094](https://github.com/NVIDIA/spark-rapids/pull/11094)|Add `HiveHash` support on GPU|
-|[#11139](https://github.com/NVIDIA/spark-rapids/pull/11139)|Improve MetricsSuite to allow more gc jitter|
-|[#11133](https://github.com/NVIDIA/spark-rapids/pull/11133)|Fix `test_window_group_limits_fallback`|
-|[#11097](https://github.com/NVIDIA/spark-rapids/pull/11097)|Fix miscellaneous integ tests for Spark 4|
-|[#11118](https://github.com/NVIDIA/spark-rapids/pull/11118)|Fix issue with DPP and AQE on reused broadcast exchanges|
-|[#11043](https://github.com/NVIDIA/spark-rapids/pull/11043)|Dataproc serverless test fixes|
-|[#10965](https://github.com/NVIDIA/spark-rapids/pull/10965)|Profiler: Disable collecting async allocation events by default|
-|[#11117](https://github.com/NVIDIA/spark-rapids/pull/11117)|Update Scala2.13 premerge CI against JDK17|
-|[#11084](https://github.com/NVIDIA/spark-rapids/pull/11084)|Introduce LORE framework.|
-|[#11099](https://github.com/NVIDIA/spark-rapids/pull/11099)|Spark 4: Handle ANSI mode in sort_test.py|
-|[#11115](https://github.com/NVIDIA/spark-rapids/pull/11115)|Fix match error in RapidsShuffleIterator.scala [scala2.13]|
-|[#11088](https://github.com/NVIDIA/spark-rapids/pull/11088)|Support regex patterns with brackets when rewriting to PrefixRange pattern in rlike.|
-|[#10950](https://github.com/NVIDIA/spark-rapids/pull/10950)|Add a heuristic to skip second or third agg pass|
-|[#11048](https://github.com/NVIDIA/spark-rapids/pull/11048)|Fixed array_tests for Spark 4.0.0|
-|[#11049](https://github.com/NVIDIA/spark-rapids/pull/11049)|Fix some cast_tests for Spark 4.0.0|
-|[#11066](https://github.com/NVIDIA/spark-rapids/pull/11066)|Replaced spark3xx-common references to spark-shared|
-|[#11083](https://github.com/NVIDIA/spark-rapids/pull/11083)|Exclude a case based on JDK version in Spark UT|
-|[#10997](https://github.com/NVIDIA/spark-rapids/pull/10997)|Fix some test issues in Spark UT and keep RapidsTestSettings update-to-date|
-|[#11073](https://github.com/NVIDIA/spark-rapids/pull/11073)|Disable ANSI mode for window function tests|
-|[#11076](https://github.com/NVIDIA/spark-rapids/pull/11076)|Improve the diagnostics for 'conv' fallback explain|
-|[#11092](https://github.com/NVIDIA/spark-rapids/pull/11092)|Add GpuBucketingUtils shim to Spark 4.0.0|
-|[#11062](https://github.com/NVIDIA/spark-rapids/pull/11062)|fix duplicate counted metrics like op time for GpuCoalesceBatches|
-|[#11044](https://github.com/NVIDIA/spark-rapids/pull/11044)|Fixed Failing tests in arithmetic_ops_tests for Spark 4.0.0|
-|[#11086](https://github.com/NVIDIA/spark-rapids/pull/11086)|upgrade blossom-ci actions version [skip ci]|
-|[#10957](https://github.com/NVIDIA/spark-rapids/pull/10957)|Support bucketing write for GPU|
-|[#10979](https://github.com/NVIDIA/spark-rapids/pull/10979)|[FEA] Introduce low shuffle merge.|
-|[#10996](https://github.com/NVIDIA/spark-rapids/pull/10996)|Fallback non-UTC TimeZoneAwareExpression with zoneId|
-|[#11072](https://github.com/NVIDIA/spark-rapids/pull/11072)|Workaround numpy2 failed fastparquet compatibility tests|
-|[#11046](https://github.com/NVIDIA/spark-rapids/pull/11046)|Calculate parallelism to speed up pre-merge CI|
-|[#11054](https://github.com/NVIDIA/spark-rapids/pull/11054)|fix flaky array_item test failures|
-|[#11051](https://github.com/NVIDIA/spark-rapids/pull/11051)|[FEA] Increase parallelism of deltalake test on databricks|
-|[#10993](https://github.com/NVIDIA/spark-rapids/pull/10993)|`binary-dedupe` changes for Spark 4.0.0|
-|[#11060](https://github.com/NVIDIA/spark-rapids/pull/11060)|Add in the ability to fingerprint JSON columns|
-|[#11059](https://github.com/NVIDIA/spark-rapids/pull/11059)|Revert "Add in the ability to fingerprint JSON columns (#11002)" [skip ci]|
-|[#11039](https://github.com/NVIDIA/spark-rapids/pull/11039)|Concat() Exception bug fix|
-|[#11002](https://github.com/NVIDIA/spark-rapids/pull/11002)|Add in the ability to fingerprint JSON columns|
-|[#10977](https://github.com/NVIDIA/spark-rapids/pull/10977)|Rewrite multiple literal choice regex to multiple contains in rlike|
-|[#11035](https://github.com/NVIDIA/spark-rapids/pull/11035)|Fix auto merge conflict 11034 [skip ci]|
-|[#11040](https://github.com/NVIDIA/spark-rapids/pull/11040)|Append new authorized user to blossom-ci whitelist [skip ci]|
-|[#11036](https://github.com/NVIDIA/spark-rapids/pull/11036)|Update blossom-ci ACL to secure format [skip ci]|
-|[#11032](https://github.com/NVIDIA/spark-rapids/pull/11032)|Fix a hive write test failure for Spark 350|
-|[#10998](https://github.com/NVIDIA/spark-rapids/pull/10998)|Improve log to print more lines in build [skip ci]|
-|[#10992](https://github.com/NVIDIA/spark-rapids/pull/10992)|Addressing the Named Parameter change in Spark 4.0.0|
-|[#10943](https://github.com/NVIDIA/spark-rapids/pull/10943)|Fix Spark UT issues in RapidsDataFrameAggregateSuite|
-|[#10963](https://github.com/NVIDIA/spark-rapids/pull/10963)|Add rapids configs to enable GPU running in Spark UT|
-|[#10978](https://github.com/NVIDIA/spark-rapids/pull/10978)|More compilation fixes for Spark 4.0.0|
-|[#10953](https://github.com/NVIDIA/spark-rapids/pull/10953)|Speed up the integration tests by running them in parallel on the Databricks cluster|
-|[#10958](https://github.com/NVIDIA/spark-rapids/pull/10958)|Fix a hive write test failure|
-|[#10970](https://github.com/NVIDIA/spark-rapids/pull/10970)|Move Support for `RaiseError` to a Shim Excluding Spark 4.0.0|
-|[#10966](https://github.com/NVIDIA/spark-rapids/pull/10966)|Add default value for REF of premerge jenkinsfile to avoid bad overwritten [skip ci]|
-|[#10959](https://github.com/NVIDIA/spark-rapids/pull/10959)|Add new ID to blossom-ci allow list [skip ci]|
-|[#10952](https://github.com/NVIDIA/spark-rapids/pull/10952)|Add shims to take care of the signature change for writeUDFs in PythonUDFRunner|
-|[#10931](https://github.com/NVIDIA/spark-rapids/pull/10931)|Add Support for Renaming of PythonMapInArrow|
-|[#10949](https://github.com/NVIDIA/spark-rapids/pull/10949)|Change dependency version to 24.08.0-SNAPSHOT|
-|[#10857](https://github.com/NVIDIA/spark-rapids/pull/10857)|[Spark 4.0] Account for `PartitionedFileUtil.splitFiles` signature change.|
-|[#10912](https://github.com/NVIDIA/spark-rapids/pull/10912)|GpuInsertIntoHiveTable supports parquet format|
-|[#10863](https://github.com/NVIDIA/spark-rapids/pull/10863)|[Spark 4.0] Account for `CommandUtils.uncacheTableOrView` signature change.|
-|[#10944](https://github.com/NVIDIA/spark-rapids/pull/10944)|Added Shim for BatchScanExec to Support Spark 4.0|
-|[#10946](https://github.com/NVIDIA/spark-rapids/pull/10946)|Unarchive Spark test jar for spark.read(ability)|
-|[#10945](https://github.com/NVIDIA/spark-rapids/pull/10945)|Add Support for Multiple Filtering Keys for Subquery Broadcast|
-|[#10871](https://github.com/NVIDIA/spark-rapids/pull/10871)|Add classloader diagnostics to initShuffleManager error message|
-|[#10933](https://github.com/NVIDIA/spark-rapids/pull/10933)|Fixed Databricks build|
-|[#10929](https://github.com/NVIDIA/spark-rapids/pull/10929)|Append new authorized user to blossom-ci whitelist [skip ci]|
## Older Releases
Changelog of older releases can be found at [docs/archives](/docs/archives)
diff --git a/docs/archives/CHANGELOG_24.02-to-24.06.md b/docs/archives/CHANGELOG_24.02-to-24.08.md
similarity index 74%
rename from docs/archives/CHANGELOG_24.02-to-24.06.md
rename to docs/archives/CHANGELOG_24.02-to-24.08.md
index d95307a1efe..075b42966d4 100644
--- a/docs/archives/CHANGELOG_24.02-to-24.06.md
+++ b/docs/archives/CHANGELOG_24.02-to-24.08.md
@@ -1,5 +1,212 @@
# Change log
-Generated on 2024-10-09
+Generated on 2024-12-10
+## Release 24.08
+
+### Features
+|||
+|:---|:---|
+|[#9259](https://github.com/NVIDIA/spark-rapids/issues/9259)|[FEA] Create Spark 4.0.0 shim and build env|
+|[#10366](https://github.com/NVIDIA/spark-rapids/issues/10366)|[FEA] It would be nice if we could support Hive-style write bucketing table|
+|[#10987](https://github.com/NVIDIA/spark-rapids/issues/10987)|[FEA] Implement lore framework to support all operators.|
+|[#11087](https://github.com/NVIDIA/spark-rapids/issues/11087)|[FEA] Support regex pattern with brackets when rewrite to PrefixRange patten in rlike|
+|[#22](https://github.com/NVIDIA/spark-rapids/issues/22)|[FEA] Add support for bucketed writes|
+|[#9939](https://github.com/NVIDIA/spark-rapids/issues/9939)|[FEA] `GpuInsertIntoHiveTable` supports parquet format|
+
+### Performance
+|||
+|:---|:---|
+|[#8750](https://github.com/NVIDIA/spark-rapids/issues/8750)|[FEA] Rework GpuSubstringIndex to use cudf::slice_strings|
+|[#7404](https://github.com/NVIDIA/spark-rapids/issues/7404)|[FEA] explore a hash agg passthrough on partial aggregates|
+|[#10976](https://github.com/NVIDIA/spark-rapids/issues/10976)|Rewrite `pattern1|pattern2|pattern3` to multiple contains in `rlike`|
+
+### Bugs Fixed
+|||
+|:---|:---|
+|[#11287](https://github.com/NVIDIA/spark-rapids/issues/11287)|[BUG] String split APIs on empty string produce incorrect result|
+|[#11270](https://github.com/NVIDIA/spark-rapids/issues/11270)|[BUG] test_regexp_replace[DATAGEN_SEED=1722297411, TZ=UTC] hanging there forever in pre-merge CI intermittently|
+|[#9682](https://github.com/NVIDIA/spark-rapids/issues/9682)|[BUG] Casting FLOAT64 to DECIMAL(12,7) produces different rows from Apache Spark CPU|
+|[#10809](https://github.com/NVIDIA/spark-rapids/issues/10809)|[BUG] cast(9.95 as decimal(3,1)), actual: 9.9, expected: 10.0|
+|[#11266](https://github.com/NVIDIA/spark-rapids/issues/11266)|[BUG] test_broadcast_hash_join_constant_keys failed in databricks runtimes|
+|[#11243](https://github.com/NVIDIA/spark-rapids/issues/11243)|[BUG] ArrayIndexOutOfBoundsException on a left outer join|
+|[#11030](https://github.com/NVIDIA/spark-rapids/issues/11030)|Fix tests failures in string_test.py|
+|[#11245](https://github.com/NVIDIA/spark-rapids/issues/11245)|[BUG] mvn verify for the source-javadoc fails and no pre-merge check catches it|
+|[#11223](https://github.com/NVIDIA/spark-rapids/issues/11223)|[BUG] Remove unreferenced `CUDF_VER=xxx` in the CI script|
+|[#11114](https://github.com/NVIDIA/spark-rapids/issues/11114)|[BUG] Update nightly tests for Scala 2.13 to use JDK 17 only|
+|[#11229](https://github.com/NVIDIA/spark-rapids/issues/11229)|[BUG] test_delta_name_column_mapping_no_field_ids fails on Spark |
+|[#11031](https://github.com/NVIDIA/spark-rapids/issues/11031)|Fix tests failures in multiple files |
+|[#10948](https://github.com/NVIDIA/spark-rapids/issues/10948)|Figure out why `MapFromArrays ` appears in the tests for hive parquet write|
+|[#11018](https://github.com/NVIDIA/spark-rapids/issues/11018)|Fix tests failures in hash_aggregate_test.py|
+|[#11173](https://github.com/NVIDIA/spark-rapids/issues/11173)|[BUG] The `rs. serialization time` metric is misleading|
+|[#11017](https://github.com/NVIDIA/spark-rapids/issues/11017)|Fix tests failures in url_test.py|
+|[#11201](https://github.com/NVIDIA/spark-rapids/issues/11201)|[BUG] Delta Lake tables with name mapping can throw exceptions on read|
+|[#11175](https://github.com/NVIDIA/spark-rapids/issues/11175)|[BUG] Clean up unused and duplicated 'org/roaringbitmap' folder in the spark3xx shims|
+|[#11196](https://github.com/NVIDIA/spark-rapids/issues/11196)|[BUG] pipeline failed due to class not found exception: NoClassDefFoundError: com/nvidia/spark/rapids/GpuScalar|
+|[#11189](https://github.com/NVIDIA/spark-rapids/issues/11189)|[BUG] regression in NDS after PR #11170|
+|[#11167](https://github.com/NVIDIA/spark-rapids/issues/11167)|[BUG] UnsupportedOperationException during delta write with `optimize()`|
+|[#11172](https://github.com/NVIDIA/spark-rapids/issues/11172)|[BUG] `get_json_object` returns wrong output with wildcard path|
+|[#11148](https://github.com/NVIDIA/spark-rapids/issues/11148)|[BUG] Integration test `test_write_hive_bucketed_table` fails|
+|[#11155](https://github.com/NVIDIA/spark-rapids/issues/11155)|[BUG] ArrayIndexOutOfBoundsException in BatchWithPartitionData.splitColumnarBatch|
+|[#11152](https://github.com/NVIDIA/spark-rapids/issues/11152)|[BUG] LORE dumping consumes too much memory.|
+|[#11029](https://github.com/NVIDIA/spark-rapids/issues/11029)|Fix tests failures in subquery_test.py|
+|[#11150](https://github.com/NVIDIA/spark-rapids/issues/11150)|[BUG] hive_parquet_write_test.py::test_insert_hive_bucketed_table failure|
+|[#11070](https://github.com/NVIDIA/spark-rapids/issues/11070)|[BUG] numpy2 fail fastparquet cases: numpy.dtype size changed|
+|[#11136](https://github.com/NVIDIA/spark-rapids/issues/11136)|UnaryPositive expression doesn't extend UnaryExpression|
+|[#11122](https://github.com/NVIDIA/spark-rapids/issues/11122)|[BUG] UT MetricRange failed 651070526 was not less than 1.5E8 in spark313|
+|[#11119](https://github.com/NVIDIA/spark-rapids/issues/11119)|[BUG] window_function_test.py::test_window_group_limits_fallback_for_row_number fails in a distributed environment|
+|[#11023](https://github.com/NVIDIA/spark-rapids/issues/11023)|Fix tests failures in dpp_test.py|
+|[#11026](https://github.com/NVIDIA/spark-rapids/issues/11026)|Fix tests failures in map_test.py|
+|[#11020](https://github.com/NVIDIA/spark-rapids/issues/11020)|Fix tests failures in grouping_sets_test.py|
+|[#11113](https://github.com/NVIDIA/spark-rapids/issues/11113)|[BUG] Update premerge tests for Scala 2.13 to use JDK 17 only|
+|[#11027](https://github.com/NVIDIA/spark-rapids/issues/11027)|Fix tests failures in sort_test.py|
+|[#10775](https://github.com/NVIDIA/spark-rapids/issues/10775)|[BUG] Issues found by Spark UT Framework on RapidsStringExpressionsSuite|
+|[#11033](https://github.com/NVIDIA/spark-rapids/issues/11033)|[BUG] CICD failed a case: cmp_test.py::test_empty_filter[>]|
+|[#11103](https://github.com/NVIDIA/spark-rapids/issues/11103)|[BUG] UCX Shuffle With scala.MatchError |
+|[#11007](https://github.com/NVIDIA/spark-rapids/issues/11007)|Fix tests failures in array_test.py|
+|[#10801](https://github.com/NVIDIA/spark-rapids/issues/10801)|[BUG] JDK17 nightly build after Spark UT Framework is merged|
+|[#11019](https://github.com/NVIDIA/spark-rapids/issues/11019)|Fix tests failures in window_function_test.py|
+|[#11063](https://github.com/NVIDIA/spark-rapids/issues/11063)|[BUG] op time for GpuCoalesceBatches is more than actual|
+|[#11006](https://github.com/NVIDIA/spark-rapids/issues/11006)|Fix test failures in arithmetic_ops_test.py|
+|[#10995](https://github.com/NVIDIA/spark-rapids/issues/10995)|Fallback TimeZoneAwareExpression that only support UTC with zoneId instead of timeZone config|
+|[#8652](https://github.com/NVIDIA/spark-rapids/issues/8652)|[BUG] array_item test failures on Spark 3.3.x|
+|[#11053](https://github.com/NVIDIA/spark-rapids/issues/11053)|[BUG] Build on Databricks 330 fails|
+|[#10925](https://github.com/NVIDIA/spark-rapids/issues/10925)| Concat cannot accept no parameter|
+|[#10975](https://github.com/NVIDIA/spark-rapids/issues/10975)|[BUG] regex `^.*literal` cannot be rewritten as `contains(literal)` for multiline strings|
+|[#10956](https://github.com/NVIDIA/spark-rapids/issues/10956)|[BUG] hive_parquet_write_test.py: test_write_compressed_parquet_into_hive_table integration test failures|
+|[#10772](https://github.com/NVIDIA/spark-rapids/issues/10772)|[BUG] Issues found by Spark UT Framework on RapidsDataFrameAggregateSuite|
+|[#10986](https://github.com/NVIDIA/spark-rapids/issues/10986)|[BUG]Cast from string to float using hand-picked values failed in CastOpSuite|
+|[#10972](https://github.com/NVIDIA/spark-rapids/issues/10972)|Spark 4.0 compile errors |
+|[#10794](https://github.com/NVIDIA/spark-rapids/issues/10794)|[BUG] Incorrect cast of string columns containing various infinity notations with trailing spaces |
+|[#10964](https://github.com/NVIDIA/spark-rapids/issues/10964)|[BUG] Improve stability of pre-merge jenkinsfile|
+|[#10714](https://github.com/NVIDIA/spark-rapids/issues/10714)|Signature changed for `PythonUDFRunner.writeUDFs` |
+|[#10712](https://github.com/NVIDIA/spark-rapids/issues/10712)|[AUDIT] BatchScanExec/DataSourceV2Relation to group splits by join keys if they differ from partition keys|
+|[#10673](https://github.com/NVIDIA/spark-rapids/issues/10673)|[AUDIT] Rename plan nodes for PythonMapInArrowExec|
+|[#10710](https://github.com/NVIDIA/spark-rapids/issues/10710)|[AUDIT] `uncacheTableOrView` changed in CommandUtils |
+|[#10711](https://github.com/NVIDIA/spark-rapids/issues/10711)|[AUDIT] Match DataSourceV2ScanExecBase changes to groupPartitions method |
+|[#10669](https://github.com/NVIDIA/spark-rapids/issues/10669)|Supporting broadcast of multiple filtering keys in DynamicPruning |
+
+### PRs
+|||
+|:---|:---|
+|[#11400](https://github.com/NVIDIA/spark-rapids/pull/11400)|[DOC] update notes in download page for the decompressing gzip issue [skip ci]|
+|[#11355](https://github.com/NVIDIA/spark-rapids/pull/11355)|Update changelog for the v24.08 release [skip ci]|
+|[#11353](https://github.com/NVIDIA/spark-rapids/pull/11353)|Update download doc for v24.08.1 [skip ci]|
+|[#11352](https://github.com/NVIDIA/spark-rapids/pull/11352)|Update version to 24.08.1-SNAPSHOT [skip ci]|
+|[#11337](https://github.com/NVIDIA/spark-rapids/pull/11337)|Update changelog for the v24.08 release [skip ci]|
+|[#11335](https://github.com/NVIDIA/spark-rapids/pull/11335)|Fix Delta Lake truncation of min/max string values|
+|[#11304](https://github.com/NVIDIA/spark-rapids/pull/11304)|Update changelog for v24.08.0 release [skip ci]|
+|[#11303](https://github.com/NVIDIA/spark-rapids/pull/11303)|Update rapids JNI and private dependency to 24.08.0|
+|[#11296](https://github.com/NVIDIA/spark-rapids/pull/11296)|[DOC] update doc for 2408 release [skip CI]|
+|[#11309](https://github.com/NVIDIA/spark-rapids/pull/11309)|[Doc ]Update lore doc about the range [skip ci]|
+|[#11292](https://github.com/NVIDIA/spark-rapids/pull/11292)|Add work around for string split with empty input.|
+|[#11278](https://github.com/NVIDIA/spark-rapids/pull/11278)|Fix formatting of advanced configs doc|
+|[#10917](https://github.com/NVIDIA/spark-rapids/pull/10917)|Adopt changes from JNI for casting from float to decimal|
+|[#11269](https://github.com/NVIDIA/spark-rapids/pull/11269)|Revert "upgrade ucx to 1.17.0"|
+|[#11260](https://github.com/NVIDIA/spark-rapids/pull/11260)|Mitigate intermittent test_buckets and shuffle_smoke_test OOM issue|
+|[#11268](https://github.com/NVIDIA/spark-rapids/pull/11268)|Fix degenerate conditional nested loop join detection|
+|[#11244](https://github.com/NVIDIA/spark-rapids/pull/11244)|Fix ArrayIndexOutOfBoundsException on join counts with constant join keys|
+|[#11259](https://github.com/NVIDIA/spark-rapids/pull/11259)|CI Docker to support integration tests with Rocky OS + jdk17 [skip ci]|
+|[#11247](https://github.com/NVIDIA/spark-rapids/pull/11247)|Fix `string_test.py` errors on Spark 4.0|
+|[#11246](https://github.com/NVIDIA/spark-rapids/pull/11246)|Rework Maven Source Plugin Skip|
+|[#11149](https://github.com/NVIDIA/spark-rapids/pull/11149)|Rework on substring index|
+|[#11236](https://github.com/NVIDIA/spark-rapids/pull/11236)|Remove the unused vars from the version-def CI script|
+|[#11237](https://github.com/NVIDIA/spark-rapids/pull/11237)|Fork jvm for maven-source-plugin|
+|[#11200](https://github.com/NVIDIA/spark-rapids/pull/11200)|Multi-get_json_object|
+|[#11230](https://github.com/NVIDIA/spark-rapids/pull/11230)|Skip test where Delta Lake may not be fully compatible with Spark|
+|[#11220](https://github.com/NVIDIA/spark-rapids/pull/11220)|Avoid failing spark bug SPARK-44242 while generate run_dir|
+|[#11226](https://github.com/NVIDIA/spark-rapids/pull/11226)|Fix auto merge conflict 11212|
+|[#11129](https://github.com/NVIDIA/spark-rapids/pull/11129)|Spark 4: Fix miscellaneous tests including logic, repart, hive_delimited.|
+|[#11163](https://github.com/NVIDIA/spark-rapids/pull/11163)|Support `MapFromArrays` on GPU|
+|[#11219](https://github.com/NVIDIA/spark-rapids/pull/11219)|Fix hash_aggregate_test.py to run with ANSI enabled|
+|[#11186](https://github.com/NVIDIA/spark-rapids/pull/11186)|from_json Json to Struct Exception Logging|
+|[#11180](https://github.com/NVIDIA/spark-rapids/pull/11180)|More accurate estimation for the result serialization time in RapidsShuffleThreadedWriterBase|
+|[#11194](https://github.com/NVIDIA/spark-rapids/pull/11194)|Fix ANSI mode test failures in url_test.py|
+|[#11202](https://github.com/NVIDIA/spark-rapids/pull/11202)|Fix read from Delta Lake table with name column mapping and missing Parquet IDs|
+|[#11185](https://github.com/NVIDIA/spark-rapids/pull/11185)|Fix multi-release jar problem|
+|[#11144](https://github.com/NVIDIA/spark-rapids/pull/11144)|Build the Scala2.13 dist jar with JDK17|
+|[#11197](https://github.com/NVIDIA/spark-rapids/pull/11197)|Fix class not found error: com/nvidia/spark/rapids/GpuScalar|
+|[#11191](https://github.com/NVIDIA/spark-rapids/pull/11191)|Fix dynamic pruning regression in GpuFileSourceScanExec|
+|[#10994](https://github.com/NVIDIA/spark-rapids/pull/10994)|Add Spark 4.0.0 Build Profile and Other Supporting Changes|
+|[#11192](https://github.com/NVIDIA/spark-rapids/pull/11192)|Append new authorized user to blossom-ci whitelist [skip ci]|
+|[#11179](https://github.com/NVIDIA/spark-rapids/pull/11179)|Allow more expressions to be tiered|
+|[#11141](https://github.com/NVIDIA/spark-rapids/pull/11141)|Enable some Rapids config in RapidsSQLTestsBaseTrait for Spark UT|
+|[#11170](https://github.com/NVIDIA/spark-rapids/pull/11170)|Avoid listFiles or inputFiles on relations with static partitioning|
+|[#11159](https://github.com/NVIDIA/spark-rapids/pull/11159)|Drop spark31x shims|
+|[#10951](https://github.com/NVIDIA/spark-rapids/pull/10951)|Case when performance improvement: reduce the `copy_if_else`|
+|[#11165](https://github.com/NVIDIA/spark-rapids/pull/11165)|Fix some GpuBroadcastToRowExec by not dropping columns|
+|[#11126](https://github.com/NVIDIA/spark-rapids/pull/11126)|Coalesce batches after a logical coalesce operation|
+|[#11164](https://github.com/NVIDIA/spark-rapids/pull/11164)|fix the bucketed write error for non-utc cases|
+|[#11132](https://github.com/NVIDIA/spark-rapids/pull/11132)|Add deletion vector metrics for low shuffle merge.|
+|[#11156](https://github.com/NVIDIA/spark-rapids/pull/11156)|Fix batch splitting for partition column size on row-count-only batches|
+|[#11153](https://github.com/NVIDIA/spark-rapids/pull/11153)|Fix LORE dump oom.|
+|[#11102](https://github.com/NVIDIA/spark-rapids/pull/11102)|Fix ANSI mode failures in subquery_test.py|
+|[#11151](https://github.com/NVIDIA/spark-rapids/pull/11151)|Fix the test error of the bucketed write for the non-utc case|
+|[#11147](https://github.com/NVIDIA/spark-rapids/pull/11147)|upgrade ucx to 1.17.0|
+|[#11138](https://github.com/NVIDIA/spark-rapids/pull/11138)|Update fastparquet to 2024.5.0 for numpy2 compatibility|
+|[#11137](https://github.com/NVIDIA/spark-rapids/pull/11137)|Handle the change for UnaryPositive now extending RuntimeReplaceable|
+|[#11094](https://github.com/NVIDIA/spark-rapids/pull/11094)|Add `HiveHash` support on GPU|
+|[#11139](https://github.com/NVIDIA/spark-rapids/pull/11139)|Improve MetricsSuite to allow more gc jitter|
+|[#11133](https://github.com/NVIDIA/spark-rapids/pull/11133)|Fix `test_window_group_limits_fallback`|
+|[#11097](https://github.com/NVIDIA/spark-rapids/pull/11097)|Fix miscellaneous integ tests for Spark 4|
+|[#11118](https://github.com/NVIDIA/spark-rapids/pull/11118)|Fix issue with DPP and AQE on reused broadcast exchanges|
+|[#11043](https://github.com/NVIDIA/spark-rapids/pull/11043)|Dataproc serverless test fixes|
+|[#10965](https://github.com/NVIDIA/spark-rapids/pull/10965)|Profiler: Disable collecting async allocation events by default|
+|[#11117](https://github.com/NVIDIA/spark-rapids/pull/11117)|Update Scala2.13 premerge CI against JDK17|
+|[#11084](https://github.com/NVIDIA/spark-rapids/pull/11084)|Introduce LORE framework.|
+|[#11099](https://github.com/NVIDIA/spark-rapids/pull/11099)|Spark 4: Handle ANSI mode in sort_test.py|
+|[#11115](https://github.com/NVIDIA/spark-rapids/pull/11115)|Fix match error in RapidsShuffleIterator.scala [scala2.13]|
+|[#11088](https://github.com/NVIDIA/spark-rapids/pull/11088)|Support regex patterns with brackets when rewriting to PrefixRange pattern in rlike.|
+|[#10950](https://github.com/NVIDIA/spark-rapids/pull/10950)|Add a heuristic to skip second or third agg pass|
+|[#11048](https://github.com/NVIDIA/spark-rapids/pull/11048)|Fixed array_tests for Spark 4.0.0|
+|[#11049](https://github.com/NVIDIA/spark-rapids/pull/11049)|Fix some cast_tests for Spark 4.0.0|
+|[#11066](https://github.com/NVIDIA/spark-rapids/pull/11066)|Replaced spark3xx-common references to spark-shared|
+|[#11083](https://github.com/NVIDIA/spark-rapids/pull/11083)|Exclude a case based on JDK version in Spark UT|
+|[#10997](https://github.com/NVIDIA/spark-rapids/pull/10997)|Fix some test issues in Spark UT and keep RapidsTestSettings update-to-date|
+|[#11073](https://github.com/NVIDIA/spark-rapids/pull/11073)|Disable ANSI mode for window function tests|
+|[#11076](https://github.com/NVIDIA/spark-rapids/pull/11076)|Improve the diagnostics for 'conv' fallback explain|
+|[#11092](https://github.com/NVIDIA/spark-rapids/pull/11092)|Add GpuBucketingUtils shim to Spark 4.0.0|
+|[#11062](https://github.com/NVIDIA/spark-rapids/pull/11062)|fix duplicate counted metrics like op time for GpuCoalesceBatches|
+|[#11044](https://github.com/NVIDIA/spark-rapids/pull/11044)|Fixed Failing tests in arithmetic_ops_tests for Spark 4.0.0|
+|[#11086](https://github.com/NVIDIA/spark-rapids/pull/11086)|upgrade blossom-ci actions version [skip ci]|
+|[#10957](https://github.com/NVIDIA/spark-rapids/pull/10957)|Support bucketing write for GPU|
+|[#10979](https://github.com/NVIDIA/spark-rapids/pull/10979)|[FEA] Introduce low shuffle merge.|
+|[#10996](https://github.com/NVIDIA/spark-rapids/pull/10996)|Fallback non-UTC TimeZoneAwareExpression with zoneId|
+|[#11072](https://github.com/NVIDIA/spark-rapids/pull/11072)|Workaround numpy2 failed fastparquet compatibility tests|
+|[#11046](https://github.com/NVIDIA/spark-rapids/pull/11046)|Calculate parallelism to speed up pre-merge CI|
+|[#11054](https://github.com/NVIDIA/spark-rapids/pull/11054)|fix flaky array_item test failures|
+|[#11051](https://github.com/NVIDIA/spark-rapids/pull/11051)|[FEA] Increase parallelism of deltalake test on databricks|
+|[#10993](https://github.com/NVIDIA/spark-rapids/pull/10993)|`binary-dedupe` changes for Spark 4.0.0|
+|[#11060](https://github.com/NVIDIA/spark-rapids/pull/11060)|Add in the ability to fingerprint JSON columns|
+|[#11059](https://github.com/NVIDIA/spark-rapids/pull/11059)|Revert "Add in the ability to fingerprint JSON columns (#11002)" [skip ci]|
+|[#11039](https://github.com/NVIDIA/spark-rapids/pull/11039)|Concat() Exception bug fix|
+|[#11002](https://github.com/NVIDIA/spark-rapids/pull/11002)|Add in the ability to fingerprint JSON columns|
+|[#10977](https://github.com/NVIDIA/spark-rapids/pull/10977)|Rewrite multiple literal choice regex to multiple contains in rlike|
+|[#11035](https://github.com/NVIDIA/spark-rapids/pull/11035)|Fix auto merge conflict 11034 [skip ci]|
+|[#11040](https://github.com/NVIDIA/spark-rapids/pull/11040)|Append new authorized user to blossom-ci whitelist [skip ci]|
+|[#11036](https://github.com/NVIDIA/spark-rapids/pull/11036)|Update blossom-ci ACL to secure format [skip ci]|
+|[#11032](https://github.com/NVIDIA/spark-rapids/pull/11032)|Fix a hive write test failure for Spark 350|
+|[#10998](https://github.com/NVIDIA/spark-rapids/pull/10998)|Improve log to print more lines in build [skip ci]|
+|[#10992](https://github.com/NVIDIA/spark-rapids/pull/10992)|Addressing the Named Parameter change in Spark 4.0.0|
+|[#10943](https://github.com/NVIDIA/spark-rapids/pull/10943)|Fix Spark UT issues in RapidsDataFrameAggregateSuite|
+|[#10963](https://github.com/NVIDIA/spark-rapids/pull/10963)|Add rapids configs to enable GPU running in Spark UT|
+|[#10978](https://github.com/NVIDIA/spark-rapids/pull/10978)|More compilation fixes for Spark 4.0.0|
+|[#10953](https://github.com/NVIDIA/spark-rapids/pull/10953)|Speed up the integration tests by running them in parallel on the Databricks cluster|
+|[#10958](https://github.com/NVIDIA/spark-rapids/pull/10958)|Fix a hive write test failure|
+|[#10970](https://github.com/NVIDIA/spark-rapids/pull/10970)|Move Support for `RaiseError` to a Shim Excluding Spark 4.0.0|
+|[#10966](https://github.com/NVIDIA/spark-rapids/pull/10966)|Add default value for REF of premerge jenkinsfile to avoid bad overwritten [skip ci]|
+|[#10959](https://github.com/NVIDIA/spark-rapids/pull/10959)|Add new ID to blossom-ci allow list [skip ci]|
+|[#10952](https://github.com/NVIDIA/spark-rapids/pull/10952)|Add shims to take care of the signature change for writeUDFs in PythonUDFRunner|
+|[#10931](https://github.com/NVIDIA/spark-rapids/pull/10931)|Add Support for Renaming of PythonMapInArrow|
+|[#10949](https://github.com/NVIDIA/spark-rapids/pull/10949)|Change dependency version to 24.08.0-SNAPSHOT|
+|[#10857](https://github.com/NVIDIA/spark-rapids/pull/10857)|[Spark 4.0] Account for `PartitionedFileUtil.splitFiles` signature change.|
+|[#10912](https://github.com/NVIDIA/spark-rapids/pull/10912)|GpuInsertIntoHiveTable supports parquet format|
+|[#10863](https://github.com/NVIDIA/spark-rapids/pull/10863)|[Spark 4.0] Account for `CommandUtils.uncacheTableOrView` signature change.|
+|[#10944](https://github.com/NVIDIA/spark-rapids/pull/10944)|Added Shim for BatchScanExec to Support Spark 4.0|
+|[#10946](https://github.com/NVIDIA/spark-rapids/pull/10946)|Unarchive Spark test jar for spark.read(ability)|
+|[#10945](https://github.com/NVIDIA/spark-rapids/pull/10945)|Add Support for Multiple Filtering Keys for Subquery Broadcast|
+|[#10871](https://github.com/NVIDIA/spark-rapids/pull/10871)|Add classloader diagnostics to initShuffleManager error message|
+|[#10933](https://github.com/NVIDIA/spark-rapids/pull/10933)|Fixed Databricks build|
+|[#10929](https://github.com/NVIDIA/spark-rapids/pull/10929)|Append new authorized user to blossom-ci whitelist [skip ci]|
+
## Release 24.06
### Features
From 795aef8dade72d76b42a247a3b7bd8eee03332b7 Mon Sep 17 00:00:00 2001
From: Tim Liu
Date: Mon, 16 Dec 2024 11:22:35 +0800
Subject: [PATCH 102/103] Remove 350db143 shim's build (#11874)
Skip the build of the 350db143 shim, as v24.12.0 will not contain the 350db143 shim
Moreover, the v24.12.0 private dependency jar is not released.
To fix below error:
[ERROR] Failed to execute goal on project rapids-4-spark-sql_2.12: Could not resolve dependencies for
project com.nvidia:rapids-4-spark-sql_2.12:jar:24.12.0: Failure to find com.nvidia:rapids-4-spark-private_2.12:jar:spark350db143:24.12.0
in https://repo1.maven.org/maven2 was cached in the local repository, resolution will not be reattempted until the update interval of central has elapsed or updates are forced
Signed-off-by: Tim Liu
---
jenkins/Jenkinsfile-blossom.premerge-databricks | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/jenkins/Jenkinsfile-blossom.premerge-databricks b/jenkins/Jenkinsfile-blossom.premerge-databricks
index 2f8b926898a..cfe3cf6940e 100644
--- a/jenkins/Jenkinsfile-blossom.premerge-databricks
+++ b/jenkins/Jenkinsfile-blossom.premerge-databricks
@@ -91,7 +91,7 @@ pipeline {
// 'name' and 'value' only supprt literal string in the declarative Jenkins
// Refer to Jenkins issue https://issues.jenkins.io/browse/JENKINS-62127
name 'DB_RUNTIME'
- values '11.3', '12.2', '13.3', '14.3'
+ values '11.3', '12.2', '13.3'
}
}
stages {
From d1ea935626797435fbf948abafd6280d12847307 Mon Sep 17 00:00:00 2001
From: Jenkins Automation <70000568+nvauto@users.noreply.github.com>
Date: Mon, 16 Dec 2024 11:33:30 +0800
Subject: [PATCH 103/103] Update latest changelog [skip ci] (#11876)
Update change log with CLI: \n\n scripts/generate-changelog --token= --releases=24.10,24.12
Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com>
---
CHANGELOG.md | 3 +++
1 file changed, 3 insertions(+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 5dd6c7b19ba..d78202be823 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -72,7 +72,10 @@ Generated on 2024-12-16
### PRs
|||
|:---|:---|
+|[#11874](https://github.com/NVIDIA/spark-rapids/pull/11874)|Remove 350db143 shim's build [skip ci]|
+|[#11851](https://github.com/NVIDIA/spark-rapids/pull/11851)|Update latest changelog [skip ci]|
|[#11849](https://github.com/NVIDIA/spark-rapids/pull/11849)|Update rapids JNI and private dependency to 24.12.0|
+|[#11841](https://github.com/NVIDIA/spark-rapids/pull/11841)|[DOC] update doc for 24.12 release [skip ci]|
|[#11857](https://github.com/NVIDIA/spark-rapids/pull/11857)|Increase the pre-merge CI timeout to 6 hours|
|[#11845](https://github.com/NVIDIA/spark-rapids/pull/11845)|Fix leak in isTimeStamp|
|[#11823](https://github.com/NVIDIA/spark-rapids/pull/11823)|Fix for `LEAD/LAG` window function test failures.|