Skip to content

Conversation

@Brijesh-Thakkar
Copy link

Which issue does this PR close?

Closes #3016

Rationale for this change

to_json could emit invalid JSON when encountering special floating-point values
such as NaN, +Infinity, or -Infinity. These values are not valid in JSON and
resulted in incorrect or unparsable output.

Apache Spark normalizes such values to null when converting to JSON. Since
DataFusion-Comet aims to be Spark-compatible, this behavior needed to be aligned.

What changes are included in this PR?

  • Normalize NaN, +Infinity, and -Infinity values to null during to_json
    conversion
  • Ensure to_json always produces valid JSON output
  • Add a regression test covering special floating-point values

How are these changes tested?

  • Added a unit test verifying to_json behavior for NaN, +Infinity, and
    -Infinity values
  • All existing tests in the native/spark-expr crate pass

Copilot AI review requested due to automatic review settings December 31, 2025 20:45
Copy link

Copilot AI left a comment

Choose a reason for hiding this comment

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

Pull request overview

This PR fixes invalid JSON generation when to_json encounters special floating-point values (NaN, Infinity, -Infinity). These values are now normalized to null for Spark compatibility, ensuring valid JSON output is always produced.

Key Changes:

  • Introduced normalize_special_floats function to convert special float string representations to null
  • Modified array_to_json_string to apply normalization after casting non-struct arrays to strings
  • Added comprehensive test coverage for NaN and Infinity handling

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

builder.append_null();
} else {
match arr.value(i) {
"Infinity" | "-Infinity" | "NaN" => builder.append_null(),
Copy link
Member

Choose a reason for hiding this comment

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

I haven't reviewed this in detail yet, but it seems odd to handle these values after they have already been converted to strings. Could the check not happen when converting float to string instead?

Copy link
Author

Choose a reason for hiding this comment

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

I agree that handling this earlier would be preferable in general. In this case, to_json delegates primitive type handling to spark_cast, and the goal here was to avoid changing spark_cast behavior globally since it is used by other expressions where preserving "NaN" / "Infinity" string output may be expected.

Normalizing the values at the to_json layer keeps the change scoped specifically to JSON semantics while still aligning the output with Spark’s behavior.

That said, I’m happy to move the check earlier or adjust the approach if you think handling this during float-to-string conversion would be more appropriate for Comet

Copy link
Contributor

Choose a reason for hiding this comment

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

@andygrove for more details, run the unit tests from this pull request: #3011

Copy link
Member

Choose a reason for hiding this comment

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

Thanks @Brijesh-Thakkar and @kazantsev-maksim. I will review this PR more carefully today.

Copy link
Member

Choose a reason for hiding this comment

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

I checked out this branch locally and then merged the changes from #3011 and saw different results from Spark and Comet for infinity:

Spark: [{"c0":false, ... "c5":"-Infinity", ...}]
Comet: [{"c0":false, ... "c5":-Infinity, ...}]

Let's get the unit tests merged first and then pull them into this PR.

Copy link
Member

Choose a reason for hiding this comment

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

Spark doesn't follow the JSON spec:

scala> spark.sql("""SELECT to_json(struct(cast('Infinity' as double) as inf, 
     |                       cast('NaN' as double) as nan))""").show(false)
+----------------------------------------------------------------------------+
|to_json(struct(CAST(Infinity AS DOUBLE) AS inf, CAST(NaN AS DOUBLE) AS nan))|
+----------------------------------------------------------------------------+
|{"inf":"Infinity","nan":"NaN"}                                              |
+----------------------------------------------------------------------------+

@andygrove andygrove changed the title Fix to_json handling of NaN and Infinity values (#3016) fix: Fix to_json handling of NaN and Infinity values (#3016) Jan 5, 2026
@codecov-commenter
Copy link

codecov-commenter commented Jan 5, 2026

Codecov Report

✅ All modified and coverable lines are covered by tests.
✅ Project coverage is 59.55%. Comparing base (f09f8af) to head (3cebe45).
⚠️ Report is 817 commits behind head on main.

Additional details and impacted files
@@             Coverage Diff              @@
##               main    #3018      +/-   ##
============================================
+ Coverage     56.12%   59.55%   +3.43%     
- Complexity      976     1368     +392     
============================================
  Files           119      167      +48     
  Lines         11743    15496    +3753     
  Branches       2251     2569     +318     
============================================
+ Hits           6591     9229    +2638     
- Misses         4012     4970     +958     
- Partials       1140     1297     +157     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Bug: to_json does not support +Infinity, -Infinity for numeric types

4 participants