Hello all,
We want to implement a custom function that sends HTTP requests to a remote
endpoint using Flink SQL. Even though the function will behave like a
normal UDF, the runtime would issue calls asynchronously to achieve high
throughput for these remote (potentially high latency) calls. What i
is to support Mini-Batch Lookup Join by the framework
> layer, do a RPC call by a batch input row, which can improve throughput.
>
> Best,
> Ron
>
> Alan Sheinberg 于2023年9月19日周二 07:34写道:
>
> > Hello all,
> >
> > We want to implement a custom function tha
a similar FLIP[1] available:
> > Can this meet your needs?
> >
> > [1].
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-313%3A+Add+support+of+User+Defined+AsyncTableFunction
> >
> >
> > Best,
> > Feng
> >
> > On
I'd like to start a discussion of FLIP-400: AsyncScalarFunction for
asynchronous scalar function support [1]
This feature proposes adding a new UDF type AsyncScalarFunction which is
invoked just like a normal ScalarFunction, but is implemented with an
asynchronous eval method. I had brought this
find
> another name which doesn't carry that connotation; maybe
> AsyncCalcSplitRuleBase. (An AsyncCalcSplitRuleBase which handles Python
> and Async functions seems reasonable.)
>
> Cheers,
>
> Jim
>
> On Wed, Dec 6, 2023 at 5:45 PM Alan Sheinberg
> wrote:
>
> > I
etRequirements():
> Set` we can add a new requirement `ORDERED` which
> should also be the default for AsyncScalarFunction. `getRequirements()`
> can be overwritten and return a set without this requirement if the user
> intents to do this.
>
>
> Thanks,
> Timo
>
>
>
and retry-strategy. We can later work on a per-function
> >>> configuration (potentially also needed for other use cases).
> >>>
> >>> 2) Semantical declaration
> >>>
> >>> Regarding
> >>>
> >>> `table.exec.async-scalar.catalog.db.func-name.outp
if the query is > an append-only `SELECT FUNC(c) FROM t`,
> >> I don't see a reason why the > operator is not allowed to produce
> unordered results.
> >
> >
> > @Timo, @Alan: IIUC, there seems to be something wrong here. Take kafka
> as source and my
parts? Esp.:
>
> @Override
> public Set getRequirements() {
> return Collections.singleton(FunctionRequirement.ORDERED);
> }
>
> Otherwise I have no objections to start a VOTE soonish. If others are
> fine as well?
>
> Regards,
> Timo
>
>
> On 19
K/FLIP-400%3A+AsyncScalarFunction+for+asynchronous+scalar+function+support
> [2]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211883963
> [3]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-234%3A+Support+Retryable+Lookup+Join+To+Solve+Delayed+Updates+Issue+In
Hi everyone,
I'd like to start a vote on FLIP-400 [1]. It covers introducing a new UDF
type, AsyncScalarFunction for completing invocations asynchronously. It
has been discussed in this thread [2].
I would like to start a vote. The vote will be open for at least 72 hours
(until December 28th 18
> > >>
> > >>> +1 (binding)
> > >>>
> > >>> On Fri, Dec 22, 2023 at 1:44 AM Jim Hughes
> >
> > >>> wrote:
> > >>>>
> > >>>> Hi Alan,
> > >>>>
> > >>>
The vote for FLIP-400: AsyncScalarFunction for asynchronous scalar function
support [1] has concluded (discussion thread [2]). The vote will be closed
[3].
There were 7 approving votes, 4 binding and 3 non-binding, and there were
no disapprovals:
- Martijn Visser (binding)
- Lincoln Lee (binding
I'd like to start a discussion of FLIP-452: Allow Skipping Invocation of
Function Calls While Constant-folding [1]
This feature proposes adding a new
method FunctionDefinition.allowConstantFolding() as part of the Flink
Table/SQL API. This would be used to determine whether an expression
containi
gt; state of the function from the outside.
>
> Otherwise +1 for the FLIP.
>
> Regards,
> Timo
>
>
> On 30.04.24 00:57, Alan Sheinberg wrote:
> > I'd like to start a discussion of FLIP-452: Allow Skipping Invocation of
> > Function Cal
h short description and examples use cases (similar to the
> > motivation of the FLIP) where this could be useful in UDFs.
> >
> > Best,
> > Muhammet
> >
> > [1]:
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/dev/table/functions/udfs/#
Hi everyone,
I'd like to start a vote on FLIP-452 [1]. It covers adding a new method
FunctionDefinition.supportsConstantFolding() as part of the Flink Table/SQL
API to allow skipping invocation of functions while constant-folding. It
has been discussed in this thread [2].
I would like to start a
The vote for FLIP-452: Allow Skipping Invocation of Function Calls While
Constant-folding [1] has concluded (discussion thread [2]). The vote will
be closed [3].
There were 4 approving votes, all binding, and there were no disapprovals:
- Stefan Richter (binding)
- Timo Walther (binding)
- Bencha
The vote for FLIP-498: AsyncTableFunction for async table function support
[1] has concluded (discussion thread [2]). The vote will be closed [3].
There were 5 approving votes, 3 binding and 2 non-binding, and there were
no disapprovals:
- Fabian Hüske (binding)
- Timo Walther (binding)
- Martijn
Hi all,
Is there any more feedback I can incorporate before calling a vote?
Thanks,
Alan
On Tue, Jan 28, 2025 at 1:50 PM Alan Sheinberg
wrote:
> Hi Fabian,
>
> I addressed your comments below.
>
>
>> * BundledKeySegement
>> * should it be accumulatorS instead of
I'd like to start a discussion of FLIP-491: BundledAggregateFunction for
batched aggregation [1]
This feature proposes adding a new interface BundledAggregateFunction that
can be implemented by AggregateFunction UDFs. This allows the use of a
batched method call so that users can handle many rows
tributor.
> >
> > So I'm +1 to continue with this FLIP, unless there are objections from
> > others?
> >
> > Cheers,
> > Timo
> >
> >
> > On 13.01.25 18:35, Alan Sheinberg wrote:
> > > I just wanted to add a little bit to my response as I co
Hi everyone,
I'd like to start a vote on FLIP-498 [1]. It proposes exposing
AsyncTableFunction as a proper user defined function. The type already
exists for Lookup Joins, but isn't usable as other UDFs. This FLIP would
bring it up to parity with others. It has been discussed in this thread [2].
nicate whether or not this is supported to the planner? Enforcing
> > the retract() feature in the interface specification could be an option,
> > but esp for window aggregations there might not be a retract required.
> >
> > Also how do you plan to support merge() in this
AccumulateRetract()`
> * example works with a single accumulator, not a list of accumulators
> * in general, check the code examples for compliance with the proposed API.
> * Some use `bundle()` instead of `bundledAccumulateRetract()`.
> * There might be other mistakes that sne
you elaborate where the
> differences come from?
>
> Best,
> Dawid
>
> On Thu, 2 Jan 2025 at 20:48, Alan Sheinberg .invalid>
> wrote:
>
> > I'd like to start a discussion of FLIP-498: AsyncTableFunction for async
> > table function support [1]
> >
> &g
ort, which seemed to be a previous point of discussion. This is in
line with what I implemented for AsycScalarFunction and has direct parity
with the configurations exposed. I just wanted to highlight that
difference.
Thanks,
Alan
On Mon, Jan 13, 2025 at 8:55 AM Alan Sheinberg
wrote:
> T
I'd like to start a discussion of FLIP-498: AsyncTableFunction for async
table function support [1]
This feature proposes exposing AsyncTableFunction as a proper user defined
function. The type already exists for Lookup Joins, but isn't usable as
other UDFs. This FLIP would bring it up to parity
n add a
> default implementation such as:
>
> default List
> bundledAccumulateRetract(List batch)
> throws Exception {
>throw new UnsupportOperationException("Method must be implemented
> when supportsBundling() is true.");
> }
>
>
&g
Hi everyone,
I'd like to start a vote on FLIP-491 [1]. It proposes
adding BundledAggregateFunction as an interface that aggregate function
UDFs can implement. The interface exposes a batch method which should make
calling out to external services more efficient, whether fetching data or
doing remo
The vote for FLIP-491: BundledAggregateFunction for batched aggregation [1]
has concluded (discussion thread [2]). The vote will be closed [3].
There were 4 approving votes, all binding, and there were no disapprovals:
-Timo Walther (binding)
- Fabian Hüske (binding)
- Stefan Richter (binding)
-
Hi Becket, Timo,
I just wanted to jump in as well, as the author of FLIP-498.
This FLIP has been subsumed by FLIP-498: AsyncTableFunction for async
> table function support [1]. In the discussion for FLIP-498, we decided
> to discard FLIP-313 as it has been abandoned for a while.
This was the i
t; And thanks for the patch, Alan. I will also take a look at it this week.
> This is really something long wanted, as it simplifies the
> AsyncTableFunction calling syntax by so much.
>
> Cheers,
>
> Jiangjie (Becket) Qin
>
> On Tue, May 27, 2025 at 9:05 AM Alan Sheinberg
Hi Weiqing,
>From your doc, the entrypoint for UDF calls in the codegen is
ExprCodeGenerator which should invoke BridgingSqlFunctionCallGen, which
could be instrumented with metrics. This works well for synchronous calls,
but what about ASYNC_SCALAR and the soon to be merged ASYNC_TABLE (
https:/
Jan 3, 2024 at 11:29 AM Alan Sheinberg
wrote:
> Thank you everyone for participating in the vote. I'm closing this vote
> now and will announce the results in a separate thread.
>
> -Alan
>
> On Tue, Jan 2, 2024 at 8:40 AM Piotr Nowojski
> wrote:
>
>> +1 (bindin
Alan Sheinberg created FLINK-33978:
--
Summary: FLIP-400: AsyncScalarFunction for asynchronous scalar
function support
Key: FLINK-33978
URL: https://issues.apache.org/jira/browse/FLINK-33978
Project
Alan Sheinberg created FLINK-34028:
--
Summary: Allow multiple parallel async calls in a single operator
Key: FLINK-34028
URL: https://issues.apache.org/jira/browse/FLINK-34028
Project: Flink
Alan Sheinberg created FLINK-34027:
--
Summary: Create initial PR for FLIP-400
Key: FLINK-34027
URL: https://issues.apache.org/jira/browse/FLINK-34027
Project: Flink
Issue Type: Sub-task
Alan Sheinberg created FLINK-35403:
--
Summary: FLIP-452: Allow Skipping Invocation of Function Calls
While Constant-folding
Key: FLINK-35403
URL: https://issues.apache.org/jira/browse/FLINK-35403
Alan Sheinberg created FLINK-35716:
--
Summary: Fix use of AsyncScalarFunction in join conditions
Key: FLINK-35716
URL: https://issues.apache.org/jira/browse/FLINK-35716
Project: Flink
Issue
Alan Sheinberg created FLINK-36472:
--
Summary: Correlates with UDTF don't handle right-side conditions
or projections
Key: FLINK-36472
URL: https://issues.apache.org/jira/browse/FLINK-36472
Pr
Alan Sheinberg created FLINK-37243:
--
Summary: Ensure that async callbacks happen on known thread
Key: FLINK-37243
URL: https://issues.apache.org/jira/browse/FLINK-37243
Project: Flink
Issue
Alan Sheinberg created FLINK-37725:
--
Summary: Share PythonCorrelateSplitRule with Async Scalar
Key: FLINK-37725
URL: https://issues.apache.org/jira/browse/FLINK-37725
Project: Flink
Issue
Alan Sheinberg created FLINK-37721:
--
Summary: Fix Janino Bug which resulted in
Key: FLINK-37721
URL: https://issues.apache.org/jira/browse/FLINK-37721
Project: Flink
Issue Type: Sub-task
Alan Sheinberg created FLINK-37724:
--
Summary: Add AsyncTableFunction as new UDF type
Key: FLINK-37724
URL: https://issues.apache.org/jira/browse/FLINK-37724
Project: Flink
Issue Type: Sub
Alan Sheinberg created FLINK-37723:
--
Summary: FLIP-498: AsyncTableFunction for async table function
support
Key: FLINK-37723
URL: https://issues.apache.org/jira/browse/FLINK-37723
Project: Flink
Alan Sheinberg created FLINK-37816:
--
Summary: FLIP-491: BundledAggregateFunction for batched aggregation
Key: FLINK-37816
URL: https://issues.apache.org/jira/browse/FLINK-37816
Project: Flink
Alan Sheinberg created FLINK-37817:
--
Summary: Add Bundled Aggregate API and operator for group by
Key: FLINK-37817
URL: https://issues.apache.org/jira/browse/FLINK-37817
Project: Flink
Alan Sheinberg created FLINK-37988:
--
Summary: Add left join support
Key: FLINK-37988
URL: https://issues.apache.org/jira/browse/FLINK-37988
Project: Flink
Issue Type: Sub-task
Alan Sheinberg created FLINK-37989:
--
Summary: Add condition support
Key: FLINK-37989
URL: https://issues.apache.org/jira/browse/FLINK-37989
Project: Flink
Issue Type: Sub-task
Alan Sheinberg created FLINK-37927:
--
Summary: Add batch mode support for AsyncTableFunction
Key: FLINK-37927
URL: https://issues.apache.org/jira/browse/FLINK-37927
Project: Flink
Issue Type
Alan Sheinberg created FLINK-37865:
--
Summary: Add documentation for AsyncTableFunction
Key: FLINK-37865
URL: https://issues.apache.org/jira/browse/FLINK-37865
Project: Flink
Issue Type: Sub
Alan Sheinberg created FLINK-38140:
--
Summary: Change name from table.exec.async-scalar.buffer-capacity
to max-concurrent-operations
Key: FLINK-38140
URL: https://issues.apache.org/jira/browse/FLINK-38140
53 matches
Mail list logo