-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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-51537][CONNECT][CORE] construct the session-specific classloader based on the default session classloader on executor #50334
base: master
Are you sure you want to change the base?
Conversation
… the session specific jars
7b963dc
to
bbe2a94
Compare
Hi @hvanhovell @zhenlineo @HyukjinKwon @vicennial, Could you help review this PR? Thx very much. |
Thanks for identifying this issue, @wbo4958! While your PR resolves the executor-side problem, I believe we have a chance to refine our approach to cover both executor operations (e.g., typical UDFs) and driver operations (e.g., custom data sources) in one unified solution. The high-level proposal: In the ArtifactManager, add an initialisation step that would copy JARs from the underlying
The negative here is duplicating the global JARs for each new Spark Connect session will naturally consume more resources. We could mitigate this by adding a Spark configuration option to toggle whether global jars are inherited into a Spark Connect session. WDYT? |
Hi @vicennial , Looks like we couldn't copy the global jars into session-specific jars. Consider this scenario, A spark plugin is added by "--jars" when launching connect server, then the executor will immediately fetch the global jars and initialize the spark plugin. If we copy the global jars into session-specific jars, when the tasks are running on the executor side, when the code hits the plugin, the session-specific classloader is going to reload the plugin which could result in plugin not working. The fist commit of my PR bbe2a94 is adding the global jars into session-specific classpath, but it's going to cause the plugin not working issue, see NVIDIA/spark-rapids#11902 |
What changes were proposed in this pull request?
This PR is to construct the session-specific classloader based on the default session classloader which has already added the global jars (e.g., added by
--jars
) into the classpath on the executor side in the connect mode.Why are the changes needed?
In Spark Connect mode, when connecting to a non-local (e.g., standalone) cluster, the executor creates an isolated session state that includes a session-specific classloader for each task. However, a notable issue arises: this session-specific classloader does not include the global JARs specified by the --jars option in the classpath. This oversight can lead to deserialization exceptions. For example:
Does this PR introduce any user-facing change?
No
How was this patch tested?
The newly added test can pass. And the below manual test can pass,
Running the below code under the pyspark client environment.
Without this PR, you're going to see the below exception
Was this patch authored or co-authored using generative AI tooling?
No