MonkeyCanCode opened a new pull request, #53078:
URL: https://github.com/apache/spark/pull/53078
<!--
Thanks for sending a pull request! Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://spark.apache.org/contributing.html
2. Ensure you have added or run the appropriate tests for your PR:
https://spark.apache.org/developer-tools.html
3. If the PR is unfinished, add '[WIP]' in your PR title, e.g.,
'[WIP][SPARK-XXXX] Your PR title ...'.
4. Be sure to keep the PR description updated to reflect all changes.
5. Please write your PR title to summarize what this PR proposes.
6. If possible, provide a concise example to reproduce the issue for a
faster review.
7. If you want to add a new configuration, please read the guideline first
for naming configurations in
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
8. If you want to add or modify an error type or message, please read the
guideline first in
'common/utils/src/main/resources/error/README.md'.
-->
### What changes were proposed in this pull request?
To fix this, I added the following changes:
1. [Apache Iceberg PR|https://github.com/apache/iceberg/pull/14590] to
implement closable catalog
2. Make `CatalogPlugin` interface extends `java.io.Closeable`
3. Implement a `close()` method in `CatalogManager` that iterates through
all registered catalogs and calls their `close()` method
4. Make `SessionState` implements `Closeable` and calls
`catalogManager.close()` from its `close()` method.
5. Invoke `session.sessionState.close()` from `SessionHolder.close()` when a
Spark Connect session is stopped
Above changes create a clean lifecycle for catalogs when a session ended, a
`close()` call is propagated down the chain which allow each catalog to release
its resources.
### Why are the changes needed?
Spark Connect server is leaking `SparkSession` objects each time a client
connects and disconnects when dealing with Apache Iceberg ([Apache Iceberg
PR|https://github.com/apache/iceberg/pull/14590]).
The `SessionHolder.close()` method in Spark Connect is responsible for
cleaning up a session. It does perform some cleanup such as artifacts and
streaming queries but it doesn't perform cleanup on the main `SessionState`.
This is where the `CatalogManager` lives which holds reference to `RESTCatalog`
and `S3FileIO`. Since the `SessionState` is never closed, these `Closeable`
catalogs are never closed and their threads leak.
### Does this PR introduce _any_ user-facing change?
N/A
### How was this patch tested?
I have a local setup which can easily reproduce this issue. Here is setup
details:
REST catalog: Apache Polaris (created the basic polaris entities via getting
start example)
Spark Connect server:
1. public released Spark distribution to show this issue is there and we
have leaks
2. local build with changes in this PR
Spark Connect client: install public released apache spark package via pip
Testing config:
1. To make the testing easy, I set
`spark.connect.session.manager.defaultSessionTimeout` from default `60m` to `1m`
Testing:
1. Check heap dump from spark UI for instance of
`org.apache.spark.sql.classic.SparkSession` and
`org.apache.spark.sql.internal.SessionState`
2. Make a connection to an Iceberg REST catalog and perform `close()` on
spark session implicitly:
```
import uuid
from pyspark.sql import SparkSession
USER_CLIENT_ID = "xxxxx"
USER_CLIENT_SECRET = "xxxxx"
# Use `create` with defined `session_id` to ensure we will be getting a new
session
session_id = str(uuid.uuid4())
spark = (
SparkSession.builder
.appName("Iceberg REST Catalog Quickstart")
.remote(f"sc://localhost/;session_id={session_id}")
.config("spark.sql.extensions",
"org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
.config("spark.sql.catalog.quickstart_catalog",
"org.apache.iceberg.spark.SparkCatalog")
.config("spark.sql.catalog.quickstart_catalog.catalog-impl",
"org.apache.iceberg.rest.RESTCatalog")
.config("spark.sql.catalog.quickstart_catalog.uri",
"http://localhost:8181/api/catalog")
.config("spark.sql.catalog.quickstart_catalog.warehouse",
"quickstart_catalog")
.config("spark.sql.catalog.quickstart_catalog.header.X-Iceberg-Access-Delegation",
"vended-credentials")
.config("spark.sql.catalog.quickstart_catalog.credential",
f"{USER_CLIENT_ID}:{USER_CLIENT_SECRET}")
.config("spark.sql.catalog.quickstart_catalog.scope",
"PRINCIPAL_ROLE:ALL")
.config("spark.sql.catalog.quickstart_catalog.token-refresh-enabled",
"true")
.config("spark.sql.catalog.quickstart_catalog.client.region",
"us-west-2")
.create()
)
# Setup basic namespace and perform a read/write:
spark.sql("USE quickstart_catalog").show()
spark.sql("CREATE NAMESPACE IF NOT EXISTS quickstart_namespace").show()
spark.sql("CREATE NAMESPACE IF NOT EXISTS
quickstart_namespace.schema").show()
spark.sql("USE NAMESPACE quickstart_namespace.schema").show()
spark.sql("CREATE TABLE IF NOT EXISTS quickstart_table (id BIGINT, data
STRING) USING ICEBERG").show()
spark.sql("INSERT INTO quickstart_table values (1, 'a')").show()
spark.sql("SELECT * FROM quickstart_table").show()
# Stop the connection
spark.stop()
```
3. Check heap dump from spark UI for instance of
`org.apache.spark.sql.classic.SparkSession` and
`org.apache.spark.sql.internal.SessionState` again and noticed resources are
not getting cleanup
4. To make it more obvious, spin up 300 sessions and let the cleaner context
stop them (with overwrite cleaning interval to 1m):
```
for _ in range (300):
session_id = str(uuid.uuid4())
spark = (
SparkSession.builder
.appName("Iceberg REST Catalog Quickstart")
.remote(f"sc://localhost/;session_id={session_id}")
.config("spark.sql.extensions",
"org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
.config("spark.sql.catalog.quickstart_catalog",
"org.apache.iceberg.spark.SparkCatalog")
.config("spark.sql.catalog.quickstart_catalog.catalog-impl",
"org.apache.iceberg.rest.RESTCatalog")
.config("spark.sql.catalog.quickstart_catalog.uri",
"http://localhost:8181/api/catalog")
.config("spark.sql.catalog.quickstart_catalog.warehouse",
"quickstart_catalog")
.config("spark.sql.catalog.quickstart_catalog.header.X-Iceberg-Access-Delegation",
"vended-credentials")
.config("spark.sql.catalog.quickstart_catalog.credential",
f"{USER_CLIENT_ID}:{USER_CLIENT_SECRET}")
.config("spark.sql.catalog.quickstart_catalog.scope",
"PRINCIPAL_ROLE:ALL")
.config("spark.sql.catalog.quickstart_catalog.token-refresh-enabled", "true")
.config("spark.sql.catalog.quickstart_catalog.client.region",
"us-west-2")
.create()
)
spark.sql("USE quickstart_catalog").show()
spark.sql("USE NAMESPACE quickstart_namespace.schema").show()
spark.sql("SELECT * FROM quickstart_table").show()
```
6. Wait for 2-3 mins (the cleaning should start happen after 1m but it may
take an extra 20-30 seconds for the cleaning to complete) then check heap dump
from spark UI for instance of `org.apache.spark.sql.classic.SparkSession` and
`org.apache.spark.sql.internal.SessionState` again. We will noticed the
instances of these classes (along with many others) are not getting cleanup
with current code. Also, heap usage will stay high and not able to garbage
collected.
7. Now to test the fixed, get a local build of iceberg spark runtime jar
from the PR above and a local build of apache spark from this PR, and repeat
the same tests listed above. This time, we will see resources getting cleanup
properly and heap usage decreased after cleanup.
### Was this patch authored or co-authored using generative AI tooling?
No
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]