NarekDW opened a new pull request, #39097:
URL: https://github.com/apache/spark/pull/39097
### What changes were proposed in this pull request?
This PR enhances `StructsToCsv` class with `doGenCode` function instead of
extending it from `CodegenFallback` trait, in order to make it deterministic.
Example:
```scala
import org.apache.spark.sql._
import org.apache.spark.sql.types._
import org.apache.spark.sql.functions._
val sparkSession = SparkSession.builder().getOrCreate()
val df = sparkSession.sparkContext.parallelize(1 to 5).toDF("x")
val v1 = rand().*(lit(10000)).cast(IntegerType)
val v2 = to_csv(struct(v1.as("a")))
df.select(v1, v1, v2, v2).show()
```
before this changes, the result was something like:
```scala
+----+----+----+----+
| a| b| c| d|
+----+----+----+----+
| 922| 922| 922|2028|
|5571|5571|5571|1640|
|5612|5612|5612|769 |
|2068|2068|2068|8924|
|5755|5755|5755|2731|
+----+----+----+----+
```
With current changes, the result looks like:
```scala
+----+----+----+----+
| a| b| c| d|
+----+----+----+----+
| 922| 922| 922| 922|
|5571|5571|5571|5571|
|5612|5612|5612|5612|
|2068|2068|2068|2068|
|5755|5755|5755|5755|
+----+----+----+----+
```
### Why are the changes needed?
To make to_csv function deterministic.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
An additional test case was added to
[CsvFunctionsSuite](https://github.com/NarekDW/spark/blob/13ab259815ad2b9351010df15ab23aacfa7ee14e/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala)
```scala
test("SPARK-41049: make to_csv function deterministic") {
...
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]