Hi Zakelly,

Thanks for your proposal. The FLIP looks good  to me +1! I'd like to ask
some minor questions
I found that there is also a definition of class `FutureUtils` under `org.
apache. flink. util. concurrent` which seems to offer more interfaces. My
question is:
1. Is it possible for all `FutureUtils` in Flink to reuse the same util
class?
2. It seems that there is no concept of retry, timeout, or delay in your
async state api design . Do we need to provide such capabilities like
`orTimeout` 、`completeDelayed`?

Jing Ge <j...@ververica.com.invalid> 于2024年3月13日周三 20:00写道:

> indeed! I missed that part. Thanks for the hint!
>
> Best regards,
> Jing
>
> On Wed, Mar 13, 2024 at 6:02 AM Zakelly Lan <zakelly....@gmail.com> wrote:
>
> > Hi Jing,
> >
> > The deprecation and removal of original APIs is beyond the scope of
> > current FLIP, but I do add/highlight such information under
> "Compatibility,
> > Deprecation, and Migration Plan" section.
> >
> >
> > Best,
> > Zakelly
> >
> > On Wed, Mar 13, 2024 at 9:18 AM Yunfeng Zhou <
> flink.zhouyunf...@gmail.com>
> > wrote:
> >
> >> Hi Zakelly,
> >>
> >> Thanks for your responses. I agree with it that we can keep the design
> >> as it is for now and see if others have any better ideas for these
> >> questions.
> >>
> >> Best,
> >> Yunfeng
> >>
> >> On Tue, Mar 12, 2024 at 5:23 PM Zakelly Lan <zakelly....@gmail.com>
> >> wrote:
> >> >
> >> > Hi Xuannan,
> >> >
> >> > Thanks for your comments, I modified the FLIP accordingly.
> >> >
> >> > Hi Yunfeng,
> >> >
> >> > Thanks for sharing your opinions!
> >> >
> >> >> Could you provide some hint on use cases where users need to mix sync
> >> >> and async state operations in spite of the performance regression?
> >> >> This information might help address our concerns on design. If the
> >> >> mixed usage is simply something not recommended, I would prefer to
> >> >> prohibit such usage from API.
> >> >
> >> > In fact, there is no scenario where users MUST use the sync APIs, but
> >> it is much easier to use for those who are not familiar with
> asynchronous
> >> programming. If they want to migrate their job from Flink 1.x to 2.0
> >> leveraging some benefits from asynchronous APIs, they may try the mixed
> >> usage. It is not user-friendly to directly throw exceptions at runtime,
> I
> >> think our better approach is to warn users and recommend avoiding this.
> I
> >> added an example in this FLIP.
> >> >
> >> > Well, I do not insist on allowing mixed usage of APIs if others reach
> >> an agreement that we won't support that . I think the most important is
> to
> >> keep the API easy to use and understand, thus I propose a unified state
> >> declaration and explicit meaning in method name. WDYT?
> >> >
> >> >> Sorry I missed the new sink API. I do still think that it would be
> >> >> better to make the package name more informative, and ".v2." does not
> >> >> contain information for new Flink users who did not know the v1 of
> >> >> state API. Unlike internal implementation and performance
> >> >> optimization, API will hardly be compromised for now and updated in
> >> >> future, so I still suggest we improve the package name now if
> >> >> possible. But given the existing practice of sink v2 and
> >> >> AbstractStreamOperatorV2, the current package name would be
> acceptable
> >> >> to me if other reviewers of this FLIP agrees on it.
> >> >
> >> > Actually, I don't like 'v2' either. So if there is another good name,
> >> I'd be happy to apply. This is a compromise to the current situation.
> Maybe
> >> we could refine this after the retirement of original state APIs.
> >> >
> >> >
> >> > Thanks & Best,
> >> > Zakelly
> >> >
> >> >
> >> > On Tue, Mar 12, 2024 at 1:42 PM Yunfeng Zhou <
> >> flink.zhouyunf...@gmail.com> wrote:
> >> >>
> >> >> Hi Zakelly,
> >> >>
> >> >> Thanks for the quick response!
> >> >>
> >> >> > Actually splitting APIs into two sets ... warn them in runtime.
> >> >>
> >> >> Could you provide some hint on use cases where users need to mix sync
> >> >> and async state operations in spite of the performance regression?
> >> >> This information might help address our concerns on design. If the
> >> >> mixed usage is simply something not recommended, I would prefer to
> >> >> prohibit such usage from API.
> >> >>
> >> >> > In fact ... .sink2`.
> >> >>
> >> >> Sorry I missed the new sink API. I do still think that it would be
> >> >> better to make the package name more informative, and ".v2." does not
> >> >> contain information for new Flink users who did not know the v1 of
> >> >> state API. Unlike internal implementation and performance
> >> >> optimization, API will hardly be compromised for now and updated in
> >> >> future, so I still suggest we improve the package name now if
> >> >> possible. But given the existing practice of sink v2 and
> >> >> AbstractStreamOperatorV2, the current package name would be
> acceptable
> >> >> to me if other reviewers of this FLIP agrees on it.
> >> >>
> >> >> Best,
> >> >> Yunfeng
> >> >>
> >> >> On Mon, Mar 11, 2024 at 5:27 PM Zakelly Lan <zakelly....@gmail.com>
> >> wrote:
> >> >> >
> >> >> > Hi Yunfeng,
> >> >> >
> >> >> > Thanks for your comments!
> >> >> >
> >> >> > +1 for JingGe's suggestion to introduce an AsyncState API, instead
> of
> >> >> > > having both get() and asyncGet() in the same State class. As a
> >> >> > > supplement to its benefits, this design could help avoid having
> >> users
> >> >> > > to use sync and async API in a mixed way (unless they create
> both a
> >> >> > > State and an AsyncState from the same state descriptor), which is
> >> >> > > supposed to bring suboptimal performance according to the FLIP's
> >> >> > > description.
> >> >> >
> >> >> >
> >> >> > Actually splitting APIs into two sets of classes also brings some
> >> >> > difficulties. In this case, users must explicitly define their
> usage
> >> before
> >> >> > actually doing state access. It is a little strange that the user
> can
> >> >> > define a sync and an async version of State with the same name,
> >> while they
> >> >> > cannot allocate two async States with the same name.
> >> >> > Another reason for distinguishing API by their method name instead
> >> of class
> >> >> > name is that users typically use the State instances to access
> state
> >> but
> >> >> > forget their type/class. For example:
> >> >> > ```
> >> >> > SyncState a = getState(xxx);
> >> >> > AsyncState b = getAsyncState(xxx);
> >> >> > //...
> >> >> > a.update(1);
> >> >> > b.update(1);
> >> >> > ```
> >> >> > Users are likely to think there is no difference between the
> >> `a.update(1)`
> >> >> > and `b.update(1)`, since they may forget the type for `a` and `b`.
> >> Thus I
> >> >> > proposed to distinguish the behavior in method names.
> >> >> > As for the suboptimal performance with mixed usage of sync and
> >> async, my
> >> >> > proposal is to warn them in runtime.
> >> >> >
> >> >> > I noticed that the FLIP proposes to place the newly introduced API
> in
> >> >> > > the package "org.apache.flink.api.common.state.v2", which seems a
> >> >> > > little strange to me as there has not been such a naming pattern
> >> >> > > ".v2." for packages in Flink.
> >> >> >
> >> >> >
> >> >> > In fact, there are some similar existing patterns, like
> >> >> > `org.apache.flink.streaming.api.functions.sink.v2` and
> >> >> > `org.apache.flink.streaming.api.connector.sink2`.
> >> >> >
> >> >> >  I would suggest discussing this topic
> >> >> > > with the main authors of Datastream V2, like Weijie Guo, so that
> >> the
> >> >> > > newly introduced APIs from both sides comply with a unified
> naming
> >> >> > > style.
> >> >> >
> >> >> > I'm afraid we are facing a different situation with the Datastream
> >> V2. For
> >> >> > total reconstruction of Datastream API, it is big enough to build a
> >> >> > seperate module and keep good package names. While for state APIs,
> we
> >> >> > should stay in the flink-core(-api) module alongside with other
> >> >> > apis, currently I tend to compromise at the expense of naming
> style.
> >> >> >
> >> >> >
> >> >> > Looking forward to hearing from you again!
> >> >> >
> >> >> > Thanks & Best,
> >> >> > Zakelly
> >> >> >
> >> >> > On Mon, Mar 11, 2024 at 4:20 PM Yunfeng Zhou <
> >> flink.zhouyunf...@gmail.com>
> >> >> > wrote:
> >> >> >
> >> >> > > Hi Zakelly,
> >> >> > >
> >> >> > > Thanks for the proposal! The structure of the Async API generally
> >> >> > > looks good to me. Some comments on the details of the design are
> as
> >> >> > > follows.
> >> >> > >
> >> >> > > +1 for JingGe's suggestion to introduce an AsyncState API,
> instead
> >> of
> >> >> > > having both get() and asyncGet() in the same State class. As a
> >> >> > > supplement to its benefits, this design could help avoid having
> >> users
> >> >> > > to use sync and async API in a mixed way (unless they create
> both a
> >> >> > > State and an AsyncState from the same state descriptor), which is
> >> >> > > supposed to bring suboptimal performance according to the FLIP's
> >> >> > > description.
> >> >> > >
> >> >> > > I noticed that the FLIP proposes to place the newly introduced
> API
> >> in
> >> >> > > the package "org.apache.flink.api.common.state.v2", which seems a
> >> >> > > little strange to me as there has not been such a naming pattern
> >> >> > > ".v2." for packages in Flink. I would suggest discussing this
> topic
> >> >> > > with the main authors of Datastream V2, like Weijie Guo, so that
> >> the
> >> >> > > newly introduced APIs from both sides comply with a unified
> naming
> >> >> > > style. If we reach an agreement on the first comment, my personal
> >> idea
> >> >> > > is that we can place the AsyncState interfaces to
> >> >> > > "org.apache.flink.api.common.state.async", and the existing state
> >> APIs
> >> >> > > to "org.apache.flink.api.common.state" or
> >> >> > > "org.apache.flink.api.common.state.sync".
> >> >> > >
> >> >> > > Best regards,
> >> >> > > Yunfeng Zhou
> >> >> > >
> >> >> > > On Thu, Mar 7, 2024 at 4:48 PM Zakelly Lan <
> zakelly....@gmail.com>
> >> wrote:
> >> >> > > >
> >> >> > > > Hi devs,
> >> >> > > >
> >> >> > > > I'd like to start a discussion on a sub-FLIP of FLIP-423:
> >> Disaggregated
> >> >> > > > State Storage and Management[1], which is a joint work of Yuan
> >> Mei,
> >> >> > > Zakelly
> >> >> > > > Lan, Jinzhong Li, Hangxiang Yu, Yanfei Lei and Feng Wang:
> >> >> > > >
> >> >> > > >  - FLIP-424: Asynchronous State APIs [2]
> >> >> > > >
> >> >> > > > This FLIP introduces new APIs for asynchronous state access.
> >> >> > > >
> >> >> > > > Please make sure you have read the FLIP-423[1] to know the
> whole
> >> story,
> >> >> > > and
> >> >> > > > we'll discuss the details of FLIP-424[2] under this mail. For
> the
> >> >> > > > discussion of overall architecture or topics related with
> >> multiple
> >> >> > > > sub-FLIPs, please post in the previous mail[3].
> >> >> > > >
> >> >> > > > Looking forward to hearing from you!
> >> >> > > >
> >> >> > > > [1] https://cwiki.apache.org/confluence/x/R4p3EQ
> >> >> > > > [2] https://cwiki.apache.org/confluence/x/SYp3EQ
> >> >> > > > [3]
> >> https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0
> >> >> > > >
> >> >> > > >
> >> >> > > > Best,
> >> >> > > > Zakelly
> >> >> > >
> >>
> >
>


-- 
Best,
Yue

Reply via email to