[SPARK-53455][CONNECT] Add `CloneSession` RPC

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

Adds a new experimental/developer RPC `CloneSession` to the `SparkConnectService`.

  ✅ CLONED (from SparkSession)

  - SessionState - SQL configs, temp views, UDFs, catalog metadata
  - ArtifactManager - JARs, files, classes added to session
  - ManagedJobTags - Job group tags for tracking
  - SharedState (reference) - Metastore, global temp views
  - SparkContext (reference) - Core Spark engine

  ❌ NOT CLONED (SessionHolder - Spark Connect layer)

  - planCache - (Partially analyzed) Logical plans for query optimization
  - operationIds - Currently executing operations
  - errorIdToError - Recent errors for debugging
  - eventManager - Session lifecycle events
  - dataFrameCache - DataFrames for foreachBatch callbacks
  - mlCache - ML models and pipelines
  - listenerCache - Streaming query listeners
  - pipelineExecutions - Active pipeline contexts
  - dataflowGraphRegistry - Registered dataflow graphs
  - streamingForeachBatchRunnerCleanerCache - Python streaming workers
  - pythonAccumulator - Python metrics collection
  - Session timings - Start time, last access, custom timeout

The clone preserves all SQL/catalog state but creates a fresh runtime environment. An analogy is cloning a database schema/config but not the active connections, caches, or running jobs.
### Why are the changes needed?

Spark Connect introduced the concept of resource isolation (via `ArtifactManager`, which has been ported to classic Spark) and thus, jars/pyfiles/artifacts added to each session are isolated from other sessions.

A slight rough edge is that if a user wishes to fork the state of a session but maintain independence, the only possible way is to create a new session and reupload/reinit all base jars/artifacts/pyfiles, etc.

Support for cloning through the API helps address the rough edge while maintaining all the benefits of session resource isolation.
### Does this PR introduce _any_ user-facing change?

Yes

```python
  spark = SparkSession.builder.remote("sc://localhost").getOrCreate()
  spark.conf.set("my.custom.config", "value")
  spark.addArtifact("/path/to/my.jar")
  spark.sql("CREATE TEMP VIEW my_view AS SELECT 1 AS id")

  # Clone the session
  cloned_spark = spark.cloneSession()

  # The cloned session has all the same state
  assert cloned_spark.conf.get("my.custom.config") == "value"
  assert cloned_spark.sql("SELECT * FROM my_view").collect() == [Row(id=1)]

  # But operations are isolated between sessions
  cloned_spark.sql("DROP VIEW my_view")  # Only affects cloned session
  spark.sql("SELECT * FROM my_view").collect()  # Original still works
```

### How was this patch tested?

New individual unit tests along with new test suites.

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

Co-authored with assistance from Claude Code.

Closes #52200 from vicennial/cloneAPI.

Authored-by: vicennial <venkata.gudesa@databricks.com>
Signed-off-by: Herman van Hovell <herman@databricks.com>
18 files changed
tree: 89bccb1440c2004b3b8c47b4ceb18ec3dafa2ee0
  1. .github/
  2. .mvn/
  3. assembly/
  4. bin/
  5. binder/
  6. build/
  7. common/
  8. conf/
  9. connector/
  10. core/
  11. data/
  12. dev/
  13. docs/
  14. examples/
  15. graphx/
  16. hadoop-cloud/
  17. launcher/
  18. licenses/
  19. licenses-binary/
  20. mllib/
  21. mllib-local/
  22. project/
  23. python/
  24. R/
  25. repl/
  26. resource-managers/
  27. sbin/
  28. sql/
  29. streaming/
  30. tools/
  31. ui-test/
  32. .asf.yaml
  33. .gitattributes
  34. .gitignore
  35. .nojekyll
  36. CONTRIBUTING.md
  37. LICENSE
  38. LICENSE-binary
  39. NOTICE
  40. NOTICE-binary
  41. pom.xml
  42. README.md
  43. scalastyle-config.xml
README.md

Apache Spark

Spark is a unified analytics engine for large-scale data processing. It provides high-level APIs in Scala, Java, Python, and R (Deprecated), and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and DataFrames, pandas API on Spark for pandas workloads, MLlib for machine learning, GraphX for graph processing, and Structured Streaming for stream processing.

GitHub Actions Build PySpark Coverage PyPI Downloads

Online Documentation

You can find the latest Spark documentation, including a programming guide, on the project web page. This README file only contains basic setup instructions.

Build Pipeline Status

BranchStatus
masterGitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
branch-4.0GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
GitHub Actions Build
branch-3.5GitHub Actions Build
GitHub Actions Build

Building Spark

Spark is built using Apache Maven. To build Spark and its example programs, run:

./build/mvn -DskipTests clean package

(You do not need to do this if you downloaded a pre-built package.)

More detailed documentation is available from the project site, at “Building Spark”.

For general development tips, including info on developing Spark using an IDE, see “Useful Developer Tools”.

Interactive Scala Shell

The easiest way to start using Spark is through the Scala shell:

./bin/spark-shell

Try the following command, which should return 1,000,000,000:

scala> spark.range(1000 * 1000 * 1000).count()

Interactive Python Shell

Alternatively, if you prefer Python, you can use the Python shell:

./bin/pyspark

And run the following command, which should also return 1,000,000,000:

>>> spark.range(1000 * 1000 * 1000).count()

Example Programs

Spark also comes with several sample programs in the examples directory. To run one of them, use ./bin/run-example <class> [params]. For example:

./bin/run-example SparkPi

will run the Pi example locally.

You can set the MASTER environment variable when running examples to submit examples to a cluster. This can be spark:// URL, “yarn” to run on YARN, and “local” to run locally with one thread, or “local[N]” to run locally with N threads. You can also use an abbreviated class name if the class is in the examples package. For instance:

MASTER=spark://host:7077 ./bin/run-example SparkPi

Many of the example programs print usage help if no params are given.

Running Tests

Testing first requires building Spark. Once Spark is built, tests can be run using:

./dev/run-tests

Please see the guidance on how to run tests for a module, or individual tests.

There is also a Kubernetes integration test, see resource-managers/kubernetes/integration-tests/README.md

A Note About Hadoop Versions

Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs.

Please refer to the build documentation at “Specifying the Hadoop Version and Enabling YARN” for detailed guidance on building for a particular distribution of Hadoop, including building for particular Hive and Hive Thriftserver distributions.

Configuration

Please refer to the Configuration Guide in the online documentation for an overview on how to configure Spark.

Contributing

Please review the Contribution to Spark guide for information on how to get started contributing to the project.