-
Notifications
You must be signed in to change notification settings - Fork 151
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Is it possible to support PyArrow backed UDFs in Comet natively? #957
Comments
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 |
I took a look on the spark code and that is what I found:
Both are What am I thinking now is about implementing 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. |
Yes, that is the approach I would take. Your high level plan sounds good to me. |
Cool, thank you! I will start working on it. |
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 alsoapplyInArrow
.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:If I try to run it with Comet enabled it will generate the following physical plan:
If I understand it right, the following happens:
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 withmaturin
to a python-module).The text was updated successfully, but these errors were encountered: