I don’t think we should so quickly move past the drawbacks of this
approach. The problems are significant enough that using invoke is not
sufficient on its own. But, I think we can add it as an optional extension
to shore up the weaknesses.

Here’s a summary of the drawbacks:

   - Magic function signatures are error-prone
   - Spark would need considerable code to help users find what went wrong
   - Spark would likely need to coerce arguments (e.g., String,
   Option[Int]) for usability
   - It is unclear how Spark will find the Java Method to call
   - Use cases that require varargs fall back to casting; users will also
   get this wrong (cast to String instead of UTF8String)
   - The non-codegen path is significantly slower

The benefit of invoke is to avoid moving data into a row, like this:

-- using invoke
int result = udfFunction(x, y)

-- using row
udfRow.update(0, x); -- actual: values[0] = x;
udfRow.update(1, y);
int result = udfFunction(udfRow);

And, again, that won’t actually help much in cases that require varargs.

I suggest we add a new marker trait for BoundMethod called SupportsInvoke.
If that interface is implemented, then Spark will look for a method that
matches the expected signature based on the bound input type. If it isn’t
found, Spark can print a warning and fall back to the InternalRow call:
“Cannot find udfFunction(int, int)”.

This approach allows the invoke optimization, but solves many of the
problems:

   - The method to invoke is found using the proposed load and bind approach
   - Magic function signatures are optional and do not cause runtime
   failures
   - Because this is an optional optimization, Spark can be more strict
   about types
   - Varargs cases can still use rows
   - Non-codegen can use an evaluation method rather than falling back to
   slow Java reflection

This seems like a good extension to me; this provides a plan for optimizing
the UDF call to avoid building a row, while the existing proposal covers
the other cases well and addresses how to locate these function calls.

This also highlights that the approach used in DSv2 and this proposal is
working: start small and use extensions to layer on more complex support.

On Wed, Feb 10, 2021 at 9:04 AM Dongjoon Hyun <dongjoon.h...@gmail.com>
wrote:

Thank you all for making a giant move forward for Apache Spark 3.2.0.
> I'm really looking forward to seeing Wenchen's implementation.
> That would be greatly helpful to make a decision!
>
> > I'll implement my idea after the holiday and then we can have
> more effective discussions. We can also do benchmarks and get some real
> numbers.
> > FYI: the Presto UDF API
> <https://prestodb.io/docs/current/develop/functions.html> also
> takes individual parameters instead of the row parameter. I think this
> direction at least worth a try so that we can see the performance
> difference. It's also mentioned in the design doc as an alternative (Trino).
>
> Bests,
> Dongjoon.
>
>
> On Tue, Feb 9, 2021 at 10:18 PM Wenchen Fan <cloud0...@gmail.com> wrote:
>
>> FYI: the Presto UDF API
>> <https://prestodb.io/docs/current/develop/functions.html> also
>> takes individual parameters instead of the row parameter. I think this
>> direction at least worth a try so that we can see the performance
>> difference. It's also mentioned in the design doc as an alternative (Trino).
>>
>> On Wed, Feb 10, 2021 at 10:18 AM Wenchen Fan <cloud0...@gmail.com> wrote:
>>
>>> Hi Holden,
>>>
>>> As Hyukjin said, following existing designs is not the principle of DS
>>> v2 API design. We should make sure the DS v2 API makes sense. AFAIK we
>>> didn't fully follow the catalog API design from Hive and I believe Ryan
>>> also agrees with it.
>>>
>>> I think the problem here is we were discussing some very detailed things
>>> without actual code. I'll implement my idea after the holiday and then we
>>> can have more effective discussions. We can also do benchmarks and get some
>>> real numbers.
>>>
>>> In the meantime, we can continue to discuss other parts of this
>>> proposal, and make a prototype if possible. Spark SQL has many active
>>> contributors/committers and this thread doesn't get much attention yet.
>>>
>>> On Wed, Feb 10, 2021 at 6:17 AM Hyukjin Kwon <gurwls...@gmail.com>
>>> wrote:
>>>
>>>> Just dropping a few lines. I remember that one of the goals in DSv2 is
>>>> to correct the mistakes we made in the current Spark codes.
>>>> It would not have much point if we will happen to just follow and mimic
>>>> what Spark currently does. It might just end up with another copy of Spark
>>>> APIs, e.g. Expression (internal) APIs. I sincerely would like to avoid this
>>>> I do believe we have been stuck mainly due to trying to come up with a
>>>> better design. We already have an ugly picture of the current Spark APIs to
>>>> draw a better bigger picture.
>>>>
>>>>
>>>> 2021년 2월 10일 (수) 오전 3:28, Holden Karau <hol...@pigscanfly.ca>님이 작성:
>>>>
>>>>> I think this proposal is a good set of trade-offs and has existed in
>>>>> the community for a long period of time. I especially appreciate how the
>>>>> design is focused on a minimal useful component, with future optimizations
>>>>> considered from a point of view of making sure it's flexible, but actual
>>>>> concrete decisions left for the future once we see how this API is used. I
>>>>> think if we try and optimize everything right out of the gate, we'll
>>>>> quickly get stuck (again) and not make any progress.
>>>>>
>>>>> On Mon, Feb 8, 2021 at 10:46 AM Ryan Blue <b...@apache.org> wrote:
>>>>>
>>>>>> Hi everyone,
>>>>>>
>>>>>> I'd like to start a discussion for adding a FunctionCatalog interface
>>>>>> to catalog plugins. This will allow catalogs to expose functions to 
>>>>>> Spark,
>>>>>> similar to how the TableCatalog interface allows a catalog to expose
>>>>>> tables. The proposal doc is available here:
>>>>>> https://docs.google.com/document/d/1PLBieHIlxZjmoUB0ERF-VozCRJ0xw2j3qKvUNWpWA2U/edit
>>>>>>
>>>>>> Here's a high-level summary of some of the main design choices:
>>>>>> * Adds the ability to list and load functions, not to create or
>>>>>> modify them in an external catalog
>>>>>> * Supports scalar, aggregate, and partial aggregate functions
>>>>>> * Uses load and bind steps for better error messages and simpler
>>>>>> implementations
>>>>>> * Like the DSv2 table read and write APIs, it uses InternalRow to
>>>>>> pass data
>>>>>> * Can be extended using mix-in interfaces to add vectorization,
>>>>>> codegen, and other future features
>>>>>>
>>>>>> There is also a PR with the proposed API:
>>>>>> https://github.com/apache/spark/pull/24559/files
>>>>>>
>>>>>> Let's discuss the proposal here rather than on that PR, to get better
>>>>>> visibility. Also, please take the time to read the proposal first. That
>>>>>> really helps clear up misconceptions.
>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> Ryan Blue
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> Twitter: https://twitter.com/holdenkarau
>>>>> Books (Learning Spark, High Performance Spark, etc.):
>>>>> https://amzn.to/2MaRAG9  <https://amzn.to/2MaRAG9>
>>>>> YouTube Live Streams: https://www.youtube.com/user/holdenkarau
>>>>>
>>>> --
Ryan Blue

Reply via email to