Thanks Kui,  the updated answers and planned doc changes address my earlier
questions well. Overall +1 on this.

On the hang case, agreed a watchdog isn't practical given the
mailbox model.
Making the javadoc explicit about what "non-blocking" rules out (no
blocking I/O,
no future.get()) and adding a short correct-impl example should be
sufficient.

On AsyncScalarFunction, it makes sense to keep it as a watch item and
revisit
when a concrete need surfaces, no need to include in this FLIP.

Thanks for driving this!


Best,
Lincoln Lee


Kui Yuan <[email protected]> 于2026年5月28日周四 14:47写道:

> Hi Alan,
> Thanks a lot for the feedback.
>
> 1) You're right — the FLIP missed the correlate query codepath. That
> was an oversight on my part. I'll update the FLIP to cover it.
>
> 2) +1, agreed. There's actually a structural reason the synchronous
> contract makes sense beyond being a workaround: AsyncWaitOperator
> dispatches timeout() from the operator main thread (mailbox), so if
> the handler itself kicked off more async work, the framework would
> have to extend in-flight tracking, retry-strategy coordination, and
> checkpoint barrier alignment to cover the fallback chain too. Keeping
> the boundary at "the handler completes synchronously" lets users own
> that complexity inside eval(...) where they already manage the
> future's lifecycle. On Nth-attempt-aware handling — agree it's beyond
> this FLIP. The existing table.exec.async-table.retry-* knobs already
> cover the simple "retry on empty response / any exception" case;
> richer per-attempt logic feels like a separate proposal.
>
> Best,
> Kui.Yuan
>
> Alan Sheinberg via dev <[email protected]> 于2026年5月27日周三 01:58写道:
> >
> > Hi Kui.Yuan,
> >
> > This is a great idea to expose the timeout method of AsyncFunction to the
> > user-facing UDF. I have a couple comments/questions about the design:
> >
> > 1) The FLIP mentions lookup joins in particular, but the correlate query
> > codepath also exists:
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-498%3A+AsyncTableFunction+for+async+table+function+support
> > .  Is this intended to be supported as well?  I think it might
> > require updating AsyncCorrelateRunner.
> >
> > > The `CompletableFuture` must be completed synchronously;
> >
> > 2) User code could always set a "soft" timeout, and use some of the
> > remaining time budget for a subsequent async call to handle the soft
> > timeout, if necessary.  So this seems like a reasonable limitation, even
> if
> > they needed async support. This is similar to the more general case of a
> > recoverable error scenario where you might want to handle the Nth attempt
> > at a request differently, but that's probably beyond this FLIP's scope.
> >
> > Thanks,
> > Alan
> >
> > On Tue, May 26, 2026 at 2:21 AM Kui Yuan <[email protected]> wrote:
> >
> > > Hi Lincoln,
> > >
> > > Thanks a lot for the great questions! I'll incorporate the relevant
> > > clarifications into the FLIP. Let me share some quick thoughts here:
> > >
> > > Q1 (interaction with retry): The timeout reuses the existing flow in
> > > AsyncWaitOperator. The configured timeout is the total budget measured
> > > from the first asyncInvoke. When the async-lookup timer expires, any
> > > pending delayed retry is cancelled and timeout(...) is invoked exactly
> > > once. The retry result/exception predicates are not consulted on the
> > > timeout path.
> > >
> > > Q2 ("CompletableFuture must be completed synchronously"): This is a
> > > runtime guarantee, not just a javadoc convention. The codegen'd code
> > > checks whether the future is completed; if it isn't, an error is
> > > raised and the job fails fast.
> > >
> > > Q3 (failure modes of the user timeout method):
> > >
> > > If the timeout method itself throws, we surface the exception and fail
> > > the job fast — no silent swallowing.
> > > If timeout blocks synchronously and never returns, it would block the
> > > mailbox thread and effectively freeze the task. This is admittedly a
> > > serious situation, and we don't have a mechanism for it today. Adding
> > > one would require a separate timeout-on-timeout, which could break the
> > > single-threaded mailbox assumption. So for now the simpler approach is
> > > to document the contract in the javadoc — timeout must be
> > > non-blocking. Open to suggestions if you have a better idea here.
> > >
> > > Q4 (var-args matching): For var-args, the signatures must mirror each
> > > other. If eval is eval(CompletableFuture<T>, String...), then timeout
> > > must be timeout(CompletableFuture<T>, String...). Any mismatch is
> > > rejected at startup with a clear error and I'll add unit tests to
> > > cover this.
> > >
> > > Q5 (AsyncScalarFunction): This was an intentional scoping choice. The
> > > use cases driving the custom-timeout requirement today are mostly LLM
> > > calls, where the output is rarely a single scalar — so users tend to
> > > reach for AsyncTableFunction instead of AsyncScalarFunction (the
> > > built-in AsyncPredictFunction also extends AsyncTableFunction). Until
> > > we hear a concrete user need on the scalar side, supporting timeout
> > > only on AsyncTableFunction feels sufficient. Happy to revisit if/when
> > > such a need surfaces.
> > >
> > > Thanks again for the careful review!
> > >
> > > Best,
> > > Kui.Yuan
> > >
> > >
> > >
> > > Lincoln Lee <[email protected]> 于2026年5月26日周二 15:16写道:
> > > >
> > > > Thanks for the FLIP, nice addition!   A few questions before the
> vote:
> > > >
> > > > 1. Interaction with the retry strategy: the FLIP doesn't say where
> the
> > > new
> > > > timeout hook sits in that flow, worth spelling out explicitly.
> > > >
> > > > 2. "CompletableFuture must be completed synchronously." Is this a
> javadoc
> > > > convention or a runtime guarantee? If it's only a convention, it
> would
> > > help
> > > > to
> > > > document what happens when users break it. If it's enforced, a brief
> note
> > > > would
> > > > be useful.
> > > >
> > > > 3. Failure modes of the user timeout method: two cases aren't
> covered:
> > > > The method itself throws, do we swallow and degrade the Exception, or
> > > fail
> > > > the job?
> > > > The method never completes (bug or hang), does the operator stall
> > > > indefinitely?
> > > >
> > > > 4. On the public interface: since timeout's parameter list mirrors
> eval,
> > > > how is
> > > > a var-args eval (e.g. eval(CompletableFuture<T>, String...))
> expected to
> > > be
> > > > matched?
> > > >
> > > > 5. How has AsyncScalarFunction been considered here? It shares the
> same
> > > > timeout-prone remote-call pattern, so it seems natural to extend the
> same
> > > > mechanism there as well, is that in scope, a follow-up, or
> intentionally
> > > > left out?
> > > >
> > > >
> > > > Best,
> > > > Lincoln Lee
> > > >
> > > >
> > > > Gen Luo <[email protected]> 于2026年5月26日周二 11:02写道:
> > > >
> > > > > Thanks for driving this proposal forward! This addresses a real
> pain
> > > point
> > > > > we've been hearing about for a while.
> > > > >
> > > > > Many of our users rely on AsyncTableFunction or Lookup Join to
> > > implement
> > > > > custom external service calls and data fetching, typically for RAG
> or
> > > LLM
> > > > > inference scenarios. Due to the inherent instability of these
> external
> > > > > services, timeouts occur occasionally, and users want to apply
> fallback
> > > > > strategies (e.g., falling back to a local rule-based model) rather
> than
> > > > > failing the entire job. However, this hasn't been achievable so
> far —
> > > the
> > > > > hard-coded TimeoutException behavior introduces stability risks,
> > > forcing
> > > > > users to keep increasing the timeout value to absurd levels and
> work
> > > around
> > > > > the issue in various hacky ways. Worse, each user tends to hit this
> > > pitfall
> > > > > independently before realizing the limitation.
> > > > >
> > > > > Adding a timeout interface not only addresses this pain point, but
> also
> > > > > aligns the API contract between AsyncTableFunction and
> AsyncFunction,
> > > > > avoiding unnecessary confusion for users.
> > > > >
> > > > > Big +1 from our side — looking forward to seeing this land.
> > > > >
> > > > > On Mon, May 25, 2026 at 7:53 PM Xia Sun <[email protected]>
> wrote:
> > > > >
> > > > > > Hi Kui.Yuan,
> > > > > >
> > > > > > Thanks for driving this!
> > > > > >
> > > > > > In our production practice, the asynchronous I/O capability of
> > > > > > AsyncTableFunction has shown excellent performance in
> > > > > > batch LLM inference scenarios. We urgently need a custom timeout
> UDF
> > > > > > for this use case. It would help us handle inference requests
> that
> > > > > > time out—especially long-context requests—more precisely, and
> avoid
> > > > > > excessive retries that could otherwise block downstream data.
> > > > > >
> > > > > > +1 to this proposal.
> > > > > >
> > > > > > Best,
> > > > > >
> > > > > > Xia
> > > > > >
> > > > > > Kui Yuan <[email protected]> 于2026年5月22日周五 11:21写道:
> > > > > >
> > > > > > > Hi All,
> > > > > > >
> > > > > > > I'd like to open a discussion for FLIP-580: AsyncTableFunction
> > > supports
> > > > > > > user-defined timeout handling logic [1].
> > > > > > >
> > > > > > > An increasing number of users are leveraging
> AsyncTableFunction to
> > > > > invoke
> > > > > > > remote inference clusters. Such invocations are essentially
> remote
> > > > > > > inference requests, which are far more prone to timeouts than
> > > regular
> > > > > I/O
> > > > > > > operations. Users expect to be able to define custom handling
> logic
> > > > > when
> > > > > > a
> > > > > > > timeout occurs — for example, falling back to default data or
> > > > > > accumulating
> > > > > > > failure statistics — rather than having a TimeoutException
> thrown
> > > > > > directly
> > > > > > > and causing the entire job to fail.
> > > > > > >
> > > > > > >
> > > > > > > This FLIP proposal allow users to define custom timeout
> handling
> > > logic
> > > > > > > inside AsyncTableFunction.
> > > > > > >
> > > > > > > I've already discussed the implementation details with @Luogen
> > > offline,
> > > > > > and
> > > > > > > there's a POC attached [2].
> > > > > > >
> > > > > > >
> > > > > > > Looking forward to your feedback.
> > > > > > >
> > > > > > > Bests,
> > > > > > > Kui.Yuan
> > > > > > >
> > > > > > > [1]:
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> https://urldefense.com/v3/__https://cwiki.apache.org/confluence/display/FLINK/FLIP-580*3A*AsyncTableFunction*supports*user-defined*timeout*handling*logic__;JSsrKysrKw!!Ayb5sqE7!uTEOgL-aoWOoxNwvQHWKbSUTZgVn_AjMT8-6Xc7Ub9Aj0UccuvazQdHffMt3DnSo-WJu42YJ04xcyKPDAYb8Sg$
> > > > > > >
> > > > > > > [2]:
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> https://urldefense.com/v3/__https://github.com/yuchengxin/flink/commit/5a46cd05c48e41a582271dcb9d9842e330871a0b__;!!Ayb5sqE7!uTEOgL-aoWOoxNwvQHWKbSUTZgVn_AjMT8-6Xc7Ub9Aj0UccuvazQdHffMt3DnSo-WJu42YJ04xcyKO3xGyfxQ$
> > > > > > >
> > > > > >
> > > > >
> > >
>

Reply via email to