Skip to content

Comments

[RayDP 2.0] Support Spark 4.1, Java 17, and Scala 2.13#460

Draft
rexminnis wants to merge 16 commits intoray-project:masterfrom
rexminnis:feature/raydp-2.0-spark-4.1.1
Draft

[RayDP 2.0] Support Spark 4.1, Java 17, and Scala 2.13#460
rexminnis wants to merge 16 commits intoray-project:masterfrom
rexminnis:feature/raydp-2.0-spark-4.1.1

Conversation

@rexminnis
Copy link
Contributor

@rexminnis rexminnis commented Feb 7, 2026

Closes #459

Overview

This PR introduces RayDP 2.0, a major architectural upgrade migrating the core runtime to Apache Spark 4.1, Java 17, and Scala 2.13. This modernization addresses the removal of Scala 2.12/Java 8 support in Spark 4.x and aligns RayDP with modern infrastructure standards.

Key Changes

1. Core Runtime & Dependencies

  • Spark: Upgraded to 4.1.1.
  • Java: Enforced Java 17 (Strict requirement).
  • Scala: Upgraded to 2.13.17.
  • Ray: Aligned with 2.53.0.

2. Build System Overhaul

  • Maven: Updated core/pom.xml to target Java 17/Scala 2.13.
  • Python (PEP 517): Migrated to pyproject.toml for build configuration.
  • Editable Installs: Rewrote setup.py to correctly package JARs into raydp/jars during pip install -e ., fixing local development workflows.

3. Shim Layer Architecture

  • New Shim: Added raydp-shims-spark411 to handle Spark 4.1 API drifts.
  • Internal Helpers: Introduced Spark411Helper and Spark411SQLHelper to bridge package-private Spark APIs (e.g., TaskContextImpl, ArrowConverters) that are no longer accessible directly.
  • Legacy Shims: Disabled Spark 3.x shims in the build reactor.

4. Code Migration

  • Migrated scala.collection.JavaConverters -> scala.jdk.CollectionConverters.
  • Refactored RayCoarseGrainedSchedulerBackend to handle removed APIs (e.g., Utils.sparkJavaOpts).
  • Implemented toDataFrame in the new shim layer to support ray.data.from_spark and ds.to_spark.

5. ML Estimator Migration to Ray Train V2 API

Migrated all ML estimators (Torch, TensorFlow, XGBoost) to the Ray Train V2 API, which is enabled by default in Ray 2.53.0. The deprecated ray.air module is no longer used.

  • Torch Estimator: Replaced ray.air.session and ray.air.config imports with ray.train equivalents. Removed deprecated TorchCheckpoint in favor of direct state_dict loading.
  • TensorFlow Estimator: Replaced ray.air.session imports, removed deprecated TensorflowCheckpoint, and dropped MultiWorkerMirroredStrategy which is incompatible with Keras 3 (ray-project/ray#47464). Each Ray worker now trains independently on its data shard. Updated model save/load to use the .keras format (Keras 3 requirement).
  • XGBoost Estimator: Rewrote from the deprecated declarative API to the functional train_loop_per_worker API with RayTrainReportCallback. Removed deprecated XGBoostCheckpoint in favor of direct xgboost.Booster loading.

Verification

A comprehensive sanity check suite (examples/test_raydp_sanity.py) was added and verified locally on macOS (M1/M2).

Verified Capabilities:

  • Spark Session Initialization on Ray.
  • Executor Registration (RPC).
  • Basic Job Execution (count, range).
  • Data Transfer: Spark DataFrame -> Ray Dataset.
  • Data Transfer: Ray Dataset -> Spark DataFrame.
  • ML Estimators: Torch, TensorFlow, and XGBoost estimators pass CI with Ray Train V2.

Breaking Changes

  • Java 8/11 Support Dropped: Users must upgrade to Java 17.
  • Spark 3.x Support Dropped: This release targets Spark 4.1+ exclusively.
  • Ray Train V1 API Removed: ML estimators now require Ray 2.53.0+ (Train V2). The deprecated ray.air imports are no longer used.

   - Upgrade core POMs to Spark 4.1.1, Java 17, Scala 2.13.12
   - Add spark411 shim module (SparkShims411, Spark411Helper, Spark411SQLHelper)
     with proper TaskContextImpl, ArrowConverters, and ClassicSparkSession APIs
   - Implement toDataFrame for Arrow batch deserialization via shim layer
   - Add RayCoarseGrainedExecutorBackend for Spark 4.1 executor backend
   - Disable legacy shim modules (spark322, spark330, spark340, spark350)
   - Add pyproject.toml (PEP 517) and simplify setup.py
   - Update CI workflow for Spark 4.1.1
@rexminnis rexminnis force-pushed the feature/raydp-2.0-spark-4.1.1 branch 3 times, most recently from a106075 to 8e9fe61 Compare February 8, 2026 04:13
- Add 13-test ScalaTest suite validating shim descriptor, SPI loading,
  Arrow schema mapping, and Arrow round-trip conversions (including
  nulls, timestamps, decimals, empty DataFrames, and multi-batch)
- Fix bug in Spark411SQLHelper.toArrowBatchRdd where df.schema was
  captured inside mapPartitions closure causing
  CANNOT_INVOKE_IN_TRANSFORMATIONS on Spark 4.1
- Bump Scala from 2.13.12 to 2.13.17 to match Spark 4.1.1
   Spark 4.1.1's BlockInfoManager requires tasks to be registered via
   registerTask() before they can acquire read locks. getRDDPartition was
   creating a dummy TaskContext without registering, causing an NPE in
   lockForReading when converting DataFrames to Ray Datasets.

   - Generate unique taskAttemptIds via AtomicLong (starting at 1000000L)
     to avoid collisions with real Spark tasks and concurrent calls
   - Register task with BlockManager before block access
   - Release all locks and unset TaskContext in finally block
   - Add regression test confirming the NPE without registration
   - Add lifecycle test validating register/read/cleanup flow
   - Default Arrow IPC zstd compression (spark.sql.execution.arrow.compression.codec)
   - Pre-size ByteArrayOutputStream in getRDDPartition using BlockResult.bytes()
   - Read arrowUseLargeVarTypes from SQLConf instead of hardcoding false
   - Bump Ray compile-time dependency from 2.34.0 to 2.47.1 (latest on Maven Central)
   RayDP 2.0 only targets Spark 4.1 These modules were already
   excluded from the build but cluttered the source tree.
   The old fromSparkRDD/RayDatasetRDD pipeline produced bytes blocks and
   was already @deprecated with zero callers. Remove it along with all
   supporting dead code: _convert_by_rdd (Python), RayDatasetRDD.scala,
   ObjectRefHolder, ObjectStoreWriter instance class, and the wasteful
   ray.get(blocks[0]) type-check in ray_dataset_to_spark_dataframe.
   df.sql_ctx is deprecated in PySpark 4.1 and emits warnings on every
   access. All usages followed the pattern df.sql_ctx.sparkSession.X
   which simplifies to df.sparkSession.X. Also remove the dead _wrapped
   guard (Spark 3.x internal) and stale TODO/comments in RayAppMaster
   for fractional resources that are already supported.
   Eliminates the Arrow→pandas→Arrow round-trip in _convert_by_udf by
   switching from mapInPandas to mapInArrow (stable since Spark 3.4).
   Data stays in Arrow format throughout: pa.concat_tables + to_batches
   yields RecordBatches directly with no serialization overhead.
…removal

   - Default Arrow compression codec from zstd to lz4 (faster for intra-cluster)
   - Set maxRecordsPerBatch=0 (unlimited) since RayDP converts whole partitions
   - Skip pa.concat_tables when partition has a single block (avoids copy)
   - Use _ray_global_worker to avoid deprecation warning in get_locations
   - Remove dead RecordPiece/RayObjectPiece classes and unused pandas import
   double-copy in Arrow transfers

   Ray→Spark: batch actor lookups and data fetches into two ray.get()
   calls instead of 2N sequential calls, letting Ray pipeline transfers.

   Spark→Ray: replace ByteArrayOutputStream + toByteArray (two full
   copies of partition data) with a single pre-allocated byte[] and
   direct System.arraycopy (one copy), halving memory and CPU overhead
   in getRDDPartition.
    Ray fetching in Spark→Ray conversion

   Instead of blocking on rdd.count() until ALL partitions are
   materialized before submitting any Ray fetch tasks, run materialization
   in a background thread and poll BlockManager for completed partitions.
   As each partition becomes available, its Ray fetch task is dispatched
   immediately — overlapping Spark computation with Ray data transfer.

   JVM: add StreamingRecoverableRDD handle with getReadyPartitions()
   that queries BlockManager.blockIdsToLocations for newly cached blocks.

   Python: from_spark_recoverable now polls the handle in a loop,
   dispatching fetch tasks incrementally as partitions complete.
@rexminnis rexminnis force-pushed the feature/raydp-2.0-spark-4.1.1 branch 6 times, most recently from 7f8049f to 1b12991 Compare February 8, 2026 04:45
   - Quote python-version strings in matrix to prevent YAML float
     parsing (3.10 → 3.1)
   - Remove stale dependency pins incompatible with Python 3.12
     (numpy<1.24, pydantic<2.0, click<8.3, tensorflow==2.13.1)
   - Skip TF and XGBoost tests until their estimators are updated
   - Use PEP 517 build (python -m build) instead of setup.py bdist_wheel
   - Update GitHub Actions to current versions (checkout v4,
     setup-python v5, setup-java v4, cache v4)
@rexminnis rexminnis force-pushed the feature/raydp-2.0-spark-4.1.1 branch 5 times, most recently from 08bdfe9 to 2e10492 Compare February 8, 2026 11:17
   2.53.0)

   Replace deprecated ray.air.session and ray.air.config imports with
   ray.train equivalents. Remove TorchCheckpoint, TensorflowCheckpoint,
   and XGBoostCheckpoint in favor of direct model loading. Rewrite
   XGBoost estimator from declarative to functional API with
   train_loop_per_worker and RayTrainReportCallback.
@rexminnis rexminnis force-pushed the feature/raydp-2.0-spark-4.1.1 branch from 2e10492 to 103d81f Compare February 8, 2026 11:37
@rexminnis
Copy link
Contributor Author

rexminnis commented Feb 12, 2026

@pang-wu @carsonwang - when you have a moment can I please get a review.

@pang-wu
Copy link
Collaborator

pang-wu commented Feb 16, 2026

Thanks for the PR, but I am not sure about the direction this PR is pointing to. It is better to split this into 3 different PRs:

  1. Spark 4.x migration, I'd like to tackle it here
  2. New build system migration
  3. Trainer API update -- we should discuss this first

@rexminnis
Copy link
Contributor Author

Thanks for the feedback @pang-wu! Happy to split this up — I agree the PR is large.

The driving motivation behind this work is that RayDP needs to modernize to stay relevant — Spark 4.x is Java 17-only, Scala 2.12 is EOL, and the ray.air API is deprecated. Staying current also unblocks future possibilities like Iceberg support and Structured Streaming integration.

I took a look at #450 and it seems like we've converged on a very similar approach for the core Spark 4.x shim (JSON4S → Jackson, CollectionConverters, toArrowBatchRDD on the shim interface, executor backend factory, TaskContextUtils, etc.). A few observations:

  1. Spark version: Support Spark 4.x #450 targets 4.0.0 while this PR targets 4.1.1, which is the current release. Happy to align on whichever version makes sense, but 4.1.1 might be the better target at this point.

  2. Legacy shims: Support Spark 4.x #450 keeps the Spark 3.x shims and bumps them all to Scala 2.13, but Spark 3.2–3.4 only shipped with Scala 2.12 — would those shims actually compile/work? My PR took the clean-break approach (drop 3.x), but I'm open to discussion on this.

  3. Java target: Support Spark 4.x #450 keeps maven.compiler.source=1.8. Spark 4.x requires Java 17 at runtime, so it may make sense to bump the compile target as well.

I can split the Spark 4.x core changes out of this PR into a focused PR so it's easier to review. For the other two pieces:

  • Build system (pyproject.toml migration) — straightforward to split into its own PR.
  • Trainer API (Ray Train V2) — agree we should discuss first. The ray.air module is fully deprecated in Ray 2.53 and will be removed, so this migration will be needed at some point. Happy to open an issue to discuss the approach before submitting a PR.

What works best for you?

@rexminnis rexminnis marked this pull request as draft February 17, 2026 14:23
@rexminnis
Copy link
Contributor Author

Converting this to draft per @pang-wu's feedback. Agreed this should be split into focused PRs:

  1. Spark 4.x core migration — will collaborate on Support Spark 4.x #450 instead
  2. Build system modernization (pyproject.toml, Python 3.12+) — separate PR after Support Spark 4.x #450 lands
  3. Ray Train V2 estimator migration — will open an RFC issue for discussion first

Keeping this as draft for reference — the working code here can inform the individual PRs.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[RFC] RayDP 2.0: Migration to Spark 4.1 & Java 17

2 participants