Skip to content
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

[SPARK-48810][CONNECT] Session stop() API should be idempotent and not fail if the session is already closed by the server. #47215

Closed
wants to merge 7 commits into from

Conversation

nija-at
Copy link
Contributor

@nija-at nija-at commented Jul 4, 2024

What changes were proposed in this pull request?

Improve the error handling of the stop() API in the SparkSesion
class to not throw if there is any error related to releasing a session or
closing the underlying GRPC channel. Both are best effort.

In the case of Pyspark, do not fail if the local Spark Connect service
cannot be stopped.

Why are the changes needed?

In some cases, the Spark Connect Service will terminate the session, usually
because the underlying cluster or driver has restarted.
In the cases, calling stop() throws an error which is unactionable. However,
stop() still needs to be called in order to reset the active session.

Further, the stop() API should be idempotent.

Does this PR introduce any user-facing change?

No.

How was this patch tested?

Attached unit tests.

Confirmed that removing the code changes results in the
tests failing (as expected).

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

No.

Comment on lines +851 to +857
try:
PySparkSession._activeSession.stop()
except Exception as e:
warnings.warn(
"session.stop(): Local Spark Connect Server could not be stopped. "
f"Error: ${e}"
)
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm wondering if this one actually needs to be wrapped because. If I'm reading this correctly, the PySparkSession._activeSession is actually a Java Session.

image

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It is the underlying Java session.

Unclear. Felt safer such as in the case that the JVM is killed out of band. I don't know all the failure modes in this code path.

@nija-at nija-at changed the title session stop [SPARK-48810]session stop Jul 4, 2024
@nija-at nija-at changed the title [SPARK-48810]session stop [SPARK-48810] session stop Jul 4, 2024
@nija-at nija-at changed the title [SPARK-48810] session stop [SPARK-48810][CONNECT] Session stop() API should be idempotent and not fail if the session is already closed by the server. Jul 4, 2024
@nija-at nija-at marked this pull request as ready for review July 4, 2024 15:26
@nija-at nija-at requested a review from grundprinzip July 4, 2024 15:26
Copy link
Contributor

@hvanhovell hvanhovell left a comment

Choose a reason for hiding this comment

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

LGTM

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in 0b0bf4f Jul 8, 2024
@cloud-fan
Copy link
Contributor

@nija-at please open a 3.5 backport PR if needed, thanks!

@prabodh1194
Copy link

wow @nija-at . please add this to 3.5 as well.
i see that the artifacts keeps piling up on my local disk. this PR will help keep my disk clean.

ericm-db pushed a commit to ericm-db/spark that referenced this pull request Jul 10, 2024
…t fail if the session is already closed by the server

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

Improve the error handling of the `stop()` API in the `SparkSesion`
class to not throw if there is any error related to releasing a session or
closing the underlying GRPC channel. Both are best effort.

In the case of Pyspark, do not fail if the local Spark Connect service
cannot be stopped.

### Why are the changes needed?

In some cases, the Spark Connect Service will terminate the session, usually
because the underlying cluster or driver has restarted.
In the cases, calling stop() throws an error which is unactionable. However,
stop() still needs to be called in order to reset the active session.

Further, the stop() API should be idempotent.

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

No.

### How was this patch tested?

Attached unit tests.

Confirmed that removing the code changes results in the
tests failing (as expected).

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

No.

Closes apache#47215 from nija-at/session-stop.

Authored-by: Niranjan Jayakar <nija@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
jingz-db pushed a commit to jingz-db/spark that referenced this pull request Jul 22, 2024
…t fail if the session is already closed by the server

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

Improve the error handling of the `stop()` API in the `SparkSesion`
class to not throw if there is any error related to releasing a session or
closing the underlying GRPC channel. Both are best effort.

In the case of Pyspark, do not fail if the local Spark Connect service
cannot be stopped.

### Why are the changes needed?

In some cases, the Spark Connect Service will terminate the session, usually
because the underlying cluster or driver has restarted.
In the cases, calling stop() throws an error which is unactionable. However,
stop() still needs to be called in order to reset the active session.

Further, the stop() API should be idempotent.

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

No.

### How was this patch tested?

Attached unit tests.

Confirmed that removing the code changes results in the
tests failing (as expected).

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

No.

Closes apache#47215 from nija-at/session-stop.

Authored-by: Niranjan Jayakar <nija@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
attilapiros pushed a commit to attilapiros/spark that referenced this pull request Oct 4, 2024
…t fail if the session is already closed by the server

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

Improve the error handling of the `stop()` API in the `SparkSesion`
class to not throw if there is any error related to releasing a session or
closing the underlying GRPC channel. Both are best effort.

In the case of Pyspark, do not fail if the local Spark Connect service
cannot be stopped.

### Why are the changes needed?

In some cases, the Spark Connect Service will terminate the session, usually
because the underlying cluster or driver has restarted.
In the cases, calling stop() throws an error which is unactionable. However,
stop() still needs to be called in order to reset the active session.

Further, the stop() API should be idempotent.

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

No.

### How was this patch tested?

Attached unit tests.

Confirmed that removing the code changes results in the
tests failing (as expected).

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

No.

Closes apache#47215 from nija-at/session-stop.

Authored-by: Niranjan Jayakar <nija@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
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.

5 participants