Skip to content

feat: Add experimental support for accelerated PyArrow UDFs#4234

Open
andygrove wants to merge 16 commits intoapache:mainfrom
andygrove:pyarrow-udf
Open

feat: Add experimental support for accelerated PyArrow UDFs#4234
andygrove wants to merge 16 commits intoapache:mainfrom
andygrove:pyarrow-udf

Conversation

@andygrove
Copy link
Copy Markdown
Member

@andygrove andygrove commented May 6, 2026

Which issue does this PR close?

Closes #957

Rationale for this change

Spark's mapInArrow / mapInPandas plans insert ColumnarToRow between a Comet scan and the Python operator, and the Python runner re-encodes the rows back into Arrow IPC. Data that starts and ends in Arrow form does an Arrow to Row to Arrow round-trip in between.

What changes are included in this PR?

  • CometPythonMapInArrowExec: takes columnar input from a Comet child, skips the UnsafeProjection that ColumnarToRow performs, and keeps the Python runner output as ColumnarBatch. isBarrier is propagated through RDD.barrier() so mapInArrow(..., barrier=True) keeps its gang-scheduling semantics.
  • EliminateRedundantTransitions rewrites ColumnarToRow + (PythonMapInArrowExec | MapInPandasExec | Spark 4.0+ MapInArrowExec) into the new operator when the upstream is columnar.
  • Per-Spark-version shim for the differing ArrowPythonRunner constructors and the 4.0+ MapInArrowExec rename.
  • New conf spark.comet.exec.pythonMapInArrow.enabled, default false while experimental.
  • User guide page documenting usage and limitations.

Limitations

  • The row-to-Arrow re-encoding inside ArrowPythonRunner is not yet eliminated. Comet still feeds ColumnarBatch.rowIterator() to the existing runner; removing it needs a custom writer thread, tracked in Eliminate remaining row<->Arrow round-trip in CometPythonMapInArrowExec #4240.
  • Spark 3.4 is a no-op (shim returns None).
  • The optimization fires across a shuffle only with Comet's columnar shuffle. A vanilla Spark Exchange outputs rows and breaks the precondition.

How are these changes tested?

pytest module test_pyarrow_udf.py covers mapInArrow and mapInPandas in both vanilla and optimized modes: nulls, empty input, Python exception propagation, decimal/date/timestamp/array/struct types, post-shuffle correctness, and barrier=True gang scheduling. Scala suite CometPythonMapInArrowSuite covers plan-level checks. Dedicated pyarrow_udf_test.yml workflow runs the pytest module on every PR.

Wall-clock benchmark (benchmark_pyarrow_udf.py, 1M rows, 5 iters, local[2], Spark 3.5):

api workload vanilla optimized speedup
mapInArrow narrow primitives 0.177 s 0.134 s 1.32x
mapInArrow mixed with strings 0.339 s 0.257 s 1.32x
mapInArrow wide rows (50 cols) 1.883 s 1.728 s 1.09x
mapInPandas narrow primitives 0.167 s 0.129 s 1.30x
mapInPandas mixed with strings 0.344 s 0.263 s 1.31x
mapInPandas wide rows (50 cols) 1.889 s 1.755 s 1.08x

Wide-rows speedup is smaller because Python-side IPC dominates when row width grows.

andygrove added 5 commits May 5, 2026 18:06
When Comet operators produce Arrow columnar data and the next operator
is a Python UDF (mapInArrow/mapInPandas), Spark currently inserts an
unnecessary ColumnarToRow transition. The Python runner then converts
those rows back to Arrow to send to Python, creating a wasteful
Arrow->Row->Arrow round-trip.

This adds CometPythonMapInArrowExec which:
- Accepts columnar input directly from Comet operators
- Uses lightweight batch.rowIterator() instead of UnsafeProjection
- Keeps the Python output as ColumnarBatch (no output row conversion)

The optimization is detected in EliminateRedundantTransitions and
controlled by spark.comet.exec.pythonMapInArrow.enabled (default: true).
Documents the CometPythonMapInArrowExec optimization, including
supported APIs, configuration, usage example, and how to verify
the optimization is active in query plans.
…ions

Fix three issues that prevented test_pyarrow_udf.py from running:

1. mapInArrow callbacks must accept Iterator[pa.RecordBatch] and yield
   batches. The previous single-batch signatures crashed with
   "'map' object has no attribute 'to_pandas'".
2. PySpark DataFrame has no `queryExecution` attribute. Use
   `_jdf.queryExecution().executedPlan().toString()` instead.
3. Replace soft plan-string heuristics with assertions that fail loudly
   if the optimization regresses. Match on `CometPythonMapInArrow` (no
   `Exec` suffix in the plan toString) and assert no `ColumnarToRow`
   transition is present.
- Rewrite test_pyarrow_udf.py as a pytest module. A session-scoped
  SparkSession fixture builds the Comet-enabled session once and a
  parametrized `accelerated` fixture toggles
  spark.comet.exec.pythonMapInArrow.enabled per test, so each case runs
  under both the optimized and fallback paths and asserts the expected
  plan operator (`CometPythonMapInArrow` vs vanilla `PythonMapInArrow`).
  The jar is auto-discovered from spark/target by matching the installed
  pyspark version, or taken from the COMET_JAR env var.
- Add a dedicated `PyArrow UDF Tests` workflow that builds Comet against
  Spark 3.5 / Scala 2.12, installs pyspark/pyarrow/pandas/pytest, and
  runs the new pytest module.
- Add CometPythonMapInArrowSuite to the `exec` suite list in both
  pr_build_linux.yml and pr_build_macos.yml so the JVM-side suite is
  exercised on every PR.
Comment thread .github/workflows/pyarrow_udf_test.yml Fixed
andygrove added 6 commits May 5, 2026 18:46
Replace the narrow paths allowlist with the same paths-ignore list used
by pr_build_linux.yml so the workflow runs on any source change that
could affect Comet's PyArrow UDF execution path, not just the few files
explicitly named.
The PR's `CometPythonMapInArrowExec` and `EliminateRedundantTransitions`
rule directly reference Spark 3.5 APIs that differ across supported
Spark versions: the `ArrowPythonRunner` constructor (4 distinct
signatures across 3.4/3.5/4.0/4.1+/4.2), `arrowUseLargeVarTypes`,
`JobArtifactSet`, `MapInBatchExec.isBarrier`, and the `PythonMapInArrowExec`
type itself (renamed to `MapInArrowExec` in 4.0+). This breaks compile
on every profile other than 3.5.

Introduce a per-version `ShimCometPythonMapInArrow` trait under
`org.apache.spark.sql.comet.shims` (placed in the spark namespace so
it can reach `private[spark]` members) that:

* matches the Spark-version-specific MapInArrow / MapInPandas exec types
  and exposes their `(func, output, child, isBarrier, evalType)` tuple,
* constructs the right `ArrowPythonRunner` for the version,
* hides `arrowUseLargeVarTypes` / `JobArtifactSet` / `getPythonRunnerConfMap`
  behind helper methods.

Spark 3.4 lacks the prerequisite APIs (no `isBarrier`, no `JobArtifactSet`,
no `arrowUseLargeVarTypes`), so its shim returns `None` from the matchers
and the optimization is a no-op there.
The default `amd64/rust` image is Debian 13 (trixie), where the system
`python3` is 3.13 and there is no `python3.11` apt package. The workflow
installed `python3.11` explicitly, which fails on trixie with `Unable to
locate package python3.11`.

Switching to `rust:bookworm` gives a Debian 12 base where `python3` is
3.11, matching the job name and pyspark 3.5.x's supported runtime.
Spark launches Python workers in fresh subprocesses that look up python3
on PATH. Without PYSPARK_PYTHON, workers use the system python (no pyarrow
installed) and UDF execution fails with ModuleNotFoundError. Point both
PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON at /tmp/venv/bin/python so workers
inherit the same interpreter that pytest uses.
@andygrove andygrove changed the title feat: Add support for accelerated PyArrow UDFs [experimental] feat: Add experimental support for accelerated PyArrow UDFs May 6, 2026
andygrove added 4 commits May 6, 2026 07:44
Flip spark.comet.exec.pythonMapInArrow.enabled default from true to false
and prefix the config doc with "Experimental:" so the default matches the
"[experimental]" label on the feature. Update the user guide to instruct
users to opt in explicitly.
Add coverage for cases that the original pytest module did not exercise:

- mapInPandas (claimed supported, previously zero coverage)
- Null preservation across long and string columns via Arrow passthrough
- Empty input from a CometScan via filter pushdown
- Python exception propagation (sentinel must surface in driver-side error)
- DecimalType(18,6), DateType, TimestampType round-trip with nulls
- ArrayType<Int> and nested StructType, including null arrays/structs and
  arrays containing null elements
- repartition between scan and UDF (correctness only; the optimization
  itself does not fire across a vanilla Exchange and is documented as
  such in the test)

Generalize _assert_plan_matches_mode to take the vanilla node name so the
fallback assertion can match either PythonMapInArrow or MapInPandas.
Expand the user guide with the limitations a user should know before
enabling the experimental optimization:

- The remaining row-to-Arrow round-trip inside the Python runner is
  documented more precisely (the input goes through ColumnarBatch.rowIterator
  to feed ArrowPythonRunner, which re-encodes to Arrow IPC).
- A vanilla Spark Exchange between the Comet scan and the UDF prevents
  the optimization from firing. Users must configure Comet's native
  shuffle manager at session startup to keep the data columnar.
- Spark 3.4 lacks the prerequisite APIs and the feature is a no-op there.
- isBarrier is captured by the operator constructor but not yet
  propagated to the Python runner.

Also explain the AQE display quirk: with AQE on and a shuffle present,
the pre-execution plan shows the unoptimized form because the rule
only sees the materialized subplan after stage execution. Running an
action and re-inspecting explain() reveals the optimized plan.
Standalone Python script that times df.mapInArrow(passthrough).count()
and the equivalent mapInPandas query with the optimization toggled on
and off. Numbers are wall-clock seconds, so they include the Python
worker, Arrow IPC, and downstream count() costs. That is the right
unit for a feature whose user surface is Python: it shows what
fraction of end-to-end time the optimization shaves off, not just the
JVM-side delta in isolation.

Three workloads exercise the dimension where the optimization helps
most:

- narrow primitives (long, int, double)
- mixed with strings (variable-length encoding)
- wide rows (50 columns, projection cost scales with column count)

Local smoke run with 200k rows shows 1.17x to 1.45x speedup across
mapInArrow and mapInPandas, narrow/wide schemas. The script is
configurable via BENCHMARK_ROWS / BENCHMARK_WARMUP / BENCHMARK_ITERS
env vars for users who want longer or shorter runs.
The operator captured isBarrier in its constructor but always called
inputRDD.mapPartitionsInternal, dropping the barrier execution mode
semantics that mapInArrow(..., barrier=True) requests. Stages running
under the optimization lost gang scheduling and the BarrierTaskContext
APIs the UDF expects.

Branch on isBarrier and route through inputRDD.barrier().mapPartitions
in the barrier case, matching what Spark's MapInBatchExec.doExecute
does. Add a pytest case that calls BarrierTaskContext.get() inside the
UDF, which raises if the task is not running in a barrier stage; runs
in both vanilla and optimized modes. Drop the isBarrier limitation
note from the user guide.
@comphead
Copy link
Copy Markdown
Contributor

comphead commented May 6, 2026

I assume not only speed is improved, would also be interesting to check memory metrics

@andygrove andygrove moved this from Todo to In progress in Comet Development May 6, 2026
Copy link
Copy Markdown
Contributor

@parthchandra parthchandra left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Some minor comments. Otherwise great PR!

* Optimized flow: CometNativeExec (Arrow) -> CometPythonMapInArrowExec (batch.rowIterator() ->
* Arrow -> Python -> Arrow columnar output)
*
* This eliminates:
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 !

child: SparkPlan,
isBarrier: Boolean,
pythonEvalType: Int)
extends UnaryExecNode
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Other Comet operators extend CometPlan. The idea was that all Comet specific common behavior (say some Comet specific metrics, for example) can be in a single place.


jobs:
pyarrow-udf:
name: PyArrow UDF (Spark 3.5, JDK 17, Python 3.11)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this be Spark 4 now?
I'm assuming that this is enabled for only one version of Spark because it is experimental?

.booleanConf
.createWithDefault(false)

val COMET_PYTHON_MAP_IN_ARROW_ENABLED: ConfigEntry[Boolean] =
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

MAP is confusing IMO, so many things related to map in Spark

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe COMET_PYARROW_SUPPORT or something like that?

// a ColumnarToRow child with CometPythonMapInArrowExec to avoid the unnecessary
// Arrow->Row->Arrow round-trip. The matchers are version-shimmed: Spark 3.4 returns None
// (it lacks the required APIs) and Spark 4.1+ matches the renamed `MapInArrowExec`.
case p: SparkPlan
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thinking if it actually makes sense to have python/udf support in separate module? so user can build comet with python support or not

* accepts columnar input directly from Comet operators, avoiding unnecessary Arrow -> Row ->
* Arrow conversions.
*
* Normal Spark flow: CometNativeExec (Arrow) -> ColumnarToRow -> PythonMapInArrowExec
Copy link
Copy Markdown
Contributor

@comphead comphead May 7, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this would be nice to have also in pyarrow-udfs.md

Copy link
Copy Markdown
Contributor

@comphead comphead left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @andygrove for pandas wondering if its the same as https://spark.apache.org/docs/latest/api/python/tutorial/sql/arrow_pandas.html#enabling-for-conversion-to-from-pandas and if its not, probably it would be good to highlight

@mbutrovich
Copy link
Copy Markdown
Contributor

I am reviewing this morning, please do not merge.

Copy link
Copy Markdown
Contributor

@mbutrovich mbutrovich left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @andygrove, this is a great direction. A few things that came up while reading, mostly framing and cleanup questions.

Framing

Checked Spark v3.4.3 / v3.5.8 / v4.0.1 / v4.1.0 / v4.2.0-preview4 to see what's actually getting skipped. Looks like what this PR delivers is:

  • one per-row UnsafeProjection on the input side (vanilla goes through ColumnarToRowExec.doExecute which applies toUnsafe per row)
  • the output-side projection back to rows that MapInBatchExec does
  • ColumnarBatch stays live so downstream Comet ops consume it directly

The ArrowPythonRunner IPC re-encoding is unchanged though, and on wide rows that's most of the time, which lines up with the 1.3x narrow vs 1.09x wide numbers. Is "drop two UnsafeProjection copies and keep the stage columnar" closer to what lands here than "eliminate the Arrow to Row to Arrow round-trip"? Full round-trip elimination is #4240, right?

Bigger items

1. Conf and class name (picking up @comphead's point)

spark.comet.exec.pythonMapInArrow.enabled and CometPythonMapInArrowExec already handle mapInPandas, and the doc lists ArrowEvalPythonExec / FlatMapGroupsInPandasExec as future work under the same toggle. Is it worth renaming to spark.comet.exec.pyarrowUdf.enabled + CometPyArrowUDFExec now? Renames are painful after merge. common/src/main/scala/org/apache/comet/CometConf.scala:317-325, spark/src/main/scala/org/apache/spark/sql/comet/CometPythonMapInArrowExec.scala:49.

2. Shim layout across 4.x

Checked the ArrowPythonRunner ctors at each tag:

version funcs trailing
3.5.8 Seq[ChainedPythonFunctions] jobArtifactUUID
4.0.1 Seq[(ChainedPythonFunctions, Long)] jobArtifactUUID, profiler
4.1.0 same jobArtifactUUID, sessionUUID, profiler
4.2.0-preview4 same jobArtifactUUID, sessionUUID

The 4.0 and 4.2 shim files are byte-identical today, but the second trailing None means profiler on 4.0 and sessionUUID on 4.2, so they'd diverge the moment either one becomes non-None. One alternative: put the identical methods (matchMapInArrow, matchMapInPandas, currentJobArtifactUUID, largeVarTypes, getPythonRunnerConfMap, all the same across 4.0/4.1/4.2) into one spark-4.x/.../ShimCometPythonMapInArrow.scala, and leave only the ArrowPythonRunner construction as a tiny per-minor factory. Roughly 60 shared lines once, 15 per minor instead of 85. 3.5 still splits since the funcs type actually differs.

3. Scala suite doesn't hit the new operator

spark/src/test/scala/org/apache/comet/exec/CometPythonMapInArrowSuite.scala:31-66 has two tests but neither runs a Python UDF. First one asserts any columnar op exists in a plain SELECT *. Second asserts CometPythonMapInArrowExec isn't in a plan that has no UDF to optimize in the first place. Both pass even if you delete the PR. Worth adding one test that plans a PythonMapInArrowExec over a CometScan and checks the rewrite fires, plus the disabled-conf negative? The test_pyarrow_udf.py coverage is solid, this is just for the Scala gate.

4. CometPlan mixin

Backing up @parthchandra's comment. CometPythonMapInArrowExec.scala:49-53 extends UnaryExecNode with PythonSQLMetrics with ShimCometPythonMapInArrow, but every other op under spark/src/main/scala/org/apache/spark/sql/comet/ mixes in CometPlan.

Smaller things

5. Tuple return type

Shims return Option[(Expression, Seq[Attribute], SparkPlan, Boolean, Int)] and the call site at EliminateRedundantTransitions.scala:103-106 destructures by position. A small MapInBatchInfo case class would survive Spark adding a sixth field better?

6. Matcher runs twice

EliminateRedundantTransitions.scala:101-106 evaluates matchMapInArrow(p).orElse(matchMapInPandas(p)) in the guard then again via .get. Planner only runs per query so perf is fine, but a single match on the result reads cleaner.

7. extractColumnarChild guard

EliminateRedundantTransitions.scala:155-160 has case ColumnarToRowExec(child) if child.supportsColumnar. By the time this rule runs, the Comet rules should have replaced raw ColumnarToRowExec over a Comet columnar child with one of the Comet variants. Is this branch reachable in practice, or defensive?

8. doExecute fallback

CometPythonMapInArrowExec.scala:74-76 wraps ColumnarToRowExec(this).doExecute(). Only fires when a row-consuming parent calls execute() (top-level collect etc.). A short why-comment might help since it reintroduces the row transition the PR is eliminating.

9. computeArrowPython has 12 params

Across every 3.5+ shim. Would passing SQLConf + pythonMetrics and deriving timeZoneId / largeVarTypes / pythonRunnerConf / jobArtifactUUID inside cut the list?

10. local* naming

CometPythonMapInArrowExec.scala:82-93 prefixes localOutput, localChildSchema, localPythonEvalType, localPythonMetrics for closure capture, but pythonUDF = func.asInstanceOf[PythonUDF] isn't prefixed. Pick one?

11. Comments that restate code

CometPythonMapInArrowExec.scala:103 (// Run on every partition.) and :133 (// Python returns a StructType column; flatten to individual columns) feel droppable. The :126 struct-wrap note and the :143 barrier note are good keeps.

Test infra

12. Jar resolution in three places

test_pyarrow_udf.py and benchmark_pyarrow_udf.py both have _resolve_comet_jar, and pyarrow_udf_test.yml:141-142 has a shell ls | grep ... version. Could a conftest.py helper + COMET_JAR= from the workflow collapse all three?

13. Workflow paths and build flavor

pyarrow_udf_test.yml uses paths-ignore, so any Rust or Scala change re-runs the 15-minute build. Pinning an allowlist to the feature files would be tighter. Also, any reason for -Prelease + cargo build --release when there are no perf assertions? Debug is 2-3x faster.

14. CI Spark version coverage

Workflow is pinned to -Pspark-3.5 -Pscala-2.12. The 4.0/4.1/4.2 shims get zero coverage, and the 5-tuple matcher contract is easy to break silently across versions. Even a compile-only 4.x matrix would catch that.

15. Benchmark caveat

benchmark_pyarrow_udf.py is local[2] + passthrough UDF + count() on 1M rows, which is mostly Python fork/IPC. Worth a sentence in the PR saying non-passthrough UDFs would shrink the delta further?

Things worth looking at and they seem fine

  • batches.flatMap { ... rowIterator.asScala } matches upstream MapInBatchExec, so JavaConverters overhead is not a regression.
  • Per-partition Array(Array(0)) / StructType(Array(...)) / BatchIterator allocations match Spark's own pattern.
  • CodeQL permissions block is already in place.

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

Labels

None yet

Projects

Status: In progress

Development

Successfully merging this pull request may close these issues.

Is it possible to support PyArrow backed UDFs in Comet natively?

5 participants