Skip to content

Conversation

@cdkrot
Copy link
Contributor

@cdkrot cdkrot commented Dec 6, 2023

What changes were proposed in this pull request?

Revert #44144, and introduce a forbid recursion guard as previously proposed. This way the infinite error handling recursion is still prevented, but the client-side knob is still present.

Why are the changes needed?

Previously proposed as part of #44144, however was discussed in favour of something else. However it seems (proposal by @grundprinzip) that the original proposal was more correct, since it seems driver stacktrace is decided on client not server (see #43667)

Does this PR introduce any user-facing change?

No

How was this patch tested?

Hand testing

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

No

@cdkrot
Copy link
Contributor Author

cdkrot commented Dec 6, 2023

FYI @heyihong. @grundprinzip proposed to take that change we discussed back and change to my original version

@heyihong
Copy link
Contributor

heyihong commented Dec 6, 2023

@cdkrot I don't have strong opinion on either approach. But we probably need to implement similar logics for Scala client as well https://github.com/apache/spark/blob/master/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala#L339-L347 (not necessarily in this pr)

client = SparkConnectClient(chan)
self.assertEqual(client._session_id, chan.session_id)

def test_forbid_recursion(self):
Copy link
Contributor

Choose a reason for hiding this comment

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

this test does not test directly the scenario, we're talking about. Ideally you can just use the mock tests we have to fail any query and see that the recursion guard works.

Copy link
Contributor Author

@cdkrot cdkrot Dec 6, 2023

Choose a reason for hiding this comment

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

I actually tried, but it seems hard to make the mock test for this because it needs to pass through this pieces of code:

status = rpc_status.from_call(cast(grpc.Call, rpc_error))

This seems hard to create a mock exception which would pass this without poking grpc's internals significantly. Alternatively we could introduce some testing clutches here, i.e. check if exception is from testing code, but that's not great either.

@grundprinzip

Copy link
Contributor

@grundprinzip grundprinzip left a comment

Choose a reason for hiding this comment

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

I requested changes to the code that I believe make it more readable and require less changes in general. Explicitly, I think we should use a simpler flag mechanisms to track if we're processing an exception or not instead of adding a new class.

@cdkrot
Copy link
Contributor Author

cdkrot commented Dec 6, 2023

changed based on @grundprinzip request

@cdkrot cdkrot requested a review from grundprinzip December 7, 2023 09:50
@cdkrot
Copy link
Contributor Author

cdkrot commented Dec 7, 2023

removed

@cdkrot
Copy link
Contributor Author

cdkrot commented Dec 7, 2023

cc @HyukjinKwon

@grundprinzip
Copy link
Contributor

Please update PR title

raise error

try:
self._inside_error_handling = True
Copy link
Contributor

@heyihong heyihong Dec 7, 2023

Choose a reason for hiding this comment

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

I know Python has Global Interpreter Lock but is this thread-safe?

@grundprinzip
Copy link
Contributor

Dang, I wanted to leave a comment on that. It's not thread safe.

@cdkrot
Copy link
Contributor Author

cdkrot commented Dec 7, 2023

Ah, I'm a simple person, and was simply following your review. Original proposal was thread safe.

@cdkrot cdkrot changed the title [SPARK-TBD] Forbid recursive error handling 2 [SPARK-TBD] Forbid recursive error handling by adding recursion guards Dec 7, 2023
@cdkrot cdkrot changed the title [SPARK-TBD] Forbid recursive error handling by adding recursion guards [SPARK-46308] Forbid recursive error handling by adding recursion guards Dec 7, 2023
@cdkrot
Copy link
Contributor Author

cdkrot commented Dec 7, 2023

updated title; added threadlocals

@cdkrot
Copy link
Contributor Author

cdkrot commented Dec 15, 2023

It seems we forgot to merge this because there were some unrelated test fails again. I retriggered those and test pass.

@HyukjinKwon
Copy link
Member

Merged to master.

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

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

This PR changes the `thread.local` in `SparkConnectClient` to be used properly to fix the bug caused by #44210. It mistakenly used `thread.local` wrongly by inheriting `thread.local` and setting the class-level variables which always exist.

### Why are the changes needed?

So users can properly use thread-based `interruptTag`. Now the code below cancels both queries:

```python
import concurrent.futures
import time
import threading
from pyspark.sql.functions import udf

def run_query_with_tag(query, tag):
    try:
        spark.addTag(tag)
        print(f"starting query {tag}")
        df = spark.sql(query).select(udf(lambda: time.sleep(10))())
        print(f"collecting query {tag}")
        res = df.collect()
        print(f"done with query {tag}")
    finally:
        spark.removeTag(tag)

queries_with_tags = [
    ("SELECT * FROM range(100)", "tag1"),
    ("SELECT * FROM range(100)", "tag2"),
]

with concurrent.futures.ThreadPoolExecutor() as executor:
    futures = {executor.submit(run_query_with_tag, query, tag): (query, tag) for query, tag in queries_with_tags}
    time.sleep(5)
    print("Interrupting tag1")
    print(spark.interruptTag("tag1"))
    for f in futures:
        try:
            f.result()
            print(f"done with {f.result()}")
        except:
            print(f"failed with {f.exception()}")
```

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

No, this was caused by #44210 but the change has not been released out.

### How was this patch tested?

Unittest was added.

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

No.

Closes #47005 from HyukjinKwon/thread-local.

Authored-by: Hyukjin Kwon <[email protected]>
Signed-off-by: Hyukjin Kwon <[email protected]>
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.

4 participants