-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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-48258][PYTHON][CONNECT] Checkpoint and localCheckpoint in Spark Connect #46570
Conversation
88b3fa1
to
7a4d6f7
Compare
d336333
to
6a3f14d
Compare
connector/connect/common/src/main/protobuf/spark/connect/base.proto
Outdated
Show resolved
Hide resolved
...ector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
Show resolved
Hide resolved
connector/connect/common/src/main/protobuf/spark/connect/base.proto
Outdated
Show resolved
Hide resolved
...connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala
Show resolved
Hide resolved
connector/connect/common/src/main/protobuf/spark/connect/commands.proto
Outdated
Show resolved
Hide resolved
4b7f7a1
to
cf26e14
Compare
…tDir ### What changes were proposed in this pull request? This PR adds `spark.checkpoint.dir` configuration so users can set the checkpoint dir when they submit their application. ### Why are the changes needed? Separate the configuration logic so the same app can run with a different checkpoint. In addition, this would be useful for Spark Connect with #46570. ### Does this PR introduce _any_ user-facing change? Yes, it adds a new user-facing configuration. ### How was this patch tested? unittest added ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46571 from HyukjinKwon/SPARK-48268. Lead-authored-by: Hyukjin Kwon <gurwls223@apache.org> Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
d0d8432
to
ec0d893
Compare
optional bool local = 2; | ||
|
||
// (Optional) Whether to checkpoint this dataframe immediately. | ||
optional bool eager = 3; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The default is eager = true
right? Should the protocol encode this better? Currently the protocol defaults to eager = false
if the field is not set, so my question is should we flip the logic (i.e. replace this with lazy
) so the default behavior does not require you to set additional fields.
The same question for local
...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can but I actually followed other cases though (see optional bool
at relations.proto
).
connector/connect/common/src/main/protobuf/spark/connect/commands.proto
Outdated
Show resolved
Hide resolved
responseObserver: StreamObserver[proto.ExecutePlanResponse]): Unit = { | ||
val target = Dataset | ||
.ofRows(session, transformRelation(checkpointCommand.getRelation)) | ||
val checkpointed = if (checkpointCommand.hasLocal && checkpointCommand.hasEager) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You could also increase the visibility of Dataset.checkpoint(eager: Boolean, reliableCheckpoint: Boolean)
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I just did it locally but I think it's actually better to keep just as is .. I think the current one is easier to read .. Dataset.checkpoint(eager: Boolean, reliableCheckpoint: Boolean)
is private as well.
@@ -138,6 +138,41 @@ def __init__( | |||
# by __repr__ and _repr_html_ while eager evaluation opens. | |||
self._support_repr_html = False | |||
self._cached_schema: Optional[StructType] = None | |||
self._cached_remote_relation_id: Optional[str] = None | |||
|
|||
def __del__(self) -> None: |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So no Python GC expert here. I am assuming some system thread is doing this work. Is it wise to execute an RPC from there?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
At least Py4J does the same thing (socket connection).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
and that (previous) maintainer knows quite well. Although now I am the maintainer for Py4J though :-).
Merged to master. I will followup the discussion if there are more to address since we're releasing preview soon. |
…Connect client ### What changes were proposed in this pull request? This PR adds `Dataset.checkpoint` and `Dataset.localCheckpoint` into Scala Spark Connect client. Python API was implemented at #46570 ### Why are the changes needed? For API parity. ### Does this PR introduce _any_ user-facing change? Yes, it adds `Dataset.checkpoint` and `Dataset.localCheckpoint` into Scala Spark Connect client. ### How was this patch tested? Unittests added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46683 from HyukjinKwon/SPARK-48370. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
… and eager required fields in CheckpointCommand ### What changes were proposed in this pull request? This PR is a followup of #46683 and #46570 that refactors `local` and `eager` required fields in `CheckpointCommand` ### Why are the changes needed? To make the code easier to maintain. ### Does this PR introduce _any_ user-facing change? No, the main change has not been released yet. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46712 from HyukjinKwon/SPARK-48370-SPARK-48258-followup. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
What changes were proposed in this pull request?
This PR proposes to
DataFrame.checkpoint
andDataFrame.localCheckpoint
API in Spark Connect.Overview
DataFrame.__del__
in Python upon garbage-collection)*In 99.999% cases, the state (map<(session_id, uuid), c'p'dataframe>) will be cleared when DataFrame is garbage-collected, e.g., unless there are some crashes. Practically, Py4J also leverages to clean up their Java objects. For 0.001% cases, the 6. and 7. address them. Both steps happen when session is closed, and session holder is released, see also apache/spark#41580.
Command/RPCs
Reuse
CachedRemoteRelation
(from apache/spark#41580)Usage
Why are the changes needed?
For feature parity without Spark Connect.
Does this PR introduce any user-facing change?
Yes, it adds both
DataFrame.checkpoint
andDataFrame.localCheckpoint
API in Spark Connect.How was this patch tested?
Unittests, and manually tested as below:
Code
./bin/pyspark --remote "local[*]"
Logs
Was this patch authored or co-authored using generative AI tooling?
No.