apache / datafusion-comet

Apache DataFusion Comet Spark Accelerator
https://datafusion.apache.org/comet
Apache License 2.0
823 stars 163 forks source link

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

Open SemyonSinchenko opened 2 months ago

SemyonSinchenko commented 2 months ago

What is the problem the feature request solves?

Spark provide multiple ways to run arrow-backed UDFs. The current 3.5 supports mapInArrow, in the future 4.0 there will be also applyInArrow.

My understanding of how it works in Spark under the hood is quite limited, so correct me if I'm wrong. At the moment, if Spark see in the plan PythonMapInArrow it will internaly do a conversion from rows to arrow-batches that should be a columnar representation of the data: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInBatchExec.scala#L36

That is a minimal example of running mapInArrow in Spark 3.4:

import pandas as pd
import pyarrow as pa
from pyspark.sql import SparkSession, types as T

if __name__ == "__main__":
    spark = SparkSession.builder.getOrCreate()
    iris = pd.read_csv("https://raw.githubusercontent.com/mwaskom/seaborn-data/refs/heads/master/iris.csv")
    iris.to_parquet("iris.parquet", index=False)

    def arrow_fun(arrow_batch: pa.RecordBatch) -> pa.RecordBatch:
        pdf = arrow_batch.to_pandas()
        pdf["avg_length"] = (pdf["sepal_length"] + pdf["petal_length"]) / 2

        return pa.RecordBatch.from_pandas(pdf)

    schema = T.StructType(
        [
            T.StructField("sepal_length", T.DoubleType()),
            T.StructField("sepal_width", T.DoubleType()),
            T.StructField("petal_length", T.DoubleType()),
            T.StructField("petal_width", T.DoubleType()),
            T.StructField("species", T.StringType()),
            T.StructField("avg_length", T.DoubleType()),
        ]
    )

    test_data = spark.read.parquet("iris.parquet")
    new_data = test_data.mapInArrow(arrow_fun, schema)
    new_data.explain(mode="extended")

If I try to run it with Comet enabled it will generate the following physical plan:

PythonMapInArrow arrow_fun(sepal_length#0, sepal_width#1, petal_length#2, petal_width#3, species#4)#10, [sepal_length#11, sepal_width#12, petal_length#13, petal_width#14,species#15, avg_length#16]
+- *(1) ColumnarToRow
   +- CometScan parquet [sepal_length#0,sepal_width#1,petal_length#2,petal_width#3,species#4] Batched: true, DataFilters: [], Format: CometParquet, Location: InMemoryFileIndex(1 paths)[file:/var/home/sem/github/tmp/iris.parquet], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<sepal_length:double,sepal_width:double,petal_length:double,petal_width:double,species:string>

If I understand it right, the following happens:

  1. Comet read parquet file in columnar form
  2. Comet do conversion of arrow-backed columnar data to row-oriented JVM data
  3. Spark do internal conversion of row-oriented JVM data to arrow-backed columnar data
  4. This columnar data is passed to PythonMapInArrow

It seems to me that points 2-3 are redundant and the arrow batches that are required for mapInArrow can be created directly from the Comet arrow-backed columns and this operation should be a kind of zero-copy... And actually the back conversion from spark columnar batch to comet columnar batch may zero-copy too, so in theory Comet does not need to make a fallback to spark in this case, right?

Describe the potential solution

I do no know an exact solution. It is mostly a question.

Additional context

I'm willing to implement it by myself, I'm ready to work on it. But I need a guidance and help with an overall design of how it should be done (if it is feasible).

The native support of arrow-backed UDFs opens a lot of new cool ways of using Comet. I see that it can gain a huge boost for most of ML/MLOps tasks that are typically done in Spark via arrow-backed UDFs (pandas, polars, pyarrow or even rust code built with maturin to a python-module).

andygrove commented 1 month ago

Thanks for writing this up @SemyonSinchenko. Your reasoning seems sound to me, and I agree that is would be quite a unique and powerful feature for Comet. I am not sure how much work it will be, but I am happy to offer guidance (and will probably need some help from @viirya as well). I will start getting up to speed on Spark's PythonMapInArrowExec.

SemyonSinchenko commented 1 month ago

I took a look on the spark code and that is what I found:

  1. Conversion from rows to arrow batches is done in org.apache.spark.sql.execution.python.BasicPythonArrowInput
  2. Back conversion from arrow batches to rows is done org.apache.spark.sql.execution.python.BasicPythonArrowOutput

Both are private[python] in spark. As I can realize BaseArrowPythonRunner expects InternalRow as input, converts it to arrow, and returns org.apache.spark.sql.vectorized.ColumnarBatch.

What am I thinking now is about implementing CometArrowPythonRunner extends BasePythonRunner[CometColumnarBatch, CometColumnarBatch]. CometColumnarBatch here is just a Scala wrapper over comet memory. There are a lot of configs that are handled by Spark, like spark.pyspark.driver.python and spark.pyspark.python. These configs define which python interpreter is used and also what is included to PYTHONPATH: that allows users to add own dependencies to python vectorized UDFs. In the case when execution will be in Rust, all these configs should be handled by Comet including managing python virtual environments, collecting python metrics, etc.

So, I see a possible solution the following:

It seems to me that re-using how spark handle python UDFs would be easier than implementing it from scratch using datafusion. But I'm not 100% sure.

andygrove commented 1 month ago

It seems to me that re-using how spark handle python UDFs would be easier than implementing it from scratch using datafusion. But I'm not 100% sure.

Yes, that is the approach I would take.

Your high level plan sounds good to me.

SemyonSinchenko commented 1 month ago

Your high level plan sounds good to me.

Cool, thank you! I will start working on it.

SemyonSinchenko commented 3 weeks ago

@andygrove It looks like this task is very hard for me and I need more guidance.

To avoid handling all the python-related configurations in comet and reuse as much as possible from spark I need create a class that is similar to the BaseArrowPythonRunner but instead of Iterator[InternalRow] it should take CometExecIterator: image

That requires from me to implement the trait similar to the BasicPythonArrowInput that implements PythonArrowInput. It looks like I need to override only this method: image

And I need to use ArrowStreamWriter. But I failed to realize how to do it having CometExecIterator and without copying the data... It seems to me that such a functionality should be already implemented in the org.apache.comet.vector.NativeUtils but I failed to find it. Can you please guide me a little to the right direction? Or maybe there is a part of comet's code that I can use as an inspiration?

Also there is a big chance that I'm just going into a wrong direction...

Thanks in advance!