|
| 1 | +.. Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +.. or more contributor license agreements. See the NOTICE file |
| 3 | +.. distributed with this work for additional information |
| 4 | +.. regarding copyright ownership. The ASF licenses this file |
| 5 | +.. to you under the Apache License, Version 2.0 (the |
| 6 | +.. "License"); you may not use this file except in compliance |
| 7 | +.. with the License. You may obtain a copy of the License at |
| 8 | +
|
| 9 | +.. http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +
|
| 11 | +.. Unless required by applicable law or agreed to in writing, |
| 12 | +.. software distributed under the License is distributed on an |
| 13 | +.. "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +.. KIND, either express or implied. See the License for the |
| 15 | +.. specific language governing permissions and limitations |
| 16 | +.. under the License. |
| 17 | +
|
| 18 | +Distributing expressions across processes |
| 19 | +========================================= |
| 20 | + |
| 21 | +DataFusion expressions (:py:class:`~datafusion.Expr`) can be serialized and |
| 22 | +shipped across process boundaries — useful for distributing work over a |
| 23 | +``multiprocessing.Pool``, a Ray actor pool, or any framework that supports a |
| 24 | +per-worker initialization hook. |
| 25 | + |
| 26 | +Pickle support |
| 27 | +-------------- |
| 28 | + |
| 29 | +:py:class:`~datafusion.Expr` implements the pickle protocol directly. Call |
| 30 | +:py:func:`pickle.dumps` on an expression and ship the bytes; the receiver |
| 31 | +calls :py:func:`pickle.loads`. Python *scalar UDFs* are cloudpickled into the |
| 32 | +proto wire format by a Rust-side codec (``PythonUDFCodec``), so the blob is |
| 33 | +self-contained — the receiver does not need to pre-register the UDF. |
| 34 | + |
| 35 | +.. code-block:: python |
| 36 | +
|
| 37 | + import multiprocessing as mp |
| 38 | + import pickle |
| 39 | +
|
| 40 | + import pyarrow as pa |
| 41 | + from datafusion import SessionContext, col, lit, udf |
| 42 | +
|
| 43 | + def init_worker(): |
| 44 | + # Optional: install a worker context for aggregate / window UDFs, |
| 45 | + # table providers, or Rust-side function registrations. Not needed |
| 46 | + # for built-ins or Python scalar UDFs. |
| 47 | + pass |
| 48 | +
|
| 49 | + def evaluate(blob_and_batch): |
| 50 | + blob, batch = blob_and_batch |
| 51 | + expr = pickle.loads(blob) |
| 52 | + ctx = SessionContext() |
| 53 | + df = ctx.from_pydict({"a": batch}) |
| 54 | + return df.with_column("result", expr).select("result").to_pydict()["result"] |
| 55 | +
|
| 56 | + if __name__ == "__main__": |
| 57 | + double = udf( |
| 58 | + lambda arr: pa.array([(v.as_py() or 0) * 2 for v in arr]), |
| 59 | + [pa.int64()], pa.int64(), volatility="immutable", name="double", |
| 60 | + ) |
| 61 | + blob = pickle.dumps(double(col("a"))) |
| 62 | +
|
| 63 | + mp_ctx = mp.get_context("forkserver") |
| 64 | + with mp_ctx.Pool(processes=4) as pool: |
| 65 | + results = pool.map( |
| 66 | + evaluate, |
| 67 | + [(blob, [1, 2, 3]), (blob, [10, 20, 30])], |
| 68 | + ) |
| 69 | + print(results) # [[2, 4, 6], [20, 40, 60]] |
| 70 | +
|
| 71 | +Worker-scoped context |
| 72 | +--------------------- |
| 73 | + |
| 74 | +For references the codec cannot inline — aggregate UDFs, window UDFs, FFI |
| 75 | +capsule UDFs, or anything resolved through the |
| 76 | +:class:`SessionContext`'s function registry — set a worker-scoped context |
| 77 | +once per process using :py:func:`datafusion.ipc.set_worker_ctx`: |
| 78 | + |
| 79 | +.. code-block:: python |
| 80 | +
|
| 81 | + from datafusion import SessionContext |
| 82 | + from datafusion.ipc import set_worker_ctx |
| 83 | +
|
| 84 | + def init_worker(): |
| 85 | + ctx = SessionContext() |
| 86 | + ctx.register_udaf(my_aggregate) # if needed |
| 87 | + set_worker_ctx(ctx) |
| 88 | +
|
| 89 | + with mp.get_context("forkserver").Pool( |
| 90 | + processes=4, initializer=init_worker |
| 91 | + ) as pool: |
| 92 | + ... |
| 93 | +
|
| 94 | +Without a worker context, unpickling falls back to a fresh |
| 95 | +:py:class:`SessionContext`. Built-in functions resolve; Python scalar UDFs |
| 96 | +ride along inside the blob via the codec. References to aggregate / window |
| 97 | +UDFs or other registry-only entries raise an informative error if not |
| 98 | +registered on the worker. |
| 99 | + |
| 100 | +Python 3.14 default change |
| 101 | +-------------------------- |
| 102 | + |
| 103 | +Python 3.14 changed the POSIX default start method for |
| 104 | +:py:mod:`multiprocessing` from ``fork`` to ``forkserver``. With ``fork``, a |
| 105 | +context set in the parent was visible in workers via copy-on-write; with |
| 106 | +``forkserver`` and ``spawn`` it is not. The codec + worker-init pattern works |
| 107 | +on every start method — prefer it over relying on inherited state. |
| 108 | + |
| 109 | +Trade-offs of inline UDFs |
| 110 | +------------------------- |
| 111 | + |
| 112 | +* **Blob size** — cloudpickled callables add bytes per blob. A trivial |
| 113 | + built-in expression is ~20 bytes; an expression referencing a Python scalar |
| 114 | + UDF is hundreds of bytes (the cloudpickled callable + signature). Pre-register |
| 115 | + shared UDFs on workers via :py:func:`~datafusion.ipc.set_worker_ctx` when |
| 116 | + the same UDF is shipped many times and you want to avoid the overhead. |
| 117 | +* **Closure capture** — cloudpickle captures closure state. Surprises are |
| 118 | + possible if the UDF closes over large objects, module-level mutable state, |
| 119 | + or non-portable file paths. |
| 120 | +* **FFI scalar UDFs cannot be inlined** — PyCapsule-backed UDFs have no |
| 121 | + Python callable to cloudpickle. The codec leaves their ``fun_definition`` |
| 122 | + empty; the receiver must have a matching registration. |
| 123 | +* **Aggregate and window UDFs cannot be inlined yet** — their Python state |
| 124 | + is held inside opaque factory closures on the Rust side. Pre-register on |
| 125 | + the worker. |
| 126 | + |
| 127 | +Security |
| 128 | +-------- |
| 129 | + |
| 130 | +.. warning:: |
| 131 | + |
| 132 | + Pickle blobs containing inlined UDFs deserialize via :py:mod:`cloudpickle`, |
| 133 | + which executes arbitrary code on the receiver. Only :py:func:`pickle.loads` |
| 134 | + blobs from trusted sources. For untrusted-source workflows, restrict the |
| 135 | + sender to built-in functions and pre-registered Rust-side UDFs. |
| 136 | + |
| 137 | +See also |
| 138 | +-------- |
| 139 | + |
| 140 | +* :py:mod:`datafusion.ipc` — module-level API reference. |
| 141 | +* ``examples/ray_pickle_expr.py`` — runnable Ray actor example. |
0 commit comments