Skip to content

[SPARK-47274][PYTHON][SQL] Provide more useful context for PySpark DataFrame API errors #45377

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 24 commits into from

Conversation

itholic
Copy link
Contributor

@itholic itholic commented Mar 5, 2024

What changes were proposed in this pull request?

This PR introduces an enhancement to the error messages generated by PySpark's DataFrame API, adding detailed context about the location within the user's PySpark code where the error occurred.

This directly adds a PySpark user call site information into DataFrameQueryContext added from #43334, aiming to provide PySpark users with the same level of detailed error context for better usability and debugging efficiency for DataFrame APIs.

This PR also introduces QueryContext.pysparkCallSite and QueryContext.pysparkFragment to get a PySpark information from the query context easily.

This PR also enhances the functionality of check_error so that it can test the query context if it exists.

Why are the changes needed?

To improve a debuggability. Errors originating from PySpark operations can be difficult to debug with limited context in the error messages. While improvements on the JVM side have been made to offer detailed error contexts, PySpark errors often lack this level of detail.

Does this PR introduce any user-facing change?

No API changes, but error messages will include a reference to the exact line of user code that triggered the error, in addition to the existing descriptive error message.

For example, consider the following PySpark code snippet that triggers a DIVIDE_BY_ZERO error:

1  spark.conf.set("spark.sql.ansi.enabled", True)
2  
3  df = spark.range(10)
4  df.select(df.id / 0).show()

Before:

pyspark.errors.exceptions.captured.ArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. SQLSTATE: 22012
== DataFrame ==
"divide" was called from
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)

After:

pyspark.errors.exceptions.captured.ArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. SQLSTATE: 22012
== DataFrame ==
"divide" was called from
/.../spark/python/test_pyspark_error.py:4

Now the error message points out the exact problematic code path with file name and line number that user writes.

Points to the actual problem site instead of the site where the action was called

Even when action calling after multiple transform operations are mixed, the exact problematic site can be provided to the user:

In:

  1 spark.conf.set("spark.sql.ansi.enabled", True)
  2 df = spark.range(10)
  3
  4 df1 = df.withColumn("div_ten", df.id / 10)
  5 df2 = df1.withColumn("plus_four", df.id + 4)
  6
  7 # This is problematic divide operation that occurs DIVIDE_BY_ZERO.
  8 df3 = df2.withColumn("div_zero", df.id / 0)
  9 df4 = df3.withColumn("minus_five", df.id / 5)
 10
 11 df4.collect()

Out:

pyspark.errors.exceptions.captured.ArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. SQLSTATE: 22012
== DataFrame ==
"divide" was called from
/.../spark/python/test_pyspark_error.py:8

How was this patch tested?

Added UTs.

Was this patch authored or co-authored using generative AI tooling?

No.

@itholic
Copy link
Contributor Author

itholic commented Mar 5, 2024

cc @HyukjinKwon I'm still working on Spark Connect support and unit tests but the basic structure is ready for review.

FYI, also cc @MaxGekk as you made a similar contribution on the JVM side.

@itholic itholic changed the title [WIP][SPARK-47274][PYTHON][CONNECT] Provide more useful context for PySpark DataFrame API errors [WIP][SPARK-47274][PYTHON][SQL][CONNECT] Provide more useful context for PySpark DataFrame API errors Mar 28, 2024
@itholic itholic force-pushed the error_context_for_dataframe_api branch from 24fd3a0 to 2c1d5d8 Compare March 28, 2024 15:42
@itholic itholic changed the title [WIP][SPARK-47274][PYTHON][SQL][CONNECT] Provide more useful context for PySpark DataFrame API errors [SPARK-47274][PYTHON][SQL][CONNECT] Provide more useful context for PySpark DataFrame API errors Mar 28, 2024
@itholic itholic marked this pull request as ready for review March 28, 2024 15:42
@itholic
Copy link
Contributor Author

itholic commented Mar 28, 2024

Hi, @HyukjinKwon @MaxGekk could you take a look at the prototype when you have some time?

@itholic itholic changed the title [SPARK-47274][PYTHON][SQL][CONNECT] Provide more useful context for PySpark DataFrame API errors [SPARK-47274][PYTHON][SQL] Provide more useful context for PySpark DataFrame API errors Apr 1, 2024
@itholic
Copy link
Contributor Author

itholic commented Apr 3, 2024

Added QueryContext testing for DataFrameContext and UTs. The CI failures seems not related. cc @HyukjinKwon FYI

@HyukjinKwon
Copy link
Member

cc @cloud-fan too

exception=pe.exception,
error_class="INVALID_IDENTIFIER",
message_parameters={"ident": "non-existing-table"},
query_context_type=None,
Copy link
Contributor Author

@itholic itholic Apr 3, 2024

Choose a reason for hiding this comment

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

FYI: None is default, so we don't need to specify like this when QueryContext not existing, but I made this test for explicit example.

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in 86ae0d2 Apr 11, 2024
@itholic
Copy link
Contributor Author

itholic commented Apr 11, 2024

Thanks @cloud-fan @ueshin @HyukjinKwon @xinrong-meng for the review!

@HyukjinKwon
Copy link
Member

Let's clarify why #45377 (comment) happens before we move further. That shouldn't happen from my understanding.

If we go with the current approach, it would need more changes. e.g., Column.substr because it takes a different set of arguments and types.

* operation originated.
* @return A Column resulting from the operation.
*/
private def fn(
Copy link
Contributor

@cloud-fan cloud-fan Apr 15, 2024

Choose a reason for hiding this comment

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

@HyukjinKwon This probably can't cover all the cases, and we may need to add more overloads for certain functions that require non-expression parameters, but it shouldn't be many.

I think it's better than using ThreadLocal which can be quite fragile to pass values between Python and JVM.

@itholic
Copy link
Contributor Author

itholic commented Apr 15, 2024

The difficulty with the previous method was that it was not easy to perfectly sync the data between two separately operating TheadLocal, CurrentOrigin and PySparkCurrentOrigin.

After taking deeper look at the structure, I think we may be able to make the CurrentOrigin more flexible to support PySpark error context instead of adding a separate ThreadLocal like PySparkCurrentOrigin.

If it works, it seems possible to improve the structure to a more flexible while maintaining the existing communication rules between Python and JVM without adding helper functions such as PySpark-specific fn.

Let me give it a try and create a PR to refactoring the current structure, and ping you guys.

@itholic
Copy link
Contributor Author

itholic commented Apr 15, 2024

Let me give it a try and create a PR to refactoring the current structure, and ping you guys.

Created #46063.

@HyukjinKwon
Copy link
Member

perfectly sync the data between two separately operating TheadLocal, CurrentOrigin and PySparkCurrentOrigin.

Why is that?

@itholic
Copy link
Contributor Author

itholic commented Apr 16, 2024

Because I called PySparkCurrentOrigin directly on the DataFrameQueryContext without utilizing withOrigin in the initial implementation. I realized it from recent review from the refactoring PR, so I'm currently trying to reintroduce PySparkCurrentOrigin there.

HyukjinKwon added a commit that referenced this pull request Jun 18, 2024
…rk Connect

### What changes were proposed in this pull request?

This PR proposes to Implement DataFrameQueryContext in Spark Connect.

1.  Add two new protobuf messages packed together with `Expression`:

    ```proto
    message Origin {
      // (Required) Indicate the origin type.
      oneof function {
        PythonOrigin python_origin = 1;
      }
    }

    message PythonOrigin {
      // (Required) Name of the origin, for example, the name of the function
      string fragment = 1;

      // (Required) Callsite to show to end users, for example, stacktrace.
      string call_site = 2;
    }
    ```

2. Merge `DataFrameQueryContext.pysparkFragment` and `DataFrameQueryContext.pysparkcallSite` to existing `DataFrameQueryContext.fragment` and `DataFrameQueryContext.callSite`

3. Separate `QueryContext` into `SQLQueryContext` and `DataFrameQueryContext` for consistency w/ Scala side

4. Implement the origin logic. `current_origin` thread local holds the current call site/the function name, and `Expression` gets it from it.
    They are set to individual expression messages, and are used when analysis happens - this resembles Spark SQL implementation.

See also #45377.

### Why are the changes needed?

See #45377

### Does this PR introduce _any_ user-facing change?

Yes, same as #45377 but in Spark Connect.

### How was this patch tested?

Same unittests reused in Spark Connect.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #46789 from HyukjinKwon/connect-context.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
HyukjinKwon pushed a commit that referenced this pull request Jun 24, 2024
### What changes were proposed in this pull request?

This PR proposes to display correct call site information from IPython Notebook.

### Why are the changes needed?

We added `DataFrameQueryContext` for PySpark error message from #45377, but it does not working very well from IPython Notebook.

### Does this PR introduce _any_ user-facing change?

No API changes, but the user-facing error message from IPython Notebook will be improved:

**Before**
<img width="1124" alt="Screenshot 2024-06-18 at 5 15 56 PM" src="https://github.com/apache/spark/assets/44108233/3e3aee2c-5bb0-4858-b392-e845b7280d31">

**After**
<img width="1163" alt="Screenshot 2024-06-19 at 8 45 05 AM" src="https://github.com/apache/spark/assets/44108233/81741d15-cac9-41e7-815a-5d84f1176c73">

**NOTE:** This also works when command is executed across multiple cells:

<img width="1175" alt="Screenshot 2024-06-19 at 8 42 29 AM" src="https://github.com/apache/spark/assets/44108233/d65fbf79-d621-4ae0-b220-2f7923cc3666">

### How was this patch tested?

Manually tested with IPython Notebook.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #47009 from itholic/error_context_on_notebook.

Authored-by: Haejoon Lee <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
Copy link
Member

@gengliangwang gengliangwang left a comment

Choose a reason for hiding this comment

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

Late LGTM!

attilapiros pushed a commit to attilapiros/spark that referenced this pull request Oct 4, 2024
### What changes were proposed in this pull request?

This PR proposes to display correct call site information from IPython Notebook.

### Why are the changes needed?

We added `DataFrameQueryContext` for PySpark error message from apache#45377, but it does not working very well from IPython Notebook.

### Does this PR introduce _any_ user-facing change?

No API changes, but the user-facing error message from IPython Notebook will be improved:

**Before**
<img width="1124" alt="Screenshot 2024-06-18 at 5 15 56 PM" src="https://github.com/apache/spark/assets/44108233/3e3aee2c-5bb0-4858-b392-e845b7280d31">

**After**
<img width="1163" alt="Screenshot 2024-06-19 at 8 45 05 AM" src="https://github.com/apache/spark/assets/44108233/81741d15-cac9-41e7-815a-5d84f1176c73">

**NOTE:** This also works when command is executed across multiple cells:

<img width="1175" alt="Screenshot 2024-06-19 at 8 42 29 AM" src="https://github.com/apache/spark/assets/44108233/d65fbf79-d621-4ae0-b220-2f7923cc3666">

### How was this patch tested?

Manually tested with IPython Notebook.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes apache#47009 from itholic/error_context_on_notebook.

Authored-by: Haejoon Lee <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants