Re: [DISCUSS] FLIP-459: Support Flink hybrid shuffle integration with Apache Celeborn

2024-06-05 Thread Junrui Lee
Thanks Yuxin for your answer. +1 for this proposal.

Best,
Junrui.

Yuxin Tan  于2024年6月6日周四 13:42写道:

> Thanks Junrui for your question.
>
> > I wonder if the current interface design support the
> future adaptation for batch job recovery
>
> I noticed that FLIP-383 supports batch job recovery by introducing
> some new APIs. These APIs can also be added to the Tier-related
> interfaces to facilitate the feature. Since these modifications are not
> directly related to the current integration tasks and the integration
> does not conflict with the batch job recovery, I propose that this FLIP
> doesn't involve these particular changes. Moreover, considering that
> the Tier interfaces are not public currently, it is also feasible to add
> the interfaces directly if necessary.
> WDYT?
>
> Best,
> Yuxin
>
>
> Junrui Lee  于2024年6月6日周四 11:02写道:
>
> > Thanks Yuxin for driving this proposal!
> >
> > I have a question about the public interface compatibility in the context
> > of FLIP-459. As we've supported batch job recovery from jobMaster
> failures
> > in FLIP-383 which will be released in Flink 1.20. I wonder if the current
> > interface design support the future adaptation for batch job recovery?
> >
> > Looking forward to your feedback.
> >
> > Best,
> > Junrui.
> >
> > weijie guo  于2024年6月5日周三 10:13写道:
> >
> > > Thanks Yuxin for the proposal!
> > >
> > > When we first proposed Hybrid Shuffle, I wanted to support pluggable
> > > storage tier in the future. However, limited by the architecture of the
> > > legacy Hybrid Shuffle at that time, this idea has not been realized.
> The
> > > new architecture abstracts the tier nicely, and now it's time to
> > introduce
> > > support for external storage.
> > >
> > > Big +1 for this one!
> > >
> > > Best regards,
> > >
> > > Weijie
> > >
> > >
> > > rexxiong  于2024年6月5日周三 00:08写道:
> > >
> > > > Thanks Yuxin for the proposal. +1,  as a member of the Apache
> Celeborn
> > > > community, I am very excited about the integration of Flink's Hybrid
> > > > Shuffle with Apache Celeborn. The whole design of CIP-6 looks good to
> > > me. I
> > > > am looking forward to this integration.
> > > >
> > > > Thanks,
> > > > Jiashu Xiong
> > > >
> > > > Ethan Feng  于2024年6月4日周二 16:47写道:
> > > >
> > > > > +1 for this proposal.
> > > > >
> > > > > After internally reviewing the prototype of CIP-6, this would
> improve
> > > > > performance and stability for Flink users using Celeborn.
> > > > >
> > > > > Expect to see this feature come out to the community.
> > > > >
> > > > > As I come from the Celeborn community, I hope more users can try to
> > > > > use Celeborn when there are Flink batch jobs.
> > > > >
> > > > > Thanks,
> > > > > Ethan Feng
> > > > >
> > > > > Yuxin Tan  于2024年6月4日周二 16:34写道:
> > > > > >
> > > > > > Hi, Venkatakrishnan,
> > > > > >
> > > > > > Thanks for joining the discussion. We appreciate your interest
> > > > > > in contributing to the work. Once the FLIP and CIP proposals
> > > > > > have been approved, we will create some JIRA tickets in Flink
> > > > > > and Celeborn projects. Please feel free to take a look at the
> > > > > > tickets and select any that resonate with your interests.
> > > > > >
> > > > > > Best,
> > > > > > Yuxin
> > > > > >
> > > > > >
> > > > > > Venkatakrishnan Sowrirajan  于2024年5月31日周五
> > 23:11写道:
> > > > > >
> > > > > > > Thanks for this FLIP. We are also interested in
> > > learning/contributing
> > > > > to
> > > > > > > the hybrid shuffle integration with celeborn for batch
> > executions.
> > > > > > >
> > > > > > > On Tue, May 28, 2024, 7:07 PM Yuxin Tan <
> tanyuxinw...@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > Hi, Xintong,
> > > > > > > >
> > > > > > > > >  I think we can also publish the prototype codes so the
> > > > > > > > c

Re: [ANNOUNCE] New Apache Flink PMC Member - Fan Rui

2024-06-05 Thread Junrui Lee
Congratulations, Rui.

Best,
Junrui

Hang Ruan  于2024年6月6日周四 10:35写道:

> Congratulations, Rui!
>
> Best,
> Hang
>
> Samrat Deb  于2024年6月6日周四 10:28写道:
>
> > Congratulations Rui
> >
> > Bests,
> > Samrat
> >
> > On Thu, 6 Jun 2024 at 7:45 AM, Yuxin Tan  wrote:
> >
> > > Congratulations, Rui!
> > >
> > > Best,
> > > Yuxin
> > >
> > >
> > > Xuannan Su  于2024年6月6日周四 09:58写道:
> > >
> > > > Congratulations!
> > > >
> > > > Best regards,
> > > > Xuannan
> > > >
> > > > On Thu, Jun 6, 2024 at 9:53 AM Hangxiang Yu 
> > wrote:
> > > > >
> > > > > Congratulations, Rui !
> > > > >
> > > > > On Thu, Jun 6, 2024 at 9:18 AM Lincoln Lee  >
> > > > wrote:
> > > > >
> > > > > > Congratulations, Rui!
> > > > > >
> > > > > > Best,
> > > > > > Lincoln Lee
> > > > > >
> > > > > >
> > > > > > Lijie Wang  于2024年6月6日周四 09:11写道:
> > > > > >
> > > > > > > Congratulations, Rui!
> > > > > > >
> > > > > > > Best,
> > > > > > > Lijie
> > > > > > >
> > > > > > > Rodrigo Meneses  于2024年6月5日周三 21:35写道:
> > > > > > >
> > > > > > > > All the best
> > > > > > > >
> > > > > > > > On Wed, Jun 5, 2024 at 5:56 AM xiangyu feng <
> > > xiangyu...@gmail.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Congratulations, Rui!
> > > > > > > > >
> > > > > > > > > Regards,
> > > > > > > > > Xiangyu Feng
> > > > > > > > >
> > > > > > > > > Feng Jin  于2024年6月5日周三 20:42写道:
> > > > > > > > >
> > > > > > > > > > Congratulations, Rui!
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Feng Jin
> > > > > > > > > >
> > > > > > > > > > On Wed, Jun 5, 2024 at 8:23 PM Yanfei Lei <
> > > fredia...@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Congratulations, Rui!
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Yanfei
> > > > > > > > > > >
> > > > > > > > > > > Luke Chen  于2024年6月5日周三 20:08写道:
> > > > > > > > > > > >
> > > > > > > > > > > > Congrats, Rui!
> > > > > > > > > > > >
> > > > > > > > > > > > Luke
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Jun 5, 2024 at 8:02 PM Jiabao Sun <
> > > > > > jiabao...@apache.org>
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Congrats, Rui. Well-deserved!
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Jiabao
> > > > > > > > > > > > >
> > > > > > > > > > > > > Zhanghao Chen 
> > 于2024年6月5日周三
> > > > > > > 19:29写道:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Congrats, Rui!
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Zhanghao Chen
> > > > > > > > > > > > > > 
> > > > > > > > > > > > > > From: Piotr Nowojski 
> > > > > > > > > > > > > > Sent: Wednesday, June 5, 2024 18:01
> > > > > > > > > > > > > > To: dev ; rui fan <
> > > > > > > 1996fan...@gmail.com>
> > > > > > > > > > > > > > Subject: [ANNOUNCE] New Apache Flink PMC Member -
> > Fan
> > > > Rui
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi everyone,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On behalf of the PMC, I'm very happy to announce
> > > > another
> > > > > > new
> > > > > > > > > Apache
> > > > > > > > > > > Flink
> > > > > > > > > > > > > > PMC Member - Fan Rui.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Rui has been active in the community since August
> > > 2019.
> > > > > > > During
> > > > > > > > > this
> > > > > > > > > > > time
> > > > > > > > > > > > > he
> > > > > > > > > > > > > > has contributed a lot of new features. Among
> > others:
> > > > > > > > > > > > > >   - Decoupling Autoscaler from Kubernetes
> Operator,
> > > and
> > > > > > > > > supporting
> > > > > > > > > > > > > > Standalone Autoscaler
> > > > > > > > > > > > > >   - Improvements to checkpointing, flamegraphs,
> > > restart
> > > > > > > > > strategies,
> > > > > > > > > > > > > > watermark alignment, network shuffles
> > > > > > > > > > > > > >   - Optimizing the memory and CPU usage of large
> > > > operators,
> > > > > > > > > greatly
> > > > > > > > > > > > > > reducing the risk and probability of TaskManager
> > OOM
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > He reviewed a significant amount of PRs and has
> > been
> > > > active
> > > > > > > > both
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > > > mailing lists and in Jira helping to both
> maintain
> > > and
> > > > grow
> > > > > > > > > Apache
> > > > > > > > > > > > > Flink's
> > > > > > > > > > > > > > community. He is also our current Flink 1.20
> > release
> > > > > > manager.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > In the last 12 months, Rui has been the most
> active
> > > > > > > contributor
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > Flink Kubernetes Operator project, while being
> the
> > > 2nd
> > > > most
> > > > > > > > > active
> > > > > > > > > > > Flink
> > > > > > > > > > > > > > contributor at the same time.
> > > > > > > 

Re: [DISCUSS] FLIP-459: Support Flink hybrid shuffle integration with Apache Celeborn

2024-06-05 Thread Junrui Lee
Thanks Yuxin for driving this proposal!

I have a question about the public interface compatibility in the context
of FLIP-459. As we've supported batch job recovery from jobMaster failures
in FLIP-383 which will be released in Flink 1.20. I wonder if the current
interface design support the future adaptation for batch job recovery?

Looking forward to your feedback.

Best,
Junrui.

weijie guo  于2024年6月5日周三 10:13写道:

> Thanks Yuxin for the proposal!
>
> When we first proposed Hybrid Shuffle, I wanted to support pluggable
> storage tier in the future. However, limited by the architecture of the
> legacy Hybrid Shuffle at that time, this idea has not been realized. The
> new architecture abstracts the tier nicely, and now it's time to introduce
> support for external storage.
>
> Big +1 for this one!
>
> Best regards,
>
> Weijie
>
>
> rexxiong  于2024年6月5日周三 00:08写道:
>
> > Thanks Yuxin for the proposal. +1,  as a member of the Apache Celeborn
> > community, I am very excited about the integration of Flink's Hybrid
> > Shuffle with Apache Celeborn. The whole design of CIP-6 looks good to
> me. I
> > am looking forward to this integration.
> >
> > Thanks,
> > Jiashu Xiong
> >
> > Ethan Feng  于2024年6月4日周二 16:47写道:
> >
> > > +1 for this proposal.
> > >
> > > After internally reviewing the prototype of CIP-6, this would improve
> > > performance and stability for Flink users using Celeborn.
> > >
> > > Expect to see this feature come out to the community.
> > >
> > > As I come from the Celeborn community, I hope more users can try to
> > > use Celeborn when there are Flink batch jobs.
> > >
> > > Thanks,
> > > Ethan Feng
> > >
> > > Yuxin Tan  于2024年6月4日周二 16:34写道:
> > > >
> > > > Hi, Venkatakrishnan,
> > > >
> > > > Thanks for joining the discussion. We appreciate your interest
> > > > in contributing to the work. Once the FLIP and CIP proposals
> > > > have been approved, we will create some JIRA tickets in Flink
> > > > and Celeborn projects. Please feel free to take a look at the
> > > > tickets and select any that resonate with your interests.
> > > >
> > > > Best,
> > > > Yuxin
> > > >
> > > >
> > > > Venkatakrishnan Sowrirajan  于2024年5月31日周五 23:11写道:
> > > >
> > > > > Thanks for this FLIP. We are also interested in
> learning/contributing
> > > to
> > > > > the hybrid shuffle integration with celeborn for batch executions.
> > > > >
> > > > > On Tue, May 28, 2024, 7:07 PM Yuxin Tan 
> > > wrote:
> > > > >
> > > > > > Hi, Xintong,
> > > > > >
> > > > > > >  I think we can also publish the prototype codes so the
> > > > > > community can better understand and help with it.
> > > > > >
> > > > > > Ok, I agree on the point. I will prepare and publish the code
> > > > > > recently.
> > > > > >
> > > > > > Rui,
> > > > > >
> > > > > > > Kindly reminder: the image of CIP-6[1] cannot be loaded.
> > > > > >
> > > > > > Thanks for the reminder. I've updated the images.
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > > Yuxin
> > > > > >
> > > > > >
> > > > > > Rui Fan <1996fan...@gmail.com> 于2024年5月29日周三 09:33写道:
> > > > > >
> > > > > > > Thanks Yuxin for driving this proposal!
> > > > > > >
> > > > > > > Kindly reminder: the image of CIP-6[1] cannot be loaded.
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> >
> https://urldefense.com/v3/__https://cwiki.apache.org/confluence/display/CELEBORN/CIP-6*Support*Flink*hybrid*shuffle*integration*with*Apache*Celeborn__;KysrKysrKys!!IKRxdwAv5BmarQ!ZRTc1aUSYMDBazuIwlet1Dzk2_DD9qKTgoDLH9jSwAVLgwplcuId_8JoXkH0i7AeWxKWXkL0sxM3AeW-H9OJ6v9uGw$
> > > > > > >
> > > > > > > Best,
> > > > > > > Rui
> > > > > > >
> > > > > > > On Wed, May 29, 2024 at 9:03 AM Xintong Song <
> > > tonysong...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > +1 for this proposal.
> > > > > > > >
> > > > > > > > We have been prototyping this feature internally at Alibaba
> > for a
> > > > > > couple
> > > > > > > of
> > > > > > > > months. Yuxin, I think we can also publish the prototype
> codes
> > > so the
> > > > > > > > community can better understand and help with it.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > >
> > > > > > > > Xintong
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, May 28, 2024 at 8:34 PM Yuxin Tan <
> > > tanyuxinw...@gmail.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi all,
> > > > > > > > >
> > > > > > > > > I would like to start a discussion on FLIP-459 Support
> Flink
> > > hybrid
> > > > > > > > shuffle
> > > > > > > > > integration with
> > > > > > > > > Apache Celeborn[1]. Flink hybrid shuffle supports
> transitions
> > > > > between
> > > > > > > > > memory, disk, and
> > > > > > > > > remote storage to improve performance and job stability.
> > > > > > Concurrently,
> > > > > > > > > Apache Celeborn
> > > > > > > > > provides a stable, performant, scalable remote shuffle
> > service.
> > > > > This
> > > > > > > > > integration proposal is to
> > > > > > > > > harness 

Re: [ANNOUNCE] New Apache Flink PMC Member - Weijie Guo

2024-06-04 Thread Junrui Lee
Congratulations Weijie!

Best,
Junrui

Xintong Song  于2024年6月4日周二 14:46写道:

> Hi everyone,
>
> On behalf of the PMC, I'm very happy to announce that Weijie Guo has joined
> the Flink PMC!
>
> Weijie has been an active member of the Apache Flink community for many
> years. He has made significant contributions in many components, including
> runtime, shuffle, sdk, connectors, etc. He has driven / participated in
> many FLIPs, authored and reviewed hundreds of PRs, been consistently active
> on mailing lists, and also helped with release management of 1.20 and
> several other bugfix releases.
>
> Congratulations and welcome Weijie!
>
> Best,
>
> Xintong (on behalf of the Flink PMC)
>


Re: [DISCUSSION] FLIP-450: Improve Runtime Configuration for Flink 2.0

2024-05-10 Thread Junrui Lee
Thanks Xuannan for driving this! +1 for this proposal.

Best,
Junrui

Rui Fan <1996fan...@gmail.com> 于2024年5月6日周一 11:27写道:

> Thanks Xuannan for driving this proposal!
>
> > taskmanager.network.memory.max-overdraft-buffers-per-gate will be removed
> and hard-coded to either 10 or 20.
>
> Currently, it's a public option. Could we determine the value of
> the overdraft buffer in the current FLIP?
>
> I vote 20 as the hard code value due to 2 reasons:
> - Removing this option means users cannot change it, it might be better to
> turn it up.
> - Most of tasks don't use the overdraft buffer, so increasing it doesn't
> introduce more risk.
>
> Best,
> Rui
>
> On Mon, May 6, 2024 at 10:47 AM Yuxin Tan  wrote:
>
> > Thanks for the effort, Xuannan.
> >
> > +1 for the proposal.
> >
> > Best,
> > Yuxin
> >
> >
> > Xintong Song  于2024年4月29日周一 15:40写道:
> >
> > > Thanks for driving this effort, Xuannan.
> > >
> > > +1 for the proposed changes.
> > >
> > > Just one suggestion: Some of the proposed changes involve not solely
> > > changing the configuration options, but are bound to changing / removal
> > of
> > > certain features. E.g., the removal of hash-blocking shuffle and legacy
> > > hybrid shuffle mode, and the behavior change of overdraft network
> > buffers.
> > > Therefore, it might be nicer to provide an implementation plan with a
> > list
> > > of related tasks in the FLIP. This should not block the FLIP though.
> > >
> > > Best,
> > >
> > > Xintong
> > >
> > >
> > >
> > > On Thu, Apr 25, 2024 at 4:35 PM Xuannan Su 
> > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I'd like to start a discussion on FLIP-450: Improve Runtime
> > > > Configuration for Flink 2.0 [1]. As Flink moves toward 2.0, we have
> > > > revisited all runtime configurations and identified several
> > > > improvements to enhance user-friendliness and maintainability. In
> this
> > > > FLIP, we aim to refine the runtime configuration.
> > > >
> > > > Looking forward to everyone's feedback and suggestions. Thank you!
> > > >
> > > > Best regards,
> > > > Xuannan
> > > >
> > > > [1]
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-450%3A+Improve+Runtime+Configuration+for+Flink+2.0
> > > >
> > >
> >
>


Re: Question around Flink's AdaptiveBatchScheduler

2024-05-06 Thread Junrui Lee
Hi,
Thanks for the reminder. I will review it soon during my free time.

Venkatakrishnan Sowrirajan  于2024年5月4日周六 10:10写道:

> Jinrui and Xia
>
> Gentle ping for reviews.
>
> On Mon, Apr 29, 2024, 8:28 PM Venkatakrishnan Sowrirajan  >
> wrote:
>
> > Hi Xia and Jinrui,
> >
> > Filed https://github.com/apache/flink/pull/24736 to address the above
> > described issue. Please take a look whenever you can.
> >
> > Thanks
> > Venkat
> >
> >
> > On Thu, Apr 18, 2024 at 12:16 PM Venkatakrishnan Sowrirajan <
> > vsowr...@asu.edu> wrote:
> >
> >> Filed https://issues.apache.org/jira/browse/FLINK-35165 to address the
> >> above described issue. Will share the PR here once it is ready for
> review.
> >>
> >> Regards
> >> Venkata krishnan
> >>
> >>
> >> On Wed, Apr 17, 2024 at 5:32 AM Junrui Lee  wrote:
> >>
> >>> Thanks Venkata and Xia for providing further clarification. I think
> your
> >>> example illustrates the significance of this proposal very well. Please
> >>> feel free go ahead and address the concerns.
> >>>
> >>> Best,
> >>> Junrui
> >>>
> >>> Venkatakrishnan Sowrirajan  于2024年4月16日周二 07:01写道:
> >>>
> >>> > Thanks for adding your thoughts to this discussion.
> >>> >
> >>> > If we all agree that the source vertex parallelism shouldn't be bound
> >>> by
> >>> > the downstream max parallelism
> >>> > (jobmanager.adaptive-batch-scheduler.max-parallelism)
> >>> > based on the rationale and the issues described above, I can take a
> >>> stab at
> >>> > addressing the issue.
> >>> >
> >>> > Let me file a ticket to track this issue. Otherwise, I'm looking
> >>> forward to
> >>> > hearing more thoughts from others as well, especially Lijie and
> Junrui
> >>> who
> >>> > have more context on the AdaptiveBatchScheduler.
> >>> >
> >>> > Regards
> >>> > Venkata krishnan
> >>> >
> >>> >
> >>> > On Mon, Apr 15, 2024 at 12:54 AM Xia Sun 
> wrote:
> >>> >
> >>> > > Hi Venkat,
> >>> > > I agree that the parallelism of source vertex should not be upper
> >>> bounded
> >>> > > by the job's global max parallelism. The case you mentioned, >>
> High
> >>> > filter
> >>> > > selectivity with huge amounts of data to read  excellently supports
> >>> this
> >>> > > viewpoint. (In fact, in the current implementation, if the source
> >>> > > parallelism is pre-specified at job create stage, rather than
> >>> relying on
> >>> > > the dynamic parallelism inference of the AdaptiveBatchScheduler,
> the
> >>> > source
> >>> > > vertex's parallelism can indeed exceed the job's global max
> >>> parallelism.)
> >>> > >
> >>> > > As Lijie and Junrui pointed out, the key issue is "semantic
> >>> consistency."
> >>> > > Currently, if a vertex has not set maxParallelism, the
> >>> > > AdaptiveBatchScheduler will use
> >>> > > `execution.batch.adaptive.auto-parallelism.max-parallelism` as the
> >>> > vertex's
> >>> > > maxParallelism. Since the current implementation does not
> distinguish
> >>> > > between source vertices and downstream vertices, source vertices
> are
> >>> also
> >>> > > subject to this limitation.
> >>> > >
> >>> > > Therefore, I believe that if the issue of "semantic consistency"
> can
> >>> be
> >>> > > well explained in the code and configuration documentation, the
> >>> > > AdaptiveBatchScheduler should support that the parallelism of
> source
> >>> > > vertices can exceed the job's global max parallelism.
> >>> > >
> >>> > > Best,
> >>> > > Xia
> >>> > >
> >>> > > Venkatakrishnan Sowrirajan  于2024年4月14日周日
> 10:31写道:
> >>> > >
> >>> > > > Let me state why I think "*jobmanager.adaptive-batch-sche*
> >>> > > > *duler.default-source-parallelism*" should not be bound by the "
> >>> > > > *jo

Re: Question around Flink's AdaptiveBatchScheduler

2024-04-17 Thread Junrui Lee
Thanks Venkata and Xia for providing further clarification. I think your
example illustrates the significance of this proposal very well. Please
feel free go ahead and address the concerns.

Best,
Junrui

Venkatakrishnan Sowrirajan  于2024年4月16日周二 07:01写道:

> Thanks for adding your thoughts to this discussion.
>
> If we all agree that the source vertex parallelism shouldn't be bound by
> the downstream max parallelism
> (jobmanager.adaptive-batch-scheduler.max-parallelism)
> based on the rationale and the issues described above, I can take a stab at
> addressing the issue.
>
> Let me file a ticket to track this issue. Otherwise, I'm looking forward to
> hearing more thoughts from others as well, especially Lijie and Junrui who
> have more context on the AdaptiveBatchScheduler.
>
> Regards
> Venkata krishnan
>
>
> On Mon, Apr 15, 2024 at 12:54 AM Xia Sun  wrote:
>
> > Hi Venkat,
> > I agree that the parallelism of source vertex should not be upper bounded
> > by the job's global max parallelism. The case you mentioned, >> High
> filter
> > selectivity with huge amounts of data to read  excellently supports this
> > viewpoint. (In fact, in the current implementation, if the source
> > parallelism is pre-specified at job create stage, rather than relying on
> > the dynamic parallelism inference of the AdaptiveBatchScheduler, the
> source
> > vertex's parallelism can indeed exceed the job's global max parallelism.)
> >
> > As Lijie and Junrui pointed out, the key issue is "semantic consistency."
> > Currently, if a vertex has not set maxParallelism, the
> > AdaptiveBatchScheduler will use
> > `execution.batch.adaptive.auto-parallelism.max-parallelism` as the
> vertex's
> > maxParallelism. Since the current implementation does not distinguish
> > between source vertices and downstream vertices, source vertices are also
> > subject to this limitation.
> >
> > Therefore, I believe that if the issue of "semantic consistency" can be
> > well explained in the code and configuration documentation, the
> > AdaptiveBatchScheduler should support that the parallelism of source
> > vertices can exceed the job's global max parallelism.
> >
> > Best,
> > Xia
> >
> > Venkatakrishnan Sowrirajan  于2024年4月14日周日 10:31写道:
> >
> > > Let me state why I think "*jobmanager.adaptive-batch-sche*
> > > *duler.default-source-parallelism*" should not be bound by the "
> > > *jobmanager.adaptive-batch-sche**duler.max-parallelism*".
> > >
> > >- Source vertex is unique and does not have any upstream vertices
> > >- Downstream vertices read shuffled data partitioned by key, which
> is
> > >not the case for the Source vertex
> > >- Limiting source parallelism by downstream vertices' max
> parallelism
> > is
> > >incorrect
> > >
> > > If we say for ""semantic consistency" the source vertex parallelism has
> > to
> > > be bound by the overall job's max parallelism, it can lead to following
> > > issues:
> > >
> > >- High filter selectivity with huge amounts of data to read -
> setting
> > >high "*jobmanager.adaptive-batch-scheduler.max-parallelism*" so that
> > >    source parallelism can be set higher can lead to small blocks and
> > >sub-optimal performance.
> > >- Setting high
> "*jobmanager.adaptive-batch-scheduler.max-parallelism*"
> > >requires careful tuning of network buffer configurations which is
> > >unnecessary in cases where it is not required just so that the
> source
> > >parallelism can be set high.
> > >
> > > Regards
> > > Venkata krishnan
> > >
> > > On Thu, Apr 11, 2024 at 9:30 PM Junrui Lee 
> wrote:
> > >
> > > > Hello Venkata krishnan,
> > > >
> > > > I think the term "semantic inconsistency" defined by
> > > > jobmanager.adaptive-batch-scheduler.max-parallelism refers to
> > > maintaining a
> > > > uniform upper limit on parallelism across all vertices within a job.
> As
> > > the
> > > > source vertices are part of the global execution graph, they should
> > also
> > > > respect this rule to ensure consistent application of parallelism
> > > > constraints.
> > > >
> > > > Best,
> > > > Junrui
> > > >
> > > > Venkatakrishnan Sowrirajan  于2024年4月12日周五 02:10写道:
> > > >
> > > > > 

Re: [ANNOUNCE] New Apache Flink Committer - Zakelly Lan

2024-04-14 Thread Junrui Lee
Congratulations, Zakelly!

Best,
Junrui

yue ma  于2024年4月15日周一 12:00写道:

> Congratulations Zakelly!
>
> Best,
> Yue
>


Re: [ANNOUNCE] New Apache Flink PMC Member - Lincoln Lee

2024-04-12 Thread Junrui Lee
Congratulations, Lincoln!

Best,
Junrui

Aleksandr Pilipenko  于2024年4月12日周五 18:29写道:

> Congratulations, Lincoln!
>
> Best Regards
> Aleksandr
>


Re: [ANNOUNCE] New Apache Flink PMC Member - Jing Ge

2024-04-12 Thread Junrui Lee
Congratulations, Jing!

Best,
Junrui

Aleksandr Pilipenko  于2024年4月12日周五 18:28写道:

> Congratulations, Jing!
>
> Best Regards,
> Aleksandr
>


Re: [VOTE] FLIP-441: Show the JobType and remove Execution Mode on Flink WebUI

2024-04-11 Thread Junrui Lee
+1 (non-binding)

Best,
Junrui

Lijie Wang  于2024年4月12日周五 12:43写道:

> +1 (binding)
>
> Thanks for driving.
>
> Best,
> Lijie
>
> Zakelly Lan  于2024年4月12日周五 11:08写道:
>
> > +1 non-binding
> >
> >
> > Best,
> > Zakelly
> >
> > On Fri, Apr 12, 2024 at 11:05 AM Yuepeng Pan 
> > wrote:
> >
> > > Hi Rui,
> > >
> > > Thanks for driving it!
> > > +1  (non-binding)
> > > Best,
> > > Yuepeng Pan
> > >
> > > At 2024-04-12 10:31:19, "Yanfei Lei"  wrote:
> > > >Hi Rui,
> > > >
> > > >Thanks for driving it!
> > > >
> > > >+1  (binding)
> > > >
> > > >Hangxiang Yu  于2024年4月12日周五 10:26写道:
> > > >>
> > > >> +1  (binding)
> > > >>
> > > >> On Fri, Apr 12, 2024 at 10:22 AM Jinzhong Li <
> > lijinzhong2...@gmail.com>
> > > >> wrote:
> > > >>
> > > >> > +1  (non binding)
> > > >> >
> > > >> > Bests,
> > > >> > Jinzhong
> > > >> >
> > > >> > On Thu, Apr 11, 2024 at 7:26 AM Muhammet Orazov
> > > >> >  wrote:
> > > >> >
> > > >> > > Hey Rui,
> > > >> > >
> > > >> > > +1 (non-binding).
> > > >> > >
> > > >> > > Thanks for driving it!
> > > >> > >
> > > >> > > Best,
> > > >> > > Muhammet
> > > >> > >
> > > >> > > On 2024-04-10 04:36, Rui Fan wrote:
> > > >> > > > Hi devs,
> > > >> > > >
> > > >> > > > Thank you to everyone for the feedback on FLIP-441: Show
> > > >> > > > the JobType and remove Execution Mode on Flink WebUI[1]
> > > >> > > > which has been discussed in this thread [2].
> > > >> > > >
> > > >> > > > I would like to start a vote for it. The vote will be open for
> > at
> > > least
> > > >> > > > 72
> > > >> > > > hours unless there is an objection or not enough votes.
> > > >> > > >
> > > >> > > > [1] https://cwiki.apache.org/confluence/x/agrPEQ
> > > >> > > > [2]
> > > https://lists.apache.org/thread/0s52w17w24x7m2zo6ogl18t1fy412vcd
> > > >> > > >
> > > >> > > > Best,
> > > >> > > > Rui
> > > >> > >
> > > >> >
> > > >>
> > > >>
> > > >> --
> > > >> Best,
> > > >> Hangxiang.
> > > >
> > > >
> > > >
> > > >--
> > > >Best,
> > > >Yanfei
> > >
> >
>


Re: Question around Flink's AdaptiveBatchScheduler

2024-04-11 Thread Junrui Lee
Hello Venkata krishnan,

I think the term "semantic inconsistency" defined by
jobmanager.adaptive-batch-scheduler.max-parallelism refers to maintaining a
uniform upper limit on parallelism across all vertices within a job. As the
source vertices are part of the global execution graph, they should also
respect this rule to ensure consistent application of parallelism
constraints.

Best,
Junrui

Venkatakrishnan Sowrirajan  于2024年4月12日周五 02:10写道:

> Gentle bump on this question. cc @Becket Qin  as
> well.
>
> Regards
> Venkata krishnan
>
>
> On Tue, Mar 12, 2024 at 10:11 PM Venkatakrishnan Sowrirajan <
> vsowr...@asu.edu> wrote:
>
> > Thanks for the response Lijie and Junrui. Sorry for the late reply. Few
> > follow up questions.
> >
> > > Source can actually ignore this limit
> > because it has no upstream, but this will lead to semantic inconsistency.
> >
> > Lijie, can you please elaborate on the above comment further? What do you
> > mean when you say it will lead to "semantic inconsistency"?
> >
> > > Secondly, we first need to limit the max parallelism of (downstream)
> > vertex, and then we can decide how many subpartitions (upstream vertex)
> > should produce. The limit should be effective, otherwise some downstream
> > tasks will have no data to process.
> >
> > This makes sense in the context of any other vertices other than the
> > source vertex. As you mentioned above ("Source can actually ignore this
> > limit because it has no upstream"), therefore I feel "
> > jobmanager.adaptive-batch-scheduler.default-source-parallelism" need not
> > be upper bounded by
> "jobmanager.adaptive-batch-scheduler.max-parallelism".
> >
> > Regards
> > Venkata krishnan
> >
> >
> > On Thu, Feb 29, 2024 at 2:11 AM Junrui Lee  wrote:
> >
> >> Hi Venkat,
> >>
> >> As Lijie mentioned,  in Flink, the parallelism is required to be less
> than
> >> or equal to the maximum parallelism. The config option
> >> jobmanager.adaptive-batch-scheduler.max-parallelism and
> >> jobmanager.adaptive-batch-scheduler.default-source-parallelism will be
> set
> >> as the source's parallelism and max-parallelism, respectively.
> Therefore,
> >> the check failed situation you encountered is in line with the
> >> expectations.
> >>
> >> Best,
> >> Junrui
> >>
> >> Lijie Wang  于2024年2月29日周四 17:35写道:
> >>
> >> > Hi Venkat,
> >> >
> >> > >> default-source-parallelism config should be independent from the
> >> > max-parallelism
> >> >
> >> > Actually, it's not.
> >> >
> >> > Firstly, it's obvious that the parallelism should be less than or
> equal
> >> to
> >> > the max parallelism(both literally and execution). The
> >> > "jobmanager.adaptive-batch-scheduler.max-parallelism" will be used as
> >> the
> >> > max parallelism for a vertex if you don't set max parallelism for it
> >> > individually (Just like the source in your case).
> >> >
> >> > Secondly, we first need to limit the max parallelism of (downstream)
> >> > vertex, and then we can decide how many subpartitions (upstream
> vertex)
> >> > should produce. The limit should be effective, otherwise some
> downstream
> >> > tasks will have no data to process. Source can actually ignore this
> >> limit
> >> > because it has no upstream, but this will lead to semantic
> >> inconsistency.
> >> >
> >> > Best,
> >> > Lijie
> >> >
> >> > Venkatakrishnan Sowrirajan  于2024年2月29日周四 05:49写道:
> >> >
> >> > > Hi Flink devs,
> >> > >
> >> > > With Flink's AdaptiveBatchScheduler
> >> > > <
> >> > >
> >> >
> >>
> https://urldefense.com/v3/__https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/elastic_scaling/*adaptive-batch-scheduler__;Iw!!IKRxdwAv5BmarQ!fwD4qP-fTEiqJH9CC3AHgXbR5MJPGm7ll1dYwElK-zrtujWDWio6_yvBa4rHlaZHP_lefLs4bZQAISrg5BrHLw$
> >> > > >
> >> > > (Note:
> >> > > this is different from AdaptiveScheduler
> >> > > <
> >> > >
> >> >
> >>
> https://urldefense.com/v3/__https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/elastic_scaling/*adaptive-scheduler__;Iw!!IKRxdwAv5BmarQ!fwD4qP-fTEiqJH9CC3AHgXbR5MJPGm7ll1dYwElK-zr

Re: Question around Flink's AdaptiveBatchScheduler

2024-02-29 Thread Junrui Lee
Hi Venkat,

As Lijie mentioned,  in Flink, the parallelism is required to be less than
or equal to the maximum parallelism. The config option
jobmanager.adaptive-batch-scheduler.max-parallelism and
jobmanager.adaptive-batch-scheduler.default-source-parallelism will be set
as the source's parallelism and max-parallelism, respectively. Therefore,
the check failed situation you encountered is in line with the expectations.

Best,
Junrui

Lijie Wang  于2024年2月29日周四 17:35写道:

> Hi Venkat,
>
> >> default-source-parallelism config should be independent from the
> max-parallelism
>
> Actually, it's not.
>
> Firstly, it's obvious that the parallelism should be less than or equal to
> the max parallelism(both literally and execution). The
> "jobmanager.adaptive-batch-scheduler.max-parallelism" will be used as the
> max parallelism for a vertex if you don't set max parallelism for it
> individually (Just like the source in your case).
>
> Secondly, we first need to limit the max parallelism of (downstream)
> vertex, and then we can decide how many subpartitions (upstream vertex)
> should produce. The limit should be effective, otherwise some downstream
> tasks will have no data to process. Source can actually ignore this limit
> because it has no upstream, but this will lead to semantic inconsistency.
>
> Best,
> Lijie
>
> Venkatakrishnan Sowrirajan  于2024年2月29日周四 05:49写道:
>
> > Hi Flink devs,
> >
> > With Flink's AdaptiveBatchScheduler
> > <
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/elastic_scaling/#adaptive-batch-scheduler
> > >
> > (Note:
> > this is different from AdaptiveScheduler
> > <
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/elastic_scaling/#adaptive-scheduler
> > >),
> > the scheduler automatically determines the correct number of downstream
> > tasks required to process the shuffle generated by the upstream vertex.
> >
> > I have a question regarding the current behavior. There are 2 configs
> which
> > are in interplay here.
> > 1. jobmanager.adaptive-batch-scheduler.default-source-parallelism
> > <
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/config/#jobmanager-adaptive-batch-scheduler-default-source-parallelism
> > >
> >  - The default parallelism of data source.
> > 2. jobmanager.adaptive-batch-scheduler.max-parallelism
> > <
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/config/#jobmanager-adaptive-batch-scheduler-max-parallelism
> > >
> > -
> > Upper bound of allowed parallelism to set adaptively.
> >
> > Currently, if "
> > jobmanager.adaptive-batch-scheduler.default-source-parallelism
> > <
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/config/#jobmanager-adaptive-batch-scheduler-default-source-parallelism
> > >"
> > is greater than "jobmanager.adaptive-batch-scheduler.max-parallelism
> > <
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/config/#jobmanager-adaptive-batch-scheduler-max-parallelism
> > >",
> > Flink application fails with the below message:
> >
> > "Vertex's parallelism should be smaller than or equal to vertex's max
> > parallelism."
> >
> > This is the corresponding code in Flink's DefaultVertexParallelismInfo
> > <
> >
> https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultVertexParallelismInfo.java#L110
> > >.
> > My question is, "default-source-parallelism" config should be independent
> > from the "max-parallelism" flag. The former controls the default source
> > parallelism while the latter controls the max number of partitions to
> write
> > the intermediate shuffle.
> >
> > If this is true, then the above check should be fixed. Otherwise, wanted
> to
> > understand why the "default-source-parallelism` should be less than the
> > "max-parallelism"
> >
> > Thanks
> > Venkat
> >
>


Re: [DISCUSS] Move CheckpointingMode to flink-core

2024-02-28 Thread Junrui Lee
Hi Zakelly,

+1 for option 1. I prefer to minimize unnecessary additional development
and discussions due to internal code relocations and to avoid imposing
migration costs on users.

Best regards,
Junrui

Zakelly Lan  于2024年2月28日周三 14:46写道:

> Hi Lincoln,
>
> Given that we have finished the testing for 1.19, I agree it is better not
> merge this into 1.19. Thanks for RMs' attention!
>
> Hi Chesney and Junrui,
>
> Thanks for your advice. My original intention is to move the class as well
> as change the package to make it clean. But it involves much more effort.
> Here are several options we have:
>
>1. Move CheckpointingMode to flink-core and keep the same package. No
>more deprecation and API changes. But it will leave a
>'org.apache.flink.streaming.api' package in flink-core.
>2. Introduce new CheckpointingMode in package
>'org.apache.flink.core.execution' and deprecate the old one. Deprecate
> the
>corresponding getter/setter of 'CheckpointConfig' and introduce new ones
>with a similar but different name (e.g. set/getCheckpointMode). We will
>discuss the removal of those deprecation later in 2.x.
>3. Based on 1, move CheckpointingMode to package
>'org.apache.flink.core.execution' in 2.0. This is a breaking change that
>needs more discussion.
>
> Both ways work. I'm slightly inclined to option 1, or option 3 if we all
> agree, since the new getter/setter may also bring in confusions thus we
> cannot make the API purely clean. WDYT?
>
>
> Best,
> Zakelly
>
> On Wed, Feb 28, 2024 at 10:14 AM Junrui Lee  wrote:
>
> > Hi Zakelly,
> >
> > I agree with Chesnay's response. I would suggest that during the process
> of
> > moving CheckpointingMode from the flink-streaming-java module to the
> > flink-core module, we should keep the package name unchanged. This
> approach
> > would be completely transparent to users. In fact, this practice should
> be
> > applicable to many of our desired moves from flink-streaming-java to
> > higher-level modules, such as flink-runtime and flink-core.
> >
> > Best,
> > Junrui
> >
> > Chesnay Schepler  于2024年2月28日周三 05:18写道:
> >
> > > Moving classes (== keep the same package) to a module higher up in the
> > > dependency tree should not be a breaking change and can imo be done
> > > anytime without any risk to users.
> > >
> > > On 27/02/2024 17:01, Lincoln Lee wrote:
> > > > Hi Zakelly,
> > > >
> > > > Thanks for letting us 1.19 RMs know about this!
> > > >
> > > > This change has been discussed during today's release sync meeting,
> we
> > > > suggest not merge it into 1.19.
> > > > We can continue discussing the removal in 2.x separately.
> > > >
> > > > Best,
> > > > Lincoln Lee
> > > >
> > > >
> > > > Hangxiang Yu  于2024年2月27日周二 11:28写道:
> > > >
> > > >> Hi, Zakelly.
> > > >> Thanks for driving this.
> > > >> Moving this class to flink-core makes sense to me which could make
> the
> > > code
> > > >> path and configs clearer.
> > > >> It's marked as @Public from 1.0 and 1.20 should be the next
> long-term
> > > >> version, so 1.19 should have been a suitable version to do it.
> > > >> And also look forward to thoughts of other developers/RMs since 1.19
> > is
> > > >> currently under a feature freeze status.
> > > >>
> > > >> On Mon, Feb 26, 2024 at 6:42 PM Zakelly Lan 
> > > wrote:
> > > >>
> > > >>> Hi devs,
> > > >>>
> > > >>> When working on the FLIP-406[1], I realized that moving all options
> > of
> > > >>> ExecutionCheckpointingOptions(flink-streaming-java) to
> > > >>> CheckpointingOptions(flink-core) depends on relocating the
> > > >>> enum CheckpointingMode(flink-streaming-java) to flink-core module.
> > > >> However,
> > > >>> the CheckpointingMode is annotated as @Public and used by
> datastream
> > > api
> > > >>> like 'CheckpointConfig#setCheckpointingMode'. So I'd like to start
> a
> > > >>> discussion on moving the CheckpointingMode to flink-core. It is in
> a
> > > >> little
> > > >>> bit of a hurry if we want the old enum to be entirely removed in
> > Flink
> > > >> 2.x
> > > >>> series, since the deprecation should be shipped in the upcoming
> Flink
> > > >> 1.19.
> > > >>> I suggest not creating a dedicated FLIP and treating this as a
> > sub-task
> > > >> of
> > > >>> FLIP-406.
> > > >>>
> > > >>> I prepared a minimal change of providing new APIs and deprecating
> the
> > > old
> > > >>> ones[2], which could be merged to 1.19 if we agree to do so.
> > > >>>
> > > >>> Looking forward to your thoughts! Also cc RMs of 1.19 about this.
> > > >>>
> > > >>> [1]
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789560
> > > >>> [2]
> > > >>>
> > > >>>
> > > >>
> > >
> >
> https://github.com/apache/flink/commit/9bdd237d0322df8853f1b9e6ae658f77b9175237
> > > >>> Best,
> > > >>> Zakelly
> > > >>>
> > > >>
> > > >> --
> > > >> Best,
> > > >> Hangxiang.
> > > >>
> > >
> > >
> >
>


Re: [DISCUSS] Move CheckpointingMode to flink-core

2024-02-27 Thread Junrui Lee
Hi Zakelly,

I agree with Chesnay's response. I would suggest that during the process of
moving CheckpointingMode from the flink-streaming-java module to the
flink-core module, we should keep the package name unchanged. This approach
would be completely transparent to users. In fact, this practice should be
applicable to many of our desired moves from flink-streaming-java to
higher-level modules, such as flink-runtime and flink-core.

Best,
Junrui

Chesnay Schepler  于2024年2月28日周三 05:18写道:

> Moving classes (== keep the same package) to a module higher up in the
> dependency tree should not be a breaking change and can imo be done
> anytime without any risk to users.
>
> On 27/02/2024 17:01, Lincoln Lee wrote:
> > Hi Zakelly,
> >
> > Thanks for letting us 1.19 RMs know about this!
> >
> > This change has been discussed during today's release sync meeting, we
> > suggest not merge it into 1.19.
> > We can continue discussing the removal in 2.x separately.
> >
> > Best,
> > Lincoln Lee
> >
> >
> > Hangxiang Yu  于2024年2月27日周二 11:28写道:
> >
> >> Hi, Zakelly.
> >> Thanks for driving this.
> >> Moving this class to flink-core makes sense to me which could make the
> code
> >> path and configs clearer.
> >> It's marked as @Public from 1.0 and 1.20 should be the next long-term
> >> version, so 1.19 should have been a suitable version to do it.
> >> And also look forward to thoughts of other developers/RMs since 1.19 is
> >> currently under a feature freeze status.
> >>
> >> On Mon, Feb 26, 2024 at 6:42 PM Zakelly Lan 
> wrote:
> >>
> >>> Hi devs,
> >>>
> >>> When working on the FLIP-406[1], I realized that moving all options of
> >>> ExecutionCheckpointingOptions(flink-streaming-java) to
> >>> CheckpointingOptions(flink-core) depends on relocating the
> >>> enum CheckpointingMode(flink-streaming-java) to flink-core module.
> >> However,
> >>> the CheckpointingMode is annotated as @Public and used by datastream
> api
> >>> like 'CheckpointConfig#setCheckpointingMode'. So I'd like to start a
> >>> discussion on moving the CheckpointingMode to flink-core. It is in a
> >> little
> >>> bit of a hurry if we want the old enum to be entirely removed in Flink
> >> 2.x
> >>> series, since the deprecation should be shipped in the upcoming Flink
> >> 1.19.
> >>> I suggest not creating a dedicated FLIP and treating this as a sub-task
> >> of
> >>> FLIP-406.
> >>>
> >>> I prepared a minimal change of providing new APIs and deprecating the
> old
> >>> ones[2], which could be merged to 1.19 if we agree to do so.
> >>>
> >>> Looking forward to your thoughts! Also cc RMs of 1.19 about this.
> >>>
> >>> [1]
> >>>
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789560
> >>> [2]
> >>>
> >>>
> >>
> https://github.com/apache/flink/commit/9bdd237d0322df8853f1b9e6ae658f77b9175237
> >>> Best,
> >>> Zakelly
> >>>
> >>
> >> --
> >> Best,
> >> Hangxiang.
> >>
>
>


Re: FW: [ANNOUNCE] New Apache Flink Committer - Alexander Fedulov

2024-01-03 Thread Junrui Lee
Congratulations, Alex!

Best,
Junrui

weijie guo  于2024年1月4日周四 09:57写道:

> Congratulations, Alex!
>
> Best regards,
>
> Weijie
>
>
> Steven Wu  于2024年1月4日周四 02:07写道:
>
> > Congra, Alex! Well deserved!
> >
> > On Wed, Jan 3, 2024 at 2:31 AM David Radley 
> > wrote:
> >
> > > Sorry for my typo.
> > >
> > > Many congratulations Alex!
> > >
> > > From: David Radley 
> > > Date: Wednesday, 3 January 2024 at 10:23
> > > To: David Anderson 
> > > Cc: dev@flink.apache.org 
> > > Subject: Re: [EXTERNAL] [ANNOUNCE] New Apache Flink Committer -
> Alexander
> > > Fedulov
> > > Many Congratulations David .
> > >
> > > From: Maximilian Michels 
> > > Date: Tuesday, 2 January 2024 at 12:16
> > > To: dev 
> > > Cc: Alexander Fedulov 
> > > Subject: [EXTERNAL] [ANNOUNCE] New Apache Flink Committer - Alexander
> > > Fedulov
> > > Happy New Year everyone,
> > >
> > > I'd like to start the year off by announcing Alexander Fedulov as a
> > > new Flink committer.
> > >
> > > Alex has been active in the Flink community since 2019. He has
> > > contributed more than 100 commits to Flink, its Kubernetes operator,
> > > and various connectors [1][2].
> > >
> > > Especially noteworthy are his contributions on deprecating and
> > > migrating the old Source API functions and test harnesses, the
> > > enhancement to flame graphs, the dynamic rescale time computation in
> > > Flink Autoscaling, as well as all the small enhancements Alex has
> > > contributed which make a huge difference.
> > >
> > > Beyond code contributions, Alex has been an active community member
> > > with his activity on the mailing lists [3][4], as well as various
> > > talks and blog posts about Apache Flink [5][6].
> > >
> > > Congratulations Alex! The Flink community is proud to have you.
> > >
> > > Best,
> > > The Flink PMC
> > >
> > > [1]
> > >
> https://github.com/search?type=commits=author%3Aafedulov+org%3Aapache
> > > [2]
> > >
> >
> https://issues.apache.org/jira/browse/FLINK-28229?jql=status%20in%20(Resolved%2C%20Closed)%20AND%20assignee%20in%20(afedulov)%20ORDER%20BY%20resolved%20DESC%2C%20created%20DESC
> > > [3]
> https://lists.apache.org/list?dev@flink.apache.org:lte=100M:Fedulov
> > > [4]
> https://lists.apache.org/list?u...@flink.apache.org:lte=100M:Fedulov
> > > [5]
> > >
> >
> https://flink.apache.org/2020/01/15/advanced-flink-application-patterns-vol.1-case-study-of-a-fraud-detection-system/
> > > [6]
> > >
> >
> https://www.ververica.com/blog/presenting-our-streaming-concepts-introduction-to-flink-video-series
> > >
> > > Unless otherwise stated above:
> > >
> > > IBM United Kingdom Limited
> > > Registered in England and Wales with number 741598
> > > Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 3AU
> > >
> >
>


Re: [DISCUSS] FLIP-406: Reorganize State & Checkpointing & Recovery Configuration

2023-12-25 Thread Junrui Lee
Hi Zakelly,

Thanks for driving this. I agree that the proposed restructuring of the
configuration options is largely positive. It will make understanding and
working with Flink configurations more intuitive.

Most of the proposed changes look great. Just a heads-up, as Rui Fan
mentioned, Flink currently requires that no configOption's key be the
prefix of another to avoid issues when we eventually adopt a standard YAML
parser, as detailed in FLINK-29372 (
https://issues.apache.org/jira/browse/FLINK-29372). Therefore, it's better
to change the key 'execution.checkpointing.local-copy' because it serves as
a prefix to the key 'execution.checkpointing.local-copy.dir'.

Best regards,
Junrui

Rui Fan <1996fan...@gmail.com> 于2023年12月25日周一 19:11写道:

> Hi Zakelly,
>
> Thank you for driving this proposal!
>
> Overall good for me. I have some questions about these names.
>
> 1. How about execution.checkpointing.storage.type instead of
> execution.checkpointing.storage?
>
> It's similar to state.backend.type.
>
> 2. How about execution.checkpointing.local-copy.enabled instead of
> execution.checkpointing.local-copy?
>
> You added a new option: execution.checkpointing.local-copy.dir.
> IIUC, one option name shouldn't be the prefix of other options.
> If you add a new option execution.checkpointing.local-copy,
> flink CI will fail directly.
>
> 3. execution.checkpointing.savepoint.dir is a little weird.
>
> For old options: state.savepoints.dir and state.checkpoints.dir,
> the savepoint and checkpoint are the same level. It means
> it's a checkpoint or savepoint.
>
> The new option execution.checkpointing.dir is fine for me.
> However, execution.checkpointing.savepoint.dir is a little weird.
> I don't know which name is better now. Let us think about it more.
>
> 4. How about execution.recovery.claim-mode instead of
> execution.recovery.mode?
>
> The meaning of mode is too broad. The claim-mode may
> be more accurate for users.
>
> WDYT?
>
> Best,
> Rui
>
> On Mon, Dec 25, 2023 at 5:14 PM Zakelly Lan  wrote:
>
> > Hi devs,
> >
> > I'd like to start a discussion on FLIP-406: Reorganize State &
> > Checkpointing & Recovery Configuration[1].
> >
> > Currently, the configuration options pertaining to checkpointing,
> recovery,
> > and state management are primarily grouped under the following prefixes:
> >
> >- state.backend.* : configurations related to state accessing and
> >checkpointing, as well as specific options for individual state
> backends
> >- execution.checkpointing.* : configurations associated with
> checkpoint
> >execution and recovery
> >- execution.savepoint.*: configurations for recovery from savepoint
> >
> > In addition, there are several individual options such as '
> > *state.checkpoint-storage*' and '*state.checkpoints.dir*' that fall
> outside
> > of these prefixes. The current arrangement of these options, which span
> > multiple modules, is somewhat haphazard and lacks a systematic structure.
> > For example, the options under the '*CheckpointingOptions*' and '
> > *ExecutionCheckpointingOptions*' are related and have no clear boundaries
> > from the user's perspective, but there is no unified prefix for them.
> With
> > the upcoming release of Flink 2.0, we have an excellent opportunity to
> > overhaul and restructure the configurations related to checkpointing,
> > recovery, and state management. This FLIP proposes to reorganize these
> > settings, making it more coherent by module, which would significantly
> > lower the barriers for understanding and reduce the development costs
> > moving forward.
> >
> > Looking forward to hearing from you!
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789560
> >
> > Best,
> > Zakelly
> >
>


[RESULT][VOTE] FLIP-391: Deprecate RuntimeContext#getExecutionConfig

2023-11-30 Thread Junrui Lee
The voting time of FLIP-391: Deprecate RuntimeContext#getExecutionConfig[1]
has passed. I'm closing the vote now.

There were 5 +1 votes, 4 of which are binding:

Rui Fan (binding)
Weijie Guo (binding)
Jing Ge (binding)
Zhu Zhu (binding)
Zhanghao Chen (non-binding)


There were no -1 votes.

Thus FLIP-391 has been accepted.

Thanks everyone for joining the discussion and giving feedback!

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278465937


Best,
Junrui


[VOTE] FLIP-391: Deprecate RuntimeContext#getExecutionConfig

2023-11-27 Thread Junrui Lee
Hi everyone,

Thank you to everyone for the feedback on FLIP-391: Deprecate
RuntimeContext#getExecutionConfig[1] which has been discussed in this
thread [2].

I would like to start a vote for it. The vote will be open for at least 72
hours unless there is an objection or not enough votes.

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278465937
[2]https://lists.apache.org/thread/cv4x3372g5txw1j20r5l1vwlqrvjoqq5


Re: [DISCUSS] FLIP-391: Deprecate RuntimeContext#getExecutionConfig

2023-11-19 Thread Junrui Lee
Hi Jing,

Thank you for your feedback. I understand your concerns regarding putting
all methods into the RuntimeContext flat.

I would like to share some of my thoughts on this matter.
Firstly, this FLIP only proposes the addition of three additional methods,
which should not impose too much extra burden on users. Secondly, I agree
that it is important to make it clearer for users to use the
RuntimeContext. However, reorganizing the RuntimeContext to achieve this
requires further discussion. We should focus on a more specific and unified
reorganization of the RuntimeContext interface in future work, rather than
implementing a temporary solution now. Therefore, I prefer not to add a
separate abstraction layer for these three methods in this FLIP.

Please feel free to share any further thoughts.

Best regards,
Junrui

Jing Ge  于2023年11月20日周一 05:46写道:

> Hi Junrui,
>
> Thanks for bringing this to our attention. First of all, it makes sense to
> deprecate RuntimeContext#getExecutionConfig.
>
> Afaic, this is an issue of how we design API with clean concepts/aspects.
> There are two issues mentioned in the FLIP:
>
> 1. short of user-facing abstraction - we just exposed ExecutionConfig
> which mixed methods for users with methods that should only be used
> internally.
> 2. mutable vs immutable - do we want users to be able to modify configs
> during job execution?
>
> An immutable user-facing abstraction design can solve both issues. All
> execution related configs are still consolidated into the abstraction class
> and easy to access. This is another design decision: flat vs. hierarchical.
> Current FLIP removed the execution config abstraction and put all methods
> into RuntimeContext flat, which will end up with more than 30 methods
> offered flat by the RuntimeContext. I am not sure if this could help users
> find the right method in the context of execution config better than
> before.
>
> I might miss something and look forward to your thoughts. Thanks!
>
> Best regards,
> Jing
>
> On Sat, Nov 18, 2023 at 11:21 AM Junrui Lee  wrote:
>
>> Hello Wencong,
>>
>> Thank you for your valuable feedback and suggestions. I want to clarify
>> that reviewing existing methods in the ExecutionConfig is not directly
>> related to the proposal in this FLIP. The main focus of this FLIP is to
>> deprecate the specific method RuntimeContext#getExecutionConfig(). I
>> believe it is important to keep the scope of this FLIP limited. However,
>> your suggestion can certainly be considered as a separate FLIP in the
>> future.
>>
>> Best regards,
>> Junrui
>>
>> Wencong Liu  于2023年11月17日周五 22:08写道:
>>
>>> Hello Junrui,
>>>
>>>
>>> Thanks for the effort. I agree with the proposal to deprecate the
>>> getExecutionConfig() method in the RuntimeContext class. Exposing
>>> the complex ExecutionConfig to user-defined functions can lead to
>>> unnecessary complexity and risks.
>>>
>>>
>>> I also have a suggestion. We could consider reviewing the existing
>>>  methods in ExecutionConfig. If there are methods that are defined
>>>  in ExecutionConfig but currently have no callers, we could consider
>>>  annotating  them as @Internal or directly removing them. Since
>>> users are no longer able to access and invoke these methods,
>>> it would be beneficial to clean up the codebase.
>>>
>>>
>>> +1 (non-binding).
>>>
>>>
>>> Best,
>>> Wencong
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> At 2023-11-15 16:51:15, "Junrui Lee"  wrote:
>>> >Hi all,
>>> >
>>> >I'd like to start a discussion of FLIP-391: Deprecate
>>> >RuntimeContext#getExecutionConfig[1].
>>> >
>>> >Currently, the FLINK RuntimeContext is important for connecting user
>>> >functions to the underlying runtime details. It provides users with
>>> >necessary runtime information during job execution.
>>> >However, he current implementation of the FLINK RuntimeContext exposes
>>> the
>>> >ExecutionConfig to users, resulting in two issues:
>>> >Firstly, the ExecutionConfig contains much unrelated information that
>>> can
>>> >confuse users and complicate management.
>>> >Secondly, exposing the ExecutionConfig allows users to modify it during
>>> job
>>> >execution, which can cause inconsistencies and probl

Re: [DISCUSS] FLIP-391: Deprecate RuntimeContext#getExecutionConfig

2023-11-18 Thread Junrui Lee
Hello Wencong,

Thank you for your valuable feedback and suggestions. I want to clarify
that reviewing existing methods in the ExecutionConfig is not directly
related to the proposal in this FLIP. The main focus of this FLIP is to
deprecate the specific method RuntimeContext#getExecutionConfig(). I
believe it is important to keep the scope of this FLIP limited. However,
your suggestion can certainly be considered as a separate FLIP in the
future.

Best regards,
Junrui

Wencong Liu  于2023年11月17日周五 22:08写道:

> Hello Junrui,
>
>
> Thanks for the effort. I agree with the proposal to deprecate the
> getExecutionConfig() method in the RuntimeContext class. Exposing
> the complex ExecutionConfig to user-defined functions can lead to
> unnecessary complexity and risks.
>
>
> I also have a suggestion. We could consider reviewing the existing
>  methods in ExecutionConfig. If there are methods that are defined
>  in ExecutionConfig but currently have no callers, we could consider
>  annotating  them as @Internal or directly removing them. Since
> users are no longer able to access and invoke these methods,
> it would be beneficial to clean up the codebase.
>
>
> +1 (non-binding).
>
>
> Best,
> Wencong
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
> At 2023-11-15 16:51:15, "Junrui Lee"  wrote:
> >Hi all,
> >
> >I'd like to start a discussion of FLIP-391: Deprecate
> >RuntimeContext#getExecutionConfig[1].
> >
> >Currently, the FLINK RuntimeContext is important for connecting user
> >functions to the underlying runtime details. It provides users with
> >necessary runtime information during job execution.
> >However, he current implementation of the FLINK RuntimeContext exposes the
> >ExecutionConfig to users, resulting in two issues:
> >Firstly, the ExecutionConfig contains much unrelated information that can
> >confuse users and complicate management.
> >Secondly, exposing the ExecutionConfig allows users to modify it during
> job
> >execution, which can cause inconsistencies and problems, especially with
> >operator chaining.
> >
> >Therefore, we propose deprecating the RuntimeContext#getExecutionConfig in
> >the FLINK RuntimeContext. In the upcoming FLINK-2.0 version, we plan to
> >completely remove the RuntimeContext#getExecutionConfig method. Instead,
> we
> >will introduce alternative getter methods that enable users to access
> >specific information without exposing unnecessary runtime details. These
> >getter methods will include:
> >
> >1. @PublicEvolving  TypeSerializer
> >createSerializer(TypeInformation typeInformation);
> >2. @PublicEvolving Map getGlobalJobParameters();
> >3. @PublicEvolving boolean isObjectReuseEnabled();
> >
> >Looking forward to your feedback and suggestions, thanks.
> >
> >[1]
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278465937
> >
> >Best regards,
> >Junrui
>


Re: [DISCUSS] FLIP-391: Deprecate RuntimeContext#getExecutionConfig

2023-11-15 Thread Junrui Lee
Hi Rui,

Thank you for your feedback. Regarding your question, I want to let you
know that we have included the alternative getter method "@PublicEvolving
Map getGlobalJobParameters()" in this FLIP. And this method
will provide users with global job parameters.
I would appreciate it if you could share your thoughts on whether this
addition adequately resolves your concerns.

Best,
Junrui

Rui Fan <1996fan...@gmail.com> 于2023年11月16日周四 13:15写道:

> Thanks Junrui for driving this proposal!
>
> Overall looks good to me! I have a question:
> How do the flink users to getGlobalJobParameters if
> RuntimeContext#getExecutionConfig is removed?
> Is there any alternative api? Or is this feature removed?
>
> Maybe this is a noob question and I may have missed
> some information.
>
> Best,
> Rui
>
> On Thu, Nov 16, 2023 at 11:05 AM Zhu Zhu  wrote:
>
>> Thanks Junrui for creating the FLIP and kicking off this discussion.
>>
>> Exposing a mutable ExecutionConfig which is even shared by multiple
>> operators is truly a defect which can result in weird results.
>>
>> +1
>>
>> Thanks,
>> Zhu
>>
>> Junrui Lee  于2023年11月15日周三 16:53写道:
>>
>>> Hi all,
>>>
>>> I'd like to start a discussion of FLIP-391: Deprecate
>>> RuntimeContext#getExecutionConfig[1].
>>>
>>> Currently, the FLINK RuntimeContext is important for connecting user
>>> functions to the underlying runtime details. It provides users with
>>> necessary runtime information during job execution.
>>> However, he current implementation of the FLINK RuntimeContext exposes
>>> the
>>> ExecutionConfig to users, resulting in two issues:
>>> Firstly, the ExecutionConfig contains much unrelated information that can
>>> confuse users and complicate management.
>>> Secondly, exposing the ExecutionConfig allows users to modify it during
>>> job
>>> execution, which can cause inconsistencies and problems, especially with
>>> operator chaining.
>>>
>>> Therefore, we propose deprecating the RuntimeContext#getExecutionConfig
>>> in
>>> the FLINK RuntimeContext. In the upcoming FLINK-2.0 version, we plan to
>>> completely remove the RuntimeContext#getExecutionConfig method. Instead,
>>> we
>>> will introduce alternative getter methods that enable users to access
>>> specific information without exposing unnecessary runtime details. These
>>> getter methods will include:
>>>
>>> 1. @PublicEvolving  TypeSerializer
>>> createSerializer(TypeInformation typeInformation);
>>> 2. @PublicEvolving Map getGlobalJobParameters();
>>> 3. @PublicEvolving boolean isObjectReuseEnabled();
>>>
>>> Looking forward to your feedback and suggestions, thanks.
>>>
>>> [1]
>>>
>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278465937
>>>
>>> Best regards,
>>> Junrui
>>>
>>


[DISCUSS] FLIP-391: Deprecate RuntimeContext#getExecutionConfig

2023-11-15 Thread Junrui Lee
Hi all,

I'd like to start a discussion of FLIP-391: Deprecate
RuntimeContext#getExecutionConfig[1].

Currently, the FLINK RuntimeContext is important for connecting user
functions to the underlying runtime details. It provides users with
necessary runtime information during job execution.
However, he current implementation of the FLINK RuntimeContext exposes the
ExecutionConfig to users, resulting in two issues:
Firstly, the ExecutionConfig contains much unrelated information that can
confuse users and complicate management.
Secondly, exposing the ExecutionConfig allows users to modify it during job
execution, which can cause inconsistencies and problems, especially with
operator chaining.

Therefore, we propose deprecating the RuntimeContext#getExecutionConfig in
the FLINK RuntimeContext. In the upcoming FLINK-2.0 version, we plan to
completely remove the RuntimeContext#getExecutionConfig method. Instead, we
will introduce alternative getter methods that enable users to access
specific information without exposing unnecessary runtime details. These
getter methods will include:

1. @PublicEvolving  TypeSerializer
createSerializer(TypeInformation typeInformation);
2. @PublicEvolving Map getGlobalJobParameters();
3. @PublicEvolving boolean isObjectReuseEnabled();

Looking forward to your feedback and suggestions, thanks.

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278465937

Best regards,
Junrui


[RESULT][VOTE] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

2023-11-14 Thread Junrui Lee
Hi all,

The voting time of FLIP-381[1]: Deprecate configuration getters/setters
that return/set complex Java objects has passed. I'm closing the vote now.

There were 12 +1 votes, 6 of which are binding:

Rui Fan (binding)
Weijie Guo (binding)
Yangze Guo (binding)
Lijie Wang (binding)
Jing Ge (binding)
Zhu Zhu (binding)
Yuepeng Pan
Yuxin Tan
Samrat Deb
Xia Sun
Hang Ruan
Mingliang Liu


There were no -1 votes.

Thus FLIP-381 has been accepted.

Thanks everyone for joining the discussion and giving feedback!

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992


Best,
Junrui


[VOTE] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

2023-11-09 Thread Junrui Lee
Hi everyone,

Thank you to everyone for the feedback on FLIP-381: Deprecate configuration
getters/setters that return/set complex Java objects[1] which has been
discussed in this thread [2].

I would like to start a vote for it. The vote will be open for at least 72
hours (excluding weekends) unless there is an objection or not enough votes.

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992
[2]https://lists.apache.org/thread/y5owjkfxq3xs9lmpdbl6d6jmqdgbjqxo


Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

2023-11-05 Thread Junrui Lee
Hi Jing,

Thank you for your feedback on the proposal. I appreciate your input and
understand your concerns.

Regarding your questions, the complex Java class that will be deprecated is
org.apache.flink.api.common.restartstrategy.RestartStrategies and and its
internal classes, which include:

- RestartStrategyConfiguration
- FallbackRestartStrategyConfiguration
- FailureRateRestartStrategyConfiguration
- ExponentialDelayRestartStrategyConfiguration
- FixedDelayRestartStrategyConfiguration
- NoRestartStrategyConfiguration

And based on your suggestion, I have restructured the content of the FLIP
to clearly indicate which entire classes will be deprecated and which
methods and fields will be deprecated. Please take a look at the updated
version.

Best regards,
Junrui

Jing Ge  于2023年11月5日周日 00:31写道:

> Hi Junrui,
>
> Thanks for bringing the proposal to our attention. It looks overall great.
>
> I am just a little bit confused with the content of Flip with the following
> questions:
>
> 1. How many complex java classes will be deprecated?
> 2. After reading the section title "Deprecate following classes, fields and
> methods", I thought that e.g.
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment will
> be deprecated. I would suggest restructure the content and point out
> clearly which entire classes will be deprecated and which methods and
> fields and not the class itself will be deprecated instead of mixing them
> up. WDYT?
>
> Best regards,
> Jing
>
>
> On Fri, Nov 3, 2023 at 8:59 AM Wencong Liu  wrote:
>
> > Thanks Junrui for your effort!
> >
> > Making all configuration code paths lead to ConfigOption is a more
> > standardized
> > approach to configuring Flink applications.
> >
> > +1 for this proposal.
> >
> > Best,
> > Wencong Liu
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> > At 2023-11-02 10:10:14, "Junrui Lee"  wrote:
> > >Hi devs,
> > >
> > >I would like to start a discussion on FLIP-381: Deprecate configuration
> > >getters/setters that return/set complex Java objects[1].
> > >
> > >Currently, the job configuration in FLINK is spread out across different
> > >components, which leads to inconsistencies and confusion. To address
> this
> > >issue, it is necessary to migrate non-ConfigOption complex Java objects
> to
> > >use ConfigOption and adopt a single Configuration object to host all the
> > >configuration.
> > >However, there is a significant blocker in implementing this solution.
> > >These complex Java objects in StreamExecutionEnvironment,
> > CheckpointConfig,
> > >and ExecutionConfig have already been exposed through the public API,
> > >making it challenging to modify the existing implementation.
> > >
> > >Therefore, I propose to deprecate these Java objects and their
> > >corresponding getter/setter interfaces, ultimately removing them in
> > >FLINK-2.0.
> > >
> > >Your feedback and thoughts on this proposal are highly appreciated.
> > >
> > >Best regards,
> > >Junrui Lee
> > >
> > >[1]
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992
> >
>


Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

2023-11-02 Thread Junrui Lee
Hi Hang,

Thank you for your feedback. Regarding your suggestion to delete the
private field 'storage', I want to clarify that in this FLIP, we are only
deprecating the getter and setter methods for CheckpointConfig#storage.
Therefore, we cannot directly remove the 'storage' field at this moment.
However, we will consider removing it altogether in FLINK-2.0.

Best regards, Junrui

Hang Ruan  于2023年11月3日周五 12:04写道:

> Thanks Junrui for driving the proposal.
>
> +1 from my side. This FLIP will help to make the configuration clearer for
> users.
>
> ps: We should also delete the private field `storage` as its getter and
> setter are deleted and it is marked as `@Deprecated`. This is not written
> in the FLIP.
>
> Best,
> Hang
>
> Yuxin Tan  于2023年11月3日周五 11:30写道:
>
> > Thanks Junrui for driving the proposal.
> >
> > +1 for this proposal. I believe this change will enhance the usability of
> > Flink configuration for both users and developers, while also ensuring
> > consistency across various types of configurations.
> >
> > Best,
> > Yuxin
> >
> >
> > Lijie Wang  于2023年11月3日周五 10:59写道:
> >
> > > Thanks Junrui for driving this.
> > >
> > > Making configurations simple and consistent has great benefits for both
> > > users and devs. +1 for the proposal.
> > >
> > > Best,
> > > Lijie
> > >
> > > weijie guo  于2023年11月2日周四 16:49写道:
> > >
> > > > Thanks Junrui for driving this proposal!
> > > >
> > > > I believe this is helpful for the new Process Function API. Because
> we
> > > > don't need to move some related class/components from flink-core to a
> > > pure
> > > > API module (maybe, called flink-core-api) after this. Even though the
> > > FLIP
> > > > related to new API is in preparation atm, I still want to emphasize
> our
> > > > goal is that user application should no longer depend on these stuff.
> > So
> > > > I'm + 1 for this proposal.
> > > >
> > > >
> > > > Best regards,
> > > >
> > > > Weijie
> > > >
> > > >
> > > > Zhu Zhu  于2023年11月2日周四 16:00写道:
> > > >
> > > > > Thanks Junrui for creating the FLIP and kicking off this
> discussion.
> > > > >
> > > > > The community has been constantly striving to unify and simplify
> the
> > > > > configuration layer of Flink. Some progress has already been made,
> > > > > such as FLINK-29379. However, the compatibility of public
> interfaces
> > > > > poses an obstacle to completing the task. The release of Flink 2.0
> > > > > presents a great opportunity to accomplish this goal.
> > > > >
> > > > > +1 for the proposal.
> > > > >
> > > > > Thanks,
> > > > > Zhu
> > > > >
> > > > > Rui Fan <1996fan...@gmail.com> 于2023年11月2日周四 10:27写道:
> > > > >
> > > > > > Thanks Junrui for driving this proposal!
> > > > > >
> > > > > > ConfigOption is easy to use for flink users, easy to manage
> options
> > > > > > for flink platform maintainers, and easy to maintain for flink
> > > > developers
> > > > > > and flink community.
> > > > > >
> > > > > > So big +1 for this proposal!
> > > > > >
> > > > > > Best,
> > > > > > Rui
> > > > > >
> > > > > > On Thu, Nov 2, 2023 at 10:10 AM Junrui Lee 
> > > > wrote:
> > > > > >
> > > > > > > Hi devs,
> > > > > > >
> > > > > > > I would like to start a discussion on FLIP-381: Deprecate
> > > > configuration
> > > > > > > getters/setters that return/set complex Java objects[1].
> > > > > > >
> > > > > > > Currently, the job configuration in FLINK is spread out across
> > > > > different
> > > > > > > components, which leads to inconsistencies and confusion. To
> > > address
> > > > > this
> > > > > > > issue, it is necessary to migrate non-ConfigOption complex Java
> > > > objects
> > > > > > to
> > > > > > > use ConfigOption and adopt a single Configuration object to
> host
> > > all
> > > > > the
> > > > > > > configuration.
> > > > > > > However, there is a significant blocker in implementing this
> > > > solution.
> > > > > > > These complex Java objects in StreamExecutionEnvironment,
> > > > > > CheckpointConfig,
> > > > > > > and ExecutionConfig have already been exposed through the
> public
> > > API,
> > > > > > > making it challenging to modify the existing implementation.
> > > > > > >
> > > > > > > Therefore, I propose to deprecate these Java objects and their
> > > > > > > corresponding getter/setter interfaces, ultimately removing
> them
> > in
> > > > > > > FLINK-2.0.
> > > > > > >
> > > > > > > Your feedback and thoughts on this proposal are highly
> > appreciated.
> > > > > > >
> > > > > > > Best regards,
> > > > > > > Junrui Lee
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


[DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

2023-11-01 Thread Junrui Lee
Hi devs,

I would like to start a discussion on FLIP-381: Deprecate configuration
getters/setters that return/set complex Java objects[1].

Currently, the job configuration in FLINK is spread out across different
components, which leads to inconsistencies and confusion. To address this
issue, it is necessary to migrate non-ConfigOption complex Java objects to
use ConfigOption and adopt a single Configuration object to host all the
configuration.
However, there is a significant blocker in implementing this solution.
These complex Java objects in StreamExecutionEnvironment, CheckpointConfig,
and ExecutionConfig have already been exposed through the public API,
making it challenging to modify the existing implementation.

Therefore, I propose to deprecate these Java objects and their
corresponding getter/setter interfaces, ultimately removing them in
FLINK-2.0.

Your feedback and thoughts on this proposal are highly appreciated.

Best regards,
Junrui Lee

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992


Re: [DISCUSS][FLINK-33240] Document deprecated options as well

2023-11-01 Thread Junrui Lee
Hi Zhanghao,

Thank you for the proposal.

+1 from my side. It would be more user-friendly to have the deprecated
options in the same section as the non-deprecated ones. Therefore, adding
them in the same section sounds good to me.

Best regards,
Junrui

Zhanghao Chen  于2023年11月1日周三 21:10写道:

> Hi Samrat and Ruan,
>
> Thanks for the suggestion. I'm actually in favor of adding the deprecated
> options in the same section as the non-deprecated ones. This would make
> user search for descriptions of the replacement options more easily. It
> would be a different story for options deprecated because the related
> API/module is entirely deprecated, e.g. DataSet API. In that case, users
> would not search for replacement on an individual option but rather need to
> migrate to a new API, and it would be better to move these options to a
> separate section. WDYT?
>
> Best,
> Zhanghao Chen
> 
> From: Samrat Deb 
> Sent: Wednesday, November 1, 2023 15:31
> To: dev@flink.apache.org 
> Cc: u...@flink.apache.org 
> Subject: Re: [DISCUSS][FLINK-33240] Document deprecated options as well
>
> Thanks for the proposal ,
> +1 for adding deprecated identifier
>
> [Thought] Can we have seperate section / page for deprecated configs ? Wdut
> ?
>
>
> Bests,
> Samrat
>
>
> On Tue, 31 Oct 2023 at 3:44 PM, Alexander Fedulov <
> alexander.fedu...@gmail.com> wrote:
>
> > Hi Zhanghao,
> >
> > Thanks for the proposition.
> > In general +1, this sounds like a good idea as long it is clear that the
> > usage of these settings is discouraged.
> > Just one minor concern - the configuration page is already very long, do
> > you have a rough estimate of how many more options would be added with
> this
> > change?
> >
> > Best,
> > Alexander Fedulov
> >
> > On Mon, 30 Oct 2023 at 18:24, Matthias Pohl  > .invalid>
> > wrote:
> >
> > > Thanks for your proposal, Zhanghao Chen. I think it adds more
> > transparency
> > > to the configuration documentation.
> > >
> > > +1 from my side on the proposal
> > >
> > > On Wed, Oct 11, 2023 at 2:09 PM Zhanghao Chen <
> zhanghao.c...@outlook.com
> > >
> > > wrote:
> > >
> > > > Hi Flink users and developers,
> > > >
> > > > Currently, Flink won't generate doc for the deprecated options. This
> > > might
> > > > confuse users when upgrading from an older version of Flink: they
> have
> > to
> > > > either carefully read the release notes or check the source code for
> > > > upgrade guidance on deprecated options.
> > > >
> > > > I propose to document deprecated options as well, with a
> "(deprecated)"
> > > > tag placed at the beginning of the option description to highlight
> the
> > > > deprecation status [1].
> > > >
> > > > Looking forward to your feedbacks on it.
> > > >
> > > > [1] https://issues.apache.org/jira/browse/FLINK-33240
> > > >
> > > > Best,
> > > > Zhanghao Chen
> > > >
> > >
> >
>


[RESULT][VOTE] FLIP-366: Support standard YAML for FLINK configuration

2023-10-17 Thread Junrui Lee
Hi all,

The voting time of FLIP-366 [1]:Support standard YAML for FLINK
configuration has passed. I'm closing the vote now.

There were 18 +1 votes, 9 of which are binding:

Rui Fan (binding)
Weijie Guo (binding)
Yangze Guo (binding)
Lijie Wang (binding)
Jing Ge (binding)
Zhu Zhu (binding)
Chesnay Schepler (binding)
Xintong Song (binding)
tison (binding)

There were no -1 votes.

Thus FLIP-366 has been accepted.

Thanks everyone for joining the discussion and giving feedback!

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-366%3A+Support+standard+YAML+for+FLINK+configuration


Best,
Junrui


Re: [ANNOUNCE] New Apache Flink Committer - Jane Chan

2023-10-15 Thread Junrui Lee
Congratulations Jane !

Best,
Junrui

Yun Tang  于2023年10月16日周一 10:21写道:

> Congratulations, Jane!
>
> Best
> Yun Tang
> 
> From: Rui Fan <1996fan...@gmail.com>
> Sent: Monday, October 16, 2023 10:16
> To: dev@flink.apache.org 
> Cc: qingyue@gmail.com 
> Subject: Re: [ANNOUNCE] New Apache Flink Committer - Jane Chan
>
> Congratulations Jane!
>
> Best,
> Rui
>
> On Mon, Oct 16, 2023 at 10:15 AM yu zelin  wrote:
>
> > Congratulations!
> >
> > Best,
> > Yu Zelin
> >
> > > 2023年10月16日 09:58,Jark Wu  写道:
> > >
> > > Hi, everyone
> > >
> > > On behalf of the PMC, I'm very happy to announce Jane Chan as a new
> Flink
> > > Committer.
> > >
> > > Jane started code contribution in Jan 2021 and has been active in the
> > Flink
> > > community since. She authored more than 60 PRs and reviewed more than
> 40
> > > PRs. Her contribution mainly revolves around Flink SQL, including Plan
> > > Advice (FLIP-280), operator-level state TTL (FLIP-292), and ALTER TABLE
> > > statements (FLINK-21634). Jane participated deeply in development
> > > discussions and also helped answer user question emails. Jane was also
> a
> > > core contributor of Flink Table Store (now Paimon) when the project was
> > in
> > > the early days.
> > >
> > > Please join me in congratulating Jane Chan for becoming a Flink
> > Committer!
> > >
> > > Best,
> > > Jark Wu (on behalf of the Flink PMC)
> >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Ron Liu

2023-10-15 Thread Junrui Lee
Congratulations Ron !

Best,
Junrui

Yun Tang  于2023年10月16日周一 10:22写道:

> Congratulations, Ron!
>
> Best
> Yun Tang
> 
> From: yu zelin 
> Sent: Monday, October 16, 2023 10:16
> To: dev@flink.apache.org 
> Cc: ron9@gmail.com 
> Subject: Re: [ANNOUNCE] New Apache Flink Committer - Ron Liu
>
> Congratulations!
>
> Best,
> Yu Zelin
>
> > 2023年10月16日 09:56,Jark Wu  写道:
> >
> > Hi, everyone
> >
> > On behalf of the PMC, I'm very happy to announce Ron Liu as a new Flink
> > Committer.
> >
> > Ron has been continuously contributing to the Flink project for many
> years,
> > authored and reviewed a lot of codes. He mainly works on Flink SQL parts
> > and drove several important FLIPs, e.g., USING JAR (FLIP-214), Operator
> > Fusion CodeGen (FLIP-315), Runtime Filter (FLIP-324). He has a great
> > knowledge of the Batch SQL and improved a lot of batch performance in the
> > past several releases. He is also quite active in mailing lists,
> > participating in discussions and answering user questions.
> >
> > Please join me in congratulating Ron Liu for becoming a Flink Committer!
> >
> > Best,
> > Jark Wu (on behalf of the Flink PMC)
>
>


[VOTE] FLIP-366: Support standard YAML for FLINK configuration

2023-10-12 Thread Junrui Lee
Hi all,

Thank you to everyone for the feedback on FLIP-366[1]: Support standard
YAML for FLINK configuration in the discussion thread [2].
I would like to start a vote for it. The vote will be open for at least 72
hours (excluding weekends, unless there is an objection or an insufficient
number of votes).

Thanks,
Junrui

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-366%3A+Support+standard+YAML+for+FLINK+configuration
[2]https://lists.apache.org/thread/qfhcm7h8r5xkv38rtxwkghkrcxg0q7k5


Re: [Discuss] FLIP-366: Support standard YAML for FLINK configuration

2023-10-12 Thread Junrui Lee
Hi everyone,
Thanks for all the comments! Based on the discussion so far, I will proceed
to initiate the vote tomorrow if there are no further discussions or
objections.

Best regards,
Junrui

Junrui Lee  于2023年10月9日周一 11:52写道:

> Hi, Chesnay and David, sorry for the late reply.
>
> Thanks for your feedback about this FLIP.
>
> @Chesnay
>
> >> Personally I'd just name it "config.yaml"
> Thanks for your suggestions, I agree that "config.yaml" is a simpler and
> more intuitive name. I have updated the FLIP accordingly.
>
> >> In terms of scope, is the migration of existing e2e tests and the
> docker image to the new parser part of the FLIP?
> The migration of existing end-to-end tests and the docker image to the new
> parser is indeed included in the FLIP. Taking into account the difficulty
> of using shell scripts to modify configuration files that adhere to
> standard YAML syntax, I propose utilizing the
> org.apache.flink.runtime.util.bash.BashJavaUtils class as a unified utility
> for modifying the configuration file.
>
> In addition, considering that users may not only configure properties in
> the configuration file, but also use dynamic parameters like '-D' or
> configure the "with" clauses in FLINK SQL. When the default parser is
> changed to the standard YAML parser, the configOption whose value is List
> or Map types also needs to be modified accordingly. This migration cost is
> relatively high because users have to modify their code or command lines.
> To reduce migration costs while considering compatibility, I propose to
> add compatibility support in FLINK-1.X versions to parse old format List
> and Map values using the standard YAML parser. In case of an error while
> parsing List or Map configurations with the standard YAML parser, a WARNING
> log will be printed, and an attempt will be made to parse using the old
> parser. And this support will be removed in the FLINK-2.0 version.
>
> Do you think the aforementioned compatibility handling is necessary in
> order to reduce migration costs?  If you agree, I will also update this
> support in the FLIP under the migration section.
>
> @David
>
> >> could we write a migration tool, that would convert the old config
> files to the new format,
> I agree that providing a more user-friendly guidance for the migration is
> important. In the FLIP, we can include a user migration manual to help
> users with the migration process. This migration manual will be presented
> on the official FLINK website, making it easily accessible for users. And
> the migrated configuration files will adhere to standard YAML syntax, and
> users can validate them using various third-party YAML tools.
>
> Regarding the automated migration tool, I think it can be considered as a
> follow-up task rather than a requirement for the initial version. We can
> focus on implementing the migration manually first and then explore the
> possibility of automating it in future updates.
>
> Best,
> Junrui
>
> David Radley  于2023年10月3日周二 18:56写道:
>
>> Hi,
>> I agree this is a standardising, simplifying change for read,
>> simplifying programmatically authoring the config file as well.  As you
>> know the mapping for the old config form to the new form, could we write a
>> migration tool, that would convert the old config files to the new format,
>> with errors and warnings if the content is such that the user would need to
>> manually fix up the file.
>> If there are minimal user fixups, we should consider automatically
>> migrating the config file to the new format,
>> Kind regards, David.
>>
>>
>>
>>
>> From: Chesnay Schepler 
>> Date: Tuesday, 3 October 2023 at 11:17
>> To: dev@flink.apache.org 
>> Subject: [EXTERNAL] Re: [Discuss] FLIP-366: Support standard YAML for
>> FLINK configuration
>> It is a unfortunate that we'll need a separate config file but the FLIP
>> does a good job justifying it.
>>
>> Personally I'd just name it "config.yaml"; I never quite understood why
>> there was a flink prefix to begin with, and the current proposal
>> ("flink-configuration.yaml") seems unnecessarily long.
>>
>> For the deprecation process we could consider logging a warning if the
>> old parser is used.
>>
>> In terms of scope, is the migration of existing e2e tests and the docker
>> image to the new parser part of the FLIP?
>>
>> On 22/09/2023 09:32, Jane Chan wrote:
>> > Hi, Junrui,
>> >
>> > Sorry for the late reply. The update looks good to me and thanks for
>> your
>> > effort!
>> >
>> >

Re: [Discuss] FLIP-366: Support standard YAML for FLINK configuration

2023-10-08 Thread Junrui Lee
; +1 for the proposal, thanks for driving.
> >>>>
> >>>> Bet,
> >>>> Shammon FY
> >>>>
> >>>> On Fri, Sep 22, 2023 at 12:41 PM Yangze Guo 
> >> wrote:
> >>>>> Thanks for driving this, +1 for the proposal.
> >>>>>
> >>>>> Best,
> >>>>> Yangze Guo
> >>>>>
> >>>>>
> >>>>> On Fri, Sep 22, 2023 at 11:59 AM Lijie Wang <
> >> wangdachui9...@gmail.com>
> >>>>> wrote:
> >>>>>> Hi Junrui,
> >>>>>>
> >>>>>> +1 for this proposal, thanks for driving.
> >>>>>>
> >>>>>> Best,
> >>>>>> Lijie
> >>>>>>
> >>>>>> ConradJam  于2023年9月22日周五 10:07写道:
> >>>>>>
> >>>>>>> +1 Support for standard YAML format facilitates specification
> >>>>>>>
> >>>>>>> Jing Ge  于2023年9月22日周五 02:23写道:
> >>>>>>>
> >>>>>>>> Hi Junrui,
> >>>>>>>>
> >>>>>>>> +1 for following the standard. Thanks for your effort!
> >>>>>>>>
> >>>>>>>> Best regards,
> >>>>>>>> Jing
> >>>>>>>>
> >>>>>>>> On Thu, Sep 21, 2023 at 5:09 AM Junrui Lee <
> >> jrlee@gmail.com>
> >>>>> wrote:
> >>>>>>>>> Hi Jane,
> >>>>>>>>>
> >>>>>>>>> Thank you for your valuable feedback and suggestions.
> >>>>>>>>> I agree with your point about differentiating between
> >>>>>>> "flink-config.yaml"
> >>>>>>>>> and "flink-conf.yaml" to determine the standard syntax at a
> >>>> glance.
> >>>>>>>>> While I understand your suggestion of using
> >>>>> "flink-conf-default.yaml"
> >>>>>>> to
> >>>>>>>>> represent the default YAML file for Flink 1.x, I have been
> >>>>> considering
> >>>>>>>>> the option of using "flink-configuration.yaml" as the file
> >> name
> >>>>> for the
> >>>>>>>>> new configuration file.
> >>>>>>>>> This name "flink-configuration.yaml" provides a clear
> >>> distinction
> >>>>>>> between
> >>>>>>>>> the new and old configuration files based on their names, and
> >>> it
> >>>>> does
> >>>>>>> not
> >>>>>>>>> introduce any additional semantics. Moreover, this name
> >>>>>>>>> "flink-configuration.yaml" can continue to be used in future
> >>>>> versions
> >>>>>>>>> FLINK-2.0.
> >>>>>>>>>
> >>>>>>>>> WDYT? If we can reach a consensus on this, I will update the
> >>> FLIP
> >>>>>>>>> documentation
> >>>>>>>>> accordingly.
> >>>>>>>>>
> >>>>>>>>> Best regards,
> >>>>>>>>> Junrui
> >>>>>>>>>
> >>>>>>>>> Jane Chan  于2023年9月20日周三 23:38写道:
> >>>>>>>>>
> >>>>>>>>>> Hi Junrui,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for driving this FLIP. +1 for adoption of the
> >> standard
> >>>>> YAML
> >>>>>>>>> syntax.
> >>>>>>>>>> I just have one minor suggestion. It's a little bit
> >>> challenging
> >>>>> to
> >>>>>>>>>> differentiate between `flink-config.yaml` and
> >>> `flink-conf.yaml`
> >>>>> to
> >>>>>>>>>> determine which one uses the standard syntax at a glance.
> >> How
> >>>>> about
> >>>>>>>>>> using `flink-conf-default.yaml` to represent the default
> >> yaml
> >>>>> file
> >>>>>>> for
> >>>>>>>>>> Flink 1.x?
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Jane
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Sep 20, 2023 at 11:06 AM Junrui Lee <
> >>>> jrlee@gmail.com
> >>>>>>>> wrote:
> >>>>>>>>>>> Hi devs,
> >>>>>>>>>>>
> >>>>>>>>>>> I would like to start a discussion about FLIP-366:
> >>>>>>>>>>> Support standard YAML for FLINK configuration[1]
> >>>>>>>>>>>
> >>>>>>>>>>> The current flink-conf.yaml parser in FLINK is not a
> >>> standard
> >>>>> YAML
> >>>>>>>>>> parser,
> >>>>>>>>>>> which has some shortcomings.
> >>>>>>>>>>> Firstly, it does not support nested structure
> >> configuration
> >>>>> items
> >>>>>>> and
> >>>>>>>>>> only
> >>>>>>>>>>> supports key-value pairs, resulting in poor readability.
> >>>>> Secondly,
> >>>>>>> if
> >>>>>>>>> the
> >>>>>>>>>>> value is a collection type, such as a List or Map, users
> >>> are
> >>>>>>> required
> >>>>>>>>> to
> >>>>>>>>>>> write the value in a FLINK-specific pattern, which is
> >>>>> inconvenient
> >>>>>>> to
> >>>>>>>>>> use.
> >>>>>>>>>>> Additionally, the parser of FLINK has some differences in
> >>>>> syntax
> >>>>>>>>> compared
> >>>>>>>>>>> to the standard YAML parser, such as the syntax for
> >> parsing
> >>>>>>> comments
> >>>>>>>>> and
> >>>>>>>>>>> null values. These inconsistencies can cause confusion
> >> for
> >>>>> users,
> >>>>>>> as
> >>>>>>>>> seen
> >>>>>>>>>>> in FLINK-15358 and FLINK-32740.
> >>>>>>>>>>>
> >>>>>>>>>>> By supporting standard YAML, these issues can be
> >> resolved,
> >>>> and
> >>>>>>> users
> >>>>>>>>> can
> >>>>>>>>>>> create a Flink configuration file using third-party tools
> >>> and
> >>>>>>>> leverage
> >>>>>>>>>>> some advanced YAML features. Therefore, we propose to
> >>> support
> >>>>>>>> standard
> >>>>>>>>>>> YAML for FLINK configuration.
> >>>>>>>>>>>
> >>>>>>>>>>> You can find more details in the FLIP-366[1]. Looking
> >>> forward
> >>>>> to
> >>>>>>> your
> >>>>>>>>>>> feedback.
> >>>>>>>>>>>
> >>>>>>>>>>> [1]
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-366%3A+Support+standard+YAML+for+FLINK+configuration
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Junrui
> >>>>>>>>>>>
> >>>>>>>
> >>>>>>> --
> >>>>>>> Best
> >>>>>>>
> >>>>>>> ConradJam
> >>>>>>>
>
> Unless otherwise stated above:
>
> IBM United Kingdom Limited
> Registered in England and Wales with number 741598
> Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 3AU
>


Re: [Discuss] FLIP-366: Support standard YAML for FLINK configuration

2023-09-20 Thread Junrui Lee
Hi Jane,

Thank you for your valuable feedback and suggestions.
I agree with your point about differentiating between "flink-config.yaml"
and "flink-conf.yaml" to determine the standard syntax at a glance.

While I understand your suggestion of using "flink-conf-default.yaml" to
represent the default YAML file for Flink 1.x, I have been considering
the option of using "flink-configuration.yaml" as the file name for the
new configuration file.
This name "flink-configuration.yaml" provides a clear distinction between
the new and old configuration files based on their names, and it does not
introduce any additional semantics. Moreover, this name
"flink-configuration.yaml" can continue to be used in future versions
FLINK-2.0.

WDYT? If we can reach a consensus on this, I will update the FLIP
documentation
accordingly.

Best regards,
Junrui

Jane Chan  于2023年9月20日周三 23:38写道:

> Hi Junrui,
>
> Thanks for driving this FLIP. +1 for adoption of the standard YAML syntax.
> I just have one minor suggestion. It's a little bit challenging to
> differentiate between `flink-config.yaml` and `flink-conf.yaml` to
> determine which one uses the standard syntax at a glance. How about
> using `flink-conf-default.yaml` to represent the default yaml file for
> Flink 1.x?
>
> Best,
> Jane
>
> On Wed, Sep 20, 2023 at 11:06 AM Junrui Lee  wrote:
>
> > Hi devs,
> >
> > I would like to start a discussion about FLIP-366:
> > Support standard YAML for FLINK configuration[1]
> >
> > The current flink-conf.yaml parser in FLINK is not a standard YAML
> parser,
> > which has some shortcomings.
> > Firstly, it does not support nested structure configuration items and
> only
> > supports key-value pairs, resulting in poor readability. Secondly, if the
> > value is a collection type, such as a List or Map, users are required to
> > write the value in a FLINK-specific pattern, which is inconvenient to
> use.
> > Additionally, the parser of FLINK has some differences in syntax compared
> > to the standard YAML parser, such as the syntax for parsing comments and
> > null values. These inconsistencies can cause confusion for users, as seen
> > in FLINK-15358 and FLINK-32740.
> >
> > By supporting standard YAML, these issues can be resolved, and users can
> > create a Flink configuration file using third-party tools and leverage
> > some advanced YAML features. Therefore, we propose to support standard
> > YAML for FLINK configuration.
> >
> > You can find more details in the FLIP-366[1]. Looking forward to your
> > feedback.
> >
> > [1]
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-366%3A+Support+standard+YAML+for+FLINK+configuration
> >
> > Best,
> > Junrui
> >
>


[Discuss] FLIP-366: Support standard YAML for FLINK configuration

2023-09-19 Thread Junrui Lee
Hi devs,

I would like to start a discussion about FLIP-366:
Support standard YAML for FLINK configuration[1]

The current flink-conf.yaml parser in FLINK is not a standard YAML parser,
which has some shortcomings.
Firstly, it does not support nested structure configuration items and only
supports key-value pairs, resulting in poor readability. Secondly, if the
value is a collection type, such as a List or Map, users are required to
write the value in a FLINK-specific pattern, which is inconvenient to use.
Additionally, the parser of FLINK has some differences in syntax compared
to the standard YAML parser, such as the syntax for parsing comments and
null values. These inconsistencies can cause confusion for users, as seen
in FLINK-15358 and FLINK-32740.

By supporting standard YAML, these issues can be resolved, and users can
create a Flink configuration file using third-party tools and leverage
some advanced YAML features. Therefore, we propose to support standard
YAML for FLINK configuration.

You can find more details in the FLIP-366[1]. Looking forward to your
feedback.

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-366%3A+Support+standard+YAML+for+FLINK+configuration

Best,
Junrui


Re: [ANNOUNCE] New Apache Flink Committer - Hangxiang Yu

2023-08-07 Thread Junrui Lee
Congratulations, Hangxiang!

Best,
Junrui

Yun Tang  于2023年8月7日周一 15:19写道:

> Congratulations, Hangxiang!
>
> Best
> Yun Tang
> 
> From: Danny Cranmer 
> Sent: Monday, August 7, 2023 15:11
> To: dev 
> Subject: Re: [ANNOUNCE] New Apache Flink Committer - Hangxiang Yu
>
> Congrats Hangxiang! Welcome to the team.
>
> Danny.
>
> On Mon, 7 Aug 2023, 08:04 Rui Fan, <1996fan...@gmail.com> wrote:
>
> > Congratulations Hangxiang!
> >
> > Best,
> > Rui
> >
> > On Mon, Aug 7, 2023 at 2:58 PM Yuan Mei  wrote:
> >
> > > On behalf of the PMC, I'm happy to announce Hangxiang Yu as a new Flink
> > > Committer.
> > >
> > > Hangxiang has been active in the Flink community for more than 1.5
> years
> > > and has played an important role in developing and maintaining State
> and
> > > Checkpoint related features/components, including Generic Incremental
> > > Checkpoints (take great efforts to make the feature prod-ready).
> > Hangxiang
> > > is also the main driver of the FLIP-263: Resolving schema
> compatibility.
> > >
> > > Hangxiang is passionate about the Flink community. Besides the
> technical
> > > contribution above, he is also actively promoting Flink: talks about
> > > Generic
> > > Incremental Checkpoints in Flink Forward and Meet-up. Hangxiang also
> > spent
> > > a good amount of time supporting users, participating in Jira/mailing
> > list
> > > discussions, and reviewing code.
> > >
> > > Please join me in congratulating Hangxiang for becoming a Flink
> > Committer!
> > >
> > > Thanks,
> > > Yuan Mei (on behalf of the Flink PMC)
> > >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Yanfei Lei

2023-08-07 Thread Junrui Lee
Congratulations Yanfei!

Best,
Junrui

Yun Tang  于2023年8月7日周一 15:19写道:

> Congratulations, Yanfei!
>
> Best
> Yun Tang
> 
> From: Danny Cranmer 
> Sent: Monday, August 7, 2023 15:10
> To: dev 
> Subject: Re: [ANNOUNCE] New Apache Flink Committer - Yanfei Lei
>
> Congrats Yanfei! Welcome to the team.
>
> Danny
>
> On Mon, 7 Aug 2023, 08:03 Rui Fan, <1996fan...@gmail.com> wrote:
>
> > Congratulations Yanfei!
> >
> > Best,
> > Rui
> >
> > On Mon, Aug 7, 2023 at 2:56 PM Yuan Mei  wrote:
> >
> > > On behalf of the PMC, I'm happy to announce Yanfei Lei as a new Flink
> > > Committer.
> > >
> > > Yanfei has been active in the Flink community for almost two years and
> > has
> > > played an important role in developing and maintaining State and
> > Checkpoint
> > > related features/components, including RocksDB Rescaling Performance
> > > Improvement and Generic Incremental Checkpoints.
> > >
> > > Yanfei also helps improve community infrastructure in many ways,
> > including
> > > migrating the Flink Daily performance benchmark to the Apache Flink
> slack
> > > channel. She is the maintainer of the benchmark and has improved its
> > > detection stability significantly. She is also one of the major
> > maintainers
> > > of the FrocksDB Repo and released FRocksDB 6.20.3 (part of Flink 1.17
> > > release). Yanfei is a very active community member, supporting users
> and
> > > participating
> > > in tons of discussions on the mailing lists.
> > >
> > > Please join me in congratulating Yanfei for becoming a Flink Committer!
> > >
> > > Thanks,
> > > Yuan Mei (on behalf of the Flink PMC)
> > >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Weihua Hu

2023-08-04 Thread Junrui Lee
Congrats, Weihua!
Best,
Junrui

Geng Biao  于2023年8月4日周五 14:25写道:

> Congrats, Weihua!
> Best,
> Biao Geng
>
> 发送自 Outlook for iOS
> 
> 发件人: 周仁祥 
> 发送时间: Friday, August 4, 2023 2:23:42 PM
> 收件人: dev@flink.apache.org 
> 抄送: Weihua Hu 
> 主题: Re: [ANNOUNCE] New Apache Flink Committer - Weihua Hu
>
> Congratulations, Weihua~
>
> > 2023年8月4日 14:21,Sergey Nuyanzin  写道:
> >
> > Congratulations, Weihua!
> >
> > On Fri, Aug 4, 2023 at 8:03 AM Chen Zhanghao 
> > wrote:
> >
> >> Congratulations, Weihua!
> >>
> >> Best,
> >> Zhanghao Chen
> >> 
> >> 发件人: Xintong Song 
> >> 发送时间: 2023年8月4日 11:18
> >> 收件人: dev 
> >> 抄送: Weihua Hu 
> >> 主题: [ANNOUNCE] New Apache Flink Committer - Weihua Hu
> >>
> >> Hi everyone,
> >>
> >> On behalf of the PMC, I'm very happy to announce Weihua Hu as a new
> Flink
> >> Committer!
> >>
> >> Weihua has been consistently contributing to the project since May
> 2022. He
> >> mainly works in Flink's distributed coordination areas. He is the main
> >> contributor of FLIP-298 and many other improvements in large-scale job
> >> scheduling and improvements. He is also quite active in mailing lists,
> >> participating discussions and answering user questions.
> >>
> >> Please join me in congratulating Weihua!
> >>
> >> Best,
> >>
> >> Xintong (on behalf of the Apache Flink PMC)
> >>
> >
> >
> > --
> > Best regards,
> > Sergey
>
>


Re: [DISCUSS][2.0] FLIP-343: Remove parameter in WindowAssigner#getDefaultTrigger()

2023-07-25 Thread Junrui Lee
+1

Best,
Junrui

weijie guo  于2023年7月24日周一 10:25写道:

> +1 for this.
>
> Best regards,
>
> Weijie
>
>
> liu ron  于2023年7月24日周一 09:58写道:
>
> > +1
> >
> > Best,
> > Ron
> >
> > Yuxin Tan  于2023年7月21日周五 16:21写道:
> >
> > > +1
> > >
> > > Best,
> > > Yuxin
> > >
> > >
> > > Jing Ge  于2023年7月21日周五 15:41写道:
> > >
> > > > +1
> > > >
> > > > NIT: the release in the FLIP is still empty, it should be 2.0
> > > >
> > > > Best regards,
> > > > Jing
> > > >
> > > > On Fri, Jul 21, 2023 at 6:03 AM Xintong Song 
> > > > wrote:
> > > >
> > > > > +1
> > > > >
> > > > > Best,
> > > > >
> > > > > Xintong
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Jul 21, 2023 at 10:53 AM Wencong Liu  >
> > > > wrote:
> > > > >
> > > > > > Hi devs,
> > > > > >
> > > > > > I would like to start a discussion on FLIP-343: Remove parameter
> in
> > > > > > WindowAssigner#getDefaultTrigger() [1].
> > > > > >
> > > > > >
> > > > > > The method getDefaultTrigger() in WindowAssigner takes a
> > > > > > StreamExecutionEnvironment
> > > > > > parameter, but this parameter is not actually used for any
> > subclasses
> > > > of
> > > > > > WindowAssigner.
> > > > > > Therefore, it is unnecessary to include this parameter.
> > > > > > As such I propose to remove the StreamExecutionEnvironment field
> > from
> > > > > > WindowAssigner#getDefaultTrigger(StreamExecutionEnvironment env).
> > > > > > Looking forward to your feedback.
> > > > > > [1]
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263425229
> > > > > > Best regards,
> > > > > >
> > > > > >
> > > > > > Wencong Liu
> > > > >
> > > >
> > >
> >
>


Re: Re: [DISCUSS][2.0] FLIP-344: Remove parameter in RichFunction#open

2023-07-25 Thread Junrui Lee
+1

Best,
Junrui

Wencong Liu  于2023年7月24日周一 20:12写道:

> Hi Timo,
>
>
> Thanks for you reply. I think adding an empty OpenContext to keep the
> signature is
> reasonable. I'll modify the FLIP at a later time.
>
>
> Best,
> Wencong Liu
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
> At 2023-07-24 17:11:44, "Timo Walther"  wrote:
> >+1
> >
> >But instead we should add a OpenContext there to keep the signature
> >stable but still be able to add parameters.
> >
> >Regards,
> >Timo
> >
> >On 21.07.23 12:24, Jing Ge wrote:
> >> +1
> >>
> >> On Fri, Jul 21, 2023 at 10:22 AM Yuxin Tan 
> wrote:
> >>
> >>> +1
> >>>
> >>> Best,
> >>> Yuxin
> >>>
> >>>
> >>> Xintong Song  于2023年7月21日周五 12:04写道:
> >>>
>  +1
> 
>  Best,
> 
>  Xintong
> 
> 
> 
>  On Fri, Jul 21, 2023 at 10:52 AM Wencong Liu 
> >>> wrote:
> 
> > Hi devs,
> >
> > I would like to start a discussion on FLIP-344: Remove parameter in
> > RichFunction#open [1].
> >
> > The open() method in RichFunction requires a Configuration instance
> as
> >>> an
> > argument,
> > which is always passed as a new instance without any configuration
> > parameters in
> > AbstractUdfStreamOperator#open. Thus, it is unnecessary to include
> this
> > parameter
> > in the open() method.
> > As such I propose to remove the Configuration field from
> > RichFunction#open(Configuration parameters).
> > Looking forward to your feedback.
> > [1]
> >
> 
> >>>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263425231
> > Best regards,
> >
> >
> > Wencong Liu
> 
> >>>
> >>
>


Re: Re: Re: [DISCUSS][2.0] FLIP-347: Remove IOReadableWritable serialization in Path

2023-07-25 Thread Junrui Lee
+1

Best,
Junrui

Jing Ge  于2023年7月24日周一 23:28写道:

> agree, since we want to try our best to deprecate APIs in 1.18, it makes
> sense.
>
>
> Best regards,
> Jing
>
> On Mon, Jul 24, 2023 at 12:11 PM Wencong Liu  wrote:
>
> > Hi Jing and Matthias,
> >
> >
> > I believe it is reasonable to examine all classes that implement the
> > IOReadableWritable
> > interface and summarize their actual usage. However, due to time
> > constraints, I suggest
> > we minimize the scope of this FLIP to focus on the Path class. As for
> > other components
> > that implement IOReadableWritable, we can make an effort to investigate
> > them
> > in the future. WDYT?
> >
> >
> > Best regards,
> > Wencong Liu
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> > At 2023-07-22 00:46:45, "Jing Ge"  wrote:
> > >Hi Wencong,
> > >
> > >Thanks for the clarification. I got your point. It makes sense.
> > >
> > >Wrt IOReadableWritable, the suggestion was to check all classes that
> > >implemented it, e.g. BlockInfo, Value, Configuration, etc. Not limited
> to
> > >the Path.
> > >
> > >Best regards,
> > >Jing
> > >
> > >On Fri, Jul 21, 2023 at 4:31 PM Wencong Liu 
> wrote:
> > >
> > >> Hello Jing,
> > >>
> > >>
> > >> Thanks for your reply. The URI field should be final and the
> > >> Path will be immutable.The static method deserializeFromDataInputView
> > >> will create a new Path object instead of replacing the URI field
> > >> in a existed Path Object.
> > >>
> > >>
> > >> For the crossing multiple modules issue, I've explained it in the
> reply
> > >> to Matthias.
> > >>
> > >>
> > >> Best regards,
> > >>
> > >>
> > >> Wencong Liu
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >> At 2023-07-21 18:05:26, "Jing Ge"  wrote:
> > >> >Hi Wencong,
> > >> >
> > >> >Just out of curiosity, will the newly introduced
> > >> >deserializeFromDataInputView() method make the Path mutable again?
> > >> >
> > >> >What Matthias suggested makes sense, although the extension might
> make
> > >> this
> > >> >FLIP cross multiple modules.
> > >> >
> > >> >Best regards,
> > >> >Jing
> > >> >
> > >> >On Fri, Jul 21, 2023 at 10:23 AM Matthias Pohl
> > >> > wrote:
> > >> >
> > >> >> There's a kind-of-related issue FLINK-4758 [1] that proposes
> removing
> > >> the
> > >> >> IOReadableWritable interface from more classes. It was briefly
> > >> mentioned in
> > >> >> the must-have work items discussion [2].
> > >> >>
> > >> >> I'm not too sure about the usage of IOReadableWritable: ...whether
> it
> > >> would
> > >> >> go away with the removal of the DataSet API in general (the Jira
> > issue
> > >> has
> > >> >> DataSet as a component), anyway.
> > >> >>
> > >> >> Otherwise, might it make sense to extend the scope of this FLIP?
> > >> >>
> > >> >> [1] https://issues.apache.org/jira/browse/FLINK-4758
> > >> >> [2]
> https://lists.apache.org/thread/gf0h4gh3xfsj78cpdsxsnj70nhzcmv9r
> > >> >>
> > >> >> On Fri, Jul 21, 2023 at 6:04 AM Xintong Song <
> tonysong...@gmail.com>
> > >> >> wrote:
> > >> >>
> > >> >> > +1
> > >> >> >
> > >> >> > Best,
> > >> >> >
> > >> >> > Xintong
> > >> >> >
> > >> >> >
> > >> >> >
> > >> >> > On Fri, Jul 21, 2023 at 10:54 AM Wencong Liu <
> liuwencle...@163.com
> > >
> > >> >> wrote:
> > >> >> >
> > >> >> > > Hi devs,
> > >> >> > >
> > >> >> > > I would like to start a discussion on FLIP-347: Remove
> > >> >> IOReadableWritable
> > >> >> > > serialization in Path [1].
> > >> >> > >
> > >> >> > >
> > >> >> > > The Path class is currently mutable to support
> IOReadableWritable
> > >> >> > > serialization. However, many parts
> > >> >> > > of the code assume that the Path is immutable. By making the
> Path
> > >> class
> > >> >> > > immutable, we can ensure
> > >> >> > > that paths are stored correctly without the possibility of
> > mutation
> > >> and
> > >> >> > > eliminate the occurrence of subtle errors.
> > >> >> > > As such I propose to modify the Path class to no longer
> implement
> > >> the
> > >> >> > > IOReadableWritable interface.
> > >> >> > > Looking forward to your feedback.
> > >> >> > > [1]
> > >> >> > >
> > >> >> >
> > >> >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-347%3A+Remove+IOReadableWritable+serialization+in+Path
> > >> >> > > Best regards,
> > >> >> > >
> > >> >> > >
> > >> >> > > Wencong Liu
> > >> >> >
> > >> >>
> > >>
> >
>


Re: [ANNOUNCE] New Apache Flink PMC Member - Qingsheng Ren

2023-04-22 Thread Junrui Lee
Congratulations, Qingsheng!

Best,
Junrui

Wencong Liu  于2023年4月23日周日 11:07写道:

> Congratulations, Qingsheng!
>
> Best,
> Wencong LIu
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
> At 2023-04-21 19:47:52, "Jark Wu"  wrote:
> >Hi everyone,
> >
> >We are thrilled to announce that Leonard Xu has joined the Flink PMC!
> >
> >Leonard has been an active member of the Apache Flink community for many
> >years and became a committer in Nov 2021. He has been involved in various
> >areas of the project, from code contributions to community building. His
> >contributions are mainly focused on Flink SQL and connectors, especially
> >leading the flink-cdc-connectors project to receive 3.8+K GitHub stars. He
> >authored 150+ PRs, and reviewed 250+ PRs, and drove several FLIPs (e.g.,
> >FLIP-132, FLIP-162). He has participated in plenty of discussions in the
> >dev mailing list, answering questions about 500+ threads in the
> >user/user-zh mailing list. Besides that, he is community minded, such as
> >being the release manager of 1.17, verifying releases, managing release
> >syncs, etc.
> >
> >Congratulations and welcome Leonard!
> >
> >Best,
> >Jark (on behalf of the Flink PMC)
>
>
>
>
>
>
>
>
>
>
>
>
>
>
> At 2023-04-21 19:50:02, "Jark Wu"  wrote:
> >Hi everyone,
> >
> >We are thrilled to announce that Qingsheng Ren has joined the Flink PMC!
> >
> >Qingsheng has been contributing to Apache Flink for a long time. He is the
> >core contributor and maintainer of the Kafka connector and
> >flink-cdc-connectors, bringing users stability and ease of use in both
> >projects. He drove discussions and implementations in FLIP-221, FLIP-288,
> >and the connector testing framework. He is continuously helping with the
> >expansion of the Flink community and has given several talks about Flink
> >connectors at many conferences, such as Flink Forward Global and Flink
> >Forward Asia. Besides that, he is willing to help a lot in the community
> >work, such as being the release manager for both 1.17 and 1.18, verifying
> >releases, and answering questions on the mailing list.
> >
> >Congratulations and welcome Qingsheng!
> >
> >Best,
> >Jark (on behalf of the Flink PMC)
>


Re: [ANNOUNCE] New Apache Flink PMC Member - Leonard Xu

2023-04-22 Thread Junrui Lee
Congratulations, Leonard!

Best,
Junrui

Wencong Liu  于2023年4月23日周日 11:05写道:

> Congratulations, Leonard!
>
> Best,
> Wencong LIu
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
> At 2023-04-21 19:47:52, "Jark Wu"  wrote:
> >Hi everyone,
> >
> >We are thrilled to announce that Leonard Xu has joined the Flink PMC!
> >
> >Leonard has been an active member of the Apache Flink community for many
> >years and became a committer in Nov 2021. He has been involved in various
> >areas of the project, from code contributions to community building. His
> >contributions are mainly focused on Flink SQL and connectors, especially
> >leading the flink-cdc-connectors project to receive 3.8+K GitHub stars. He
> >authored 150+ PRs, and reviewed 250+ PRs, and drove several FLIPs (e.g.,
> >FLIP-132, FLIP-162). He has participated in plenty of discussions in the
> >dev mailing list, answering questions about 500+ threads in the
> >user/user-zh mailing list. Besides that, he is community minded, such as
> >being the release manager of 1.17, verifying releases, managing release
> >syncs, etc.
> >
> >Congratulations and welcome Leonard!
> >
> >Best,
> >Jark (on behalf of the Flink PMC)
>


Re: [VOTE] FLIP-301: Hybrid Shuffle supports Remote Storage

2023-03-20 Thread Junrui Lee
+1 (non-binding)

Best regards,
Junrui

Weihua Hu  于2023年3月20日周一 14:24写道:

> +1 (non-binding)
>
> Best,
> Weihua
>
>
> On Mon, Mar 20, 2023 at 12:39 PM Wencong Liu  wrote:
>
> > +1 (non-binding)
> >
> > Best regards,
> >
> > Wencong Liu
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> > At 2023-03-20 12:05:47, "Yuxin Tan"  wrote:
> > >Hi, everyone,
> > >
> > >Thanks for all your feedback for FLIP-301: Hybrid Shuffle
> > >supports Remote Storage[1] on the discussion thread[2].
> > >
> > >I'd like to start a vote for it. The vote will be open for at
> > >least 72 hours (03/23, 13:00 UTC+8) unless there is an
> > >objection or not enough votes.
> > >
> > >[1]
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-301%3A+Hybrid+Shuffle+supports+Remote+Storage
> > >[2] https://lists.apache.org/thread/nwrqd5jtqwks89tbxpcrgto6r2bhdhno
> > >
> > >Best,
> > >Yuxin
> >
>


Re: [VOTE] Release 1.17.0, release candidate #2

2023-03-15 Thread Junrui Lee
Thanks Qingsheng for driving this.
+1 (non-binding)

- built from source code
- run some batch jobs, query results are expected, the default batch
scheduler is AdaptiveBatchScheduler, WebUI looks
good,  no suspicious output/log.
- start a Flink yarn session cluster, run the 10TB tpc-ds, all job can
finish without any exception by AdaptiveBatchScheduler, only configuring
parallelism.default enables operators whose parallelism is not specified by
the user to automatically derive the parallelism based on the amount of
data, without being limited by the power of 2.

Best,
Junrui

Etienne Chauchot  于2023年3月14日周二 19:00写道:

> Hi all,
>
> As promised, I ran the same tests on 1.17.0 RC2 I also verified the
> release notes.
>
> Based on the scope of these tests : +1 (non-binding)
>
> Etienne
>
> Le 14/03/2023 à 07:44, Qingsheng Ren a écrit :
> > Hi everyone,
> >
> > Please review and vote on the release candidate #2 for the version
> > 1.17.0, as follows:
> > [ ] +1, Approve the release
> > [ ] -1, Do not approve the release (please provide specific comments)
> >
> > The complete staging area is available for your review, which includes:
> >
> > * JIRA release notes [1], and the pull request adding release note for
> > users [2]
> > * the official Apache source release and binary convenience releases to
> be
> > deployed to dist.apache.org [3], which are signed with the key with
> > fingerprint A1BD477F79D036D2C30CA7DBCA8AEEC2F6EB040B [4],
> > * all artifacts to be deployed to the Maven Central Repository [5],
> > * source code tag "release-1.17.0-rc2" [6],
> > * website pull request listing the new release and adding announcement
> blog
> > post [7].
> >
> > The vote will be open for at least 72 hours. It is adopted by majority
> > approval, with at least 3 PMC affirmative votes.
> >
> > [1]
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12351585
> > [2] https://github.com/apache/flink/pull/22146
> > [3] https://dist.apache.org/repos/dist/dev/flink/flink-1.17.0-rc2/
> > [4] https://dist.apache.org/repos/dist/release/flink/KEYS
> > [5]
> https://repository.apache.org/content/repositories/orgapacheflink-1595
> > [6] https://github.com/apache/flink/releases/tag/release-1.17.0-rc2
> > [7] https://github.com/apache/flink-web/pull/618
> >
> > Thanks,
> > Martijn and Matthias, Leonard and Qingsheng
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Yuxia Luo

2023-03-12 Thread Junrui Lee
Congratulations, Yuxia!

Best,
Junrui

Yanfei Lei  于2023年3月13日周一 10:42写道:

> Congratulations, Yuxia!
>
> Best,
> Yanfei
>
>
> Samrat Deb  于2023年3月13日周一 10:41写道:
> >
> > congratulations Yuxia
> >
> > Bests,
> > Samrat
> >
> > On Mon, 13 Mar 2023 at 8:06 AM, Yuxin Tan 
> wrote:
> >
> > > Congratulations, Yuxia!
> > >
> > > Best,
> > > Yuxin
> > >
> > >
> > > Jark Wu  于2023年3月13日周一 10:26写道:
> > >
> > > > Hi, everyone
> > > >
> > > > On behalf of the PMC, I'm very happy to announce Yuxia Luo as a new
> Flink
> > > > Committer.
> > > >
> > > > Yuxia has been continuously contributing to the Flink project for
> almost
> > > > two
> > > > years, authored and reviewed hundreds of PRs over this time. He is
> > > > currently
> > > > the core maintainer of the Hive component, where he contributed many
> > > > valuable
> > > > features, including the Hive dialect with 95% compatibility and small
> > > file
> > > > compaction.
> > > > In addition, Yuxia driven FLIP-282 (DELETE & UPDATE API) to better
> > > > integrate
> > > > Flink with data lakes. He actively participated in dev discussions
> and
> > > > answered
> > > > many questions on the user mailing list.
> > > >
> > > > Please join me in congratulating Yuxia Luo for becoming a Flink
> > > Committer!
> > > >
> > > > Best,
> > > > Jark Wu (on behalf of the Flink PMC)
> > > >
> > >
>


Re: [DISCUSS] FLIP-301: Hybrid Shuffle supports Remote Storage

2023-03-07 Thread Junrui Lee
Hi Yuxin,

This FLIP looks quite reasonable. Flink can solve the problem of Batch
shuffle by
combining local and remote storage, and can use fixed local disks for
better performance
 in most scenarios, while using remote storage as a supplement when local
disks are not
 sufficient, avoiding wasteful costs and poor job stability. Moreover, the
solution also
considers the issue of dynamic switching, which can automatically switch to
remote
storage when the local disk is full, saving costs, and automatically switch
back when
there is available space on the local disk.

As Wencong Liu stated, an appropriate segment size is essential, as it can
significantly
affect shuffle performance. I also agree that the first version should
focus mainly on the
design and implementation. However, I have a small question about FLIP. I
did not see
any information regarding the segment size of memory, local disk, and
remote storage
in this FLIP. Are these three values fixed at present? If they are fixed, I
suggest that FLIP
could provide clearer explanations. Moreover, although a dynamic segment
size
mechanism is not necessary at the moment, can we provide configuration
options for users
 to manually adjust these sizes? I think it might be useful.

Best,
Junrui.

Yuxin Tan  于2023年3月7日周二 20:14写道:

> Thanks for joining the discussion.
>
> @weijie guo
> > 1. How to optimize the broadcast result partition?
> For the partitions with multi-consumers, e.g., broadcast result partition,
> partition reuse,
> speculative, etc, the processing logic is the same as the original Hybrid
> Shuffle, that is,
> using the full spilling strategy. It indeed may reduce the opportunity to
> consume from
> memory, but the PoC shows that it has no effect on the performance
> basically.
>
> > 2. Can the new proposal completely avoid this problem of inaccurate
> backlog
> calculation?
> Yes, this can avoid the problem completely. About the read buffers, the N
> is to reserve
> one exclusive buffer per channel, which is to avoid the deadlock because
> the buffers
> are acquired by some channels and other channels can not request any
> buffers. But
> the buffers except for the N can be floating (competing to request the
> buffers) by all
> channels.
>
> @Wencong Liu
> > Deciding the Segment size dynamically will be helpful.
> I agree that it may be better if the segment size is dynamically decided,
> but for simplifying
> the implementation of the first version, we want to make this a fixed value
> for each tier.
> In the future, this can be a good improvement if necessary. In the first
> version, we will mainly
> focus on the more important features, such as the tiered storage
> architecture, dynamic
> switching tiers, supporting remote storage, memory management, etc.
>
> Best,
> Yuxin
>
>
> Wencong Liu  于2023年3月7日周二 16:48写道:
>
> > Hello Yuxin,
> >
> >
> > Thanks for your proposal! Adding remote storage capability to Flink's
> > Hybrid Shuffle is a significant improvement that addresses the issue of
> > local disk storage limitations. This enhancement not only ensures
> > uninterrupted Shuffle, but also enables Flink to handle larger workloads
> > and more complex data processing tasks. With the ability to seamlessly
> > shift between local and remote storage, Flink's Hybrid Shuffle will be
> more
> > versatile and scalable, making it an ideal choice for organizations
> looking
> > to build distributed data processing applications with ease.
> > Besides, I've a small question about the size of Segment in different
> > storages. According to the FLIP, the size of Segment may be fixed for
> each
> > Storage Tier, but I think the fixed size may affect the shuffle
> > performance. For example, smaller segment size will improve the
> utilization
> > rate of Memory Storage Tier, but it may brings extra cost to Disk Storage
> > Tier or Remote Storage Tier. Deciding the size of Segment dynamicly will
> be
> > helpful.
> >
> > Best,
> >
> >
> > Wencong Liu
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> > At 2023-03-06 13:51:21, "Yuxin Tan"  wrote:
> > >Hi everyone,
> > >
> > >I would like to start a discussion on FLIP-301: Hybrid Shuffle supports
> > >Remote Storage[1].
> > >
> > >In the cloud-native environment, it is difficult to determine the
> > >appropriate
> > >disk space for Batch shuffle, which will affect job stability.
> > >
> > >This FLIP is to support Remote Storage for Hybrid Shuffle to improve the
> > >Batch job stability in the cloud-native environment.
> > >
> > >The goals of this FLIP are as follows.
> > >1. By default, use the local memory and disk to ensure high shuffle
> > >performance if the local storage space is sufficient.
> > >2. When the local storage space is insufficient, use remote storage as
> > >a supplement to avoid large-scale Batch job failure.
> > >
> > >Looking forward to hearing from you.
> > >
> > >[1]
> > >
> >
> 

Re: [VOTE] Flink minor version support policy for old releases

2023-02-28 Thread Junrui Lee
Thanks Danny for driving it.

+1 (non-binding)

Best regards,
Junrui

yuxia  于2023年2月28日周二 14:04写道:

> Thanks Danny for driving it.
>
> +1 (non-binding)
>
> Best regards,
> Yuxia
>
> - 原始邮件 -
> 发件人: "Weihua Hu" 
> 收件人: "dev" 
> 发送时间: 星期二, 2023年 2 月 28日 下午 12:48:09
> 主题: Re: [VOTE] Flink minor version support policy for old releases
>
> Thanks, Danny.
>
> +1 (non-binding)
>
> Best,
> Weihua
>
>
> On Tue, Feb 28, 2023 at 12:38 PM weijie guo 
> wrote:
>
> > Thanks Danny for bring this.
> >
> > +1 (non-binding)
> >
> > Best regards,
> >
> > Weijie
> >
> >
> > Jing Ge  于2023年2月27日周一 20:23写道:
> >
> > > +1 (non-binding)
> > >
> > > BTW, should we follow the content style [1] to describe the new rule
> > using
> > > 1.2.x, 1.1.y, 1.1.z?
> > >
> > > [1] https://flink.apache.org/downloads/#update-policy-for-old-releases
> > >
> > > Best regards,
> > > Jing
> > >
> > > On Mon, Feb 27, 2023 at 1:06 PM Matthias Pohl
> > >  wrote:
> > >
> > > > Thanks, Danny. Sounds good to me.
> > > >
> > > > +1 (non-binding)
> > > >
> > > > On Wed, Feb 22, 2023 at 10:11 AM Danny Cranmer <
> > dannycran...@apache.org>
> > > > wrote:
> > > >
> > > > > I am starting a vote to update the "Update Policy for old releases"
> > [1]
> > > > to
> > > > > include additional bugfix support for end of life versions.
> > > > >
> > > > > As per the discussion thread [2], the change we are voting on is:
> > > > > - Support policy: updated to include: "Upon release of a new Flink
> > > minor
> > > > > version, the community will perform one final bugfix release for
> > > resolved
> > > > > critical/blocker issues in the Flink minor version losing support."
> > > > > - Release process: add a step to start the discussion thread for
> the
> > > > final
> > > > > patch version, if there are resolved critical/blocking issues to
> > flush.
> > > > >
> > > > > Voting schema: since our bylaws [3] do not cover this particular
> > > > scenario,
> > > > > and releases require PMC involvement, we will use a consensus vote
> > with
> > > > PMC
> > > > > binding votes.
> > > > >
> > > > > Thanks,
> > > > > Danny
> > > > >
> > > > > [1]
> > > >
> https://flink.apache.org/downloads.html#update-policy-for-old-releases
> > > > > [2]
> https://lists.apache.org/thread/szq23kr3rlkm80rw7k9n95js5vqpsnbv
> > > > > [3] https://cwiki.apache.org/confluence/display/FLINK/Flink+Bylaws
> > > > >
> > > >
> > >
> >
>


Re: [ANNOUNCE] Flink project website is now powered by Hugo

2023-02-23 Thread Junrui Lee
Thanks Martijn for your great work.

Best regards,
Junrui

Jane Chan  于2023年2月24日周五 10:37写道:

> Thanks Martijn for your great work.
>
> Best regards,
> Jane
>
> On Fri, Feb 24, 2023 at 10:24 AM weijie guo 
> wrote:
>
> > Thanks Martijn for your great work.
> >
> > Best regards,
> >
> > Weijie
> >
> >
> > Jingsong Li  于2023年2月24日周五 09:49写道:
> >
> > > Thanks Martijn!
> > >
> > > On Fri, Feb 24, 2023 at 9:46 AM yuxia 
> > wrote:
> > > >
> > > > Thanks Martijn for your work.
> > > >
> > > > Best regards,
> > > > Yuxia
> > > >
> > > > - 原始邮件 -
> > > > 发件人: "Jing Ge" 
> > > > 收件人: "dev" 
> > > > 发送时间: 星期五, 2023年 2 月 24日 上午 5:20:30
> > > > 主题: Re: [ANNOUNCE] Flink project website is now powered by Hugo
> > > >
> > > > Congrats Martijn! You have made great progress. Thanks for your
> effort!
> > > >
> > > > Best regards,
> > > > Jing
> > > >
> > > > On Thu, Feb 23, 2023 at 8:47 PM Konstantin Knauf 
> > > wrote:
> > > >
> > > > > Thanks, Martijn. That was a lot of work.
> > > > >
> > > > > Am Do., 23. Feb. 2023 um 16:33 Uhr schrieb Maximilian Michels <
> > > > > m...@apache.org>:
> > > > >
> > > > > > Congrats! Great work. This was a long time in the making!
> > > > > >
> > > > > > -Max
> > > > > >
> > > > > > On Thu, Feb 23, 2023 at 3:28 PM Martijn Visser <
> > > martijnvis...@apache.org
> > > > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > Hi everyone,
> > > > > > >
> > > > > > > The project website at https://flink.apache.org is now powered
> > by
> > > Hugo
> > > > > > [1]
> > > > > > > which is the same system as the documentation.
> > > > > > >
> > > > > > > The theme is the same as the documentation website, so there's
> no
> > > > > > redesign
> > > > > > > involved.
> > > > > > >
> > > > > > > If you encounter any issues, please create a Jira ticket and
> feel
> > > free
> > > > > to
> > > > > > > ping me in it.
> > > > > > >
> > > > > > > Thanks to all that have been involved with testing and
> reviewing!
> > > > > > >
> > > > > > > Best regards,
> > > > > > >
> > > > > > > Martijn
> > > > > > >
> > > > > > > [1] https://issues.apache.org/jira/browse/FLINK-22922
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > https://twitter.com/snntrable
> > > > > https://github.com/knaufk
> > > > >
> > >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Sergey Nuyanzin

2023-02-21 Thread Junrui Lee
Congratulations Sergey!

weijie guo  于2023年2月21日周二 20:54写道:

> Congratulations, Sergey~
>
> Best regards,
>
> Weijie
>
>
> Jing Ge  于2023年2月21日周二 20:52写道:
>
> > congrats Sergey!
> >
> > On Tue, Feb 21, 2023 at 1:15 PM Matthias Pohl
> >  wrote:
> >
> > > Congratulations, Sergey! Good job & well-deserved! :)
> > >
> > > On Tue, Feb 21, 2023 at 1:03 PM yuxia 
> > wrote:
> > >
> > > > Congratulations Sergey!
> > > >
> > > > Best regards,
> > > > Yuxia
> > > >
> > > > - 原始邮件 -
> > > > 发件人: "Martijn Visser" 
> > > > 收件人: "dev" 
> > > > 发送时间: 星期二, 2023年 2 月 21日 下午 7:58:35
> > > > 主题: Re: [ANNOUNCE] New Apache Flink Committer - Sergey Nuyanzin
> > > >
> > > > Congrats Sergey, well deserved :)
> > > >
> > > > On Tue, Feb 21, 2023 at 12:53 PM Benchao Li 
> > > wrote:
> > > >
> > > > > Congratulations Sergey!
> > > > >
> > > > > Timo Walther  于2023年2月21日周二 19:51写道:
> > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > > On behalf of the PMC, I'm very happy to announce Sergey Nuyanzin
> > as a
> > > > > > new Flink Committer.
> > > > > >
> > > > > > Sergey started contributing small improvements to the project in
> > > 2018.
> > > > > > Over the past 1.5 years, he has become more active and focused on
> > > > adding
> > > > > > and reviewing changes to the Flink SQL ecosystem.
> > > > > >
> > > > > > Currently, he is upgrading Flink's SQL engine to the latest
> Apache
> > > > > > Calcite version [1][2][3] and helps in updating other
> project-wide
> > > > > > dependencies as well.
> > > > > >
> > > > > > Please join me in congratulating Sergey Nuyanzin for becoming a
> > Flink
> > > > > > Committer!
> > > > > >
> > > > > > Best,
> > > > > > Timo Walther (on behalf of the Flink PMC)
> > > > > >
> > > > > > [1] https://issues.apache.org/jira/browse/FLINK-29932
> > > > > > [2] https://issues.apache.org/jira/browse/FLINK-21239
> > > > > > [3] https://issues.apache.org/jira/browse/FLINK-20873
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > >
> > > > > Best,
> > > > > Benchao Li
> > > > >
> > > >
> > >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Rui Fan

2023-02-20 Thread Junrui Lee
Congratulations, Rui

Best,
Junrui

Matt Wang  于2023年2月21日周二 09:51写道:

> Congrats Rui
>
>
> --
>
> Best,
> Matt Wang
>
>
>  Replied Message 
> | From | yuxia |
> | Date | 02/21/2023 09:22 |
> | To | dev |
> | Subject | Re: [ANNOUNCE] New Apache Flink Committer - Rui Fan |
> Congrats Rui
>
> Best regards,
> Yuxia
>
> - 原始邮件 -
> 发件人: "Samrat Deb" 
> 收件人: "dev" 
> 发送时间: 星期二, 2023年 2 月 21日 上午 1:09:25
> 主题: Re: [ANNOUNCE] New Apache Flink Committer - Rui Fan
>
> Congrats Rui
>
> On Mon, 20 Feb 2023 at 10:28 PM, Anton Kalashnikov 
> wrote:
>
> Congrats Rui!
>
> --
> Best regards,
> Anton Kalashnikov
>
> On 20.02.23 17:53, Matthias Pohl wrote:
> Congratulations, Rui :)
>
> On Mon, Feb 20, 2023 at 5:10 PM Jing Ge 
> wrote:
>
> Congrats Rui!
>
> On Mon, Feb 20, 2023 at 3:19 PM Piotr Nowojski 
> wrote:
>
> Hi, everyone
>
> On behalf of the PMC, I'm very happy to announce Rui Fan as a new Flink
> Committer.
>
> Rui Fan has been active on a small scale since August 2019, and ramped
> up
> his contributions in the 2nd half of 2021. He was mostly involved in
> quite
> demanding performance related work around the network stack and
> checkpointing, like re-using TCP connections [1], and many crucial
> improvements to the unaligned checkpoints. Among others: FLIP-227:
> Support
> overdraft buffer [2], Merge small ChannelState file for Unaligned
> Checkpoint [3], Timeout aligned to unaligned checkpoint barrier in the
> output buffers [4].
>
> Please join me in congratulating Rui Fan for becoming a Flink
> Committer!
>
> Best,
> Piotr Nowojski (on behalf of the Flink PMC)
>
> [1] https://issues.apache.org/jira/browse/FLINK-22643
> [2]
>
>
>
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-227%3A+Support+overdraft+buffer
> [3] https://issues.apache.org/jira/browse/FLINK-26803
> [4] https://issues.apache.org/jira/browse/FLINK-27251
>
>
>


Re: [ANNOUNCE] New Apache Flink Committer - Anton Kalashnikov

2023-02-20 Thread Junrui Lee
Congratulations, Anton!

Best,
Junrui

Rui Fan  于2023年2月21日周二 10:02写道:

> Congratulations, Anton!
>
> Best,
> Rui Fan
>
> On Tue, Feb 21, 2023 at 9:23 AM yuxia  wrote:
>
> > Congrats Anton!
> >
> > Best regards,
> > Yuxia
> >
> > - 原始邮件 -
> > 发件人: "Matthias Pohl" 
> > 收件人: "dev" 
> > 发送时间: 星期二, 2023年 2 月 21日 上午 12:52:40
> > 主题: Re: [ANNOUNCE] New Apache Flink Committer - Anton Kalashnikov
> >
> > Congratulations, Anton! :-)
> >
> > On Mon, Feb 20, 2023 at 5:09 PM Jing Ge 
> > wrote:
> >
> > > Congrats Anton!
> > >
> > > On Mon, Feb 20, 2023 at 5:02 PM Samrat Deb 
> > wrote:
> > >
> > > > congratulations Anton!
> > > >
> > > > Bests,
> > > > Samrat
> > > >
> > > > On Mon, 20 Feb 2023 at 9:29 PM, John Roesler 
> > > wrote:
> > > >
> > > > > Congratulations, Anton!
> > > > > -John
> > > > >
> > > > > On Mon, Feb 20, 2023, at 08:18, Piotr Nowojski wrote:
> > > > > > Hi, everyone
> > > > > >
> > > > > > On behalf of the PMC, I'm very happy to announce Anton
> Kalashnikov
> > > as a
> > > > > new
> > > > > > Flink Committer.
> > > > > >
> > > > > > Anton has been very active for almost two years already, authored
> > and
> > > > > > reviewed many PRs over this time. He is active in the Flink's
> > > runtime,
> > > > > > being the main author of improvements like Buffer Debloating
> > > (FLIP-183)
> > > > > > [1], solved many bugs and fixed many test instabilities,
> generally
> > > > > speaking
> > > > > > helping with the maintenance of runtime components.
> > > > > >
> > > > > > Please join me in congratulating Anton Kalashnikov for becoming a
> > > Flink
> > > > > > Committer!
> > > > > >
> > > > > > Best,
> > > > > > Piotr Nowojski (on behalf of the Flink PMC)
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-183%3A+Dynamic+buffer+size+adjustment
> > > > >
> > > >
> > >
> >
>


Re: [ANNOUNCE] New Apache Flink PMC Member - Dong Lin

2023-02-15 Thread Junrui Lee
Congratulations, Dong!

Best,
Junrui

Guowei Ma  于2023年2月16日周四 14:21写道:

> Hi, everyone
>
> On behalf of the PMC, I'm very happy to announce Dong Lin as a new
> Flink PMC.
>
> Dong is currently the main driver of Flink ML. He reviewed a large
> number of Flink ML related PRs and also participated in many Flink ML
> improvements, such as "FLIP-173","FLIP-174" etc. At the same time, he made
> a lot of evangelism events contributions for the Flink ML ecosystem.
> In fact, in addition to the Flink machine learning field, Dong has also
> participated in many other improvements in Flink, such as "FLIP-205",
> "FLIP-266","FLIP-269","FLIP-274" etc.
> Please join me in congratulating Dong Lin for becoming a Flink PMC!
>
> Best,
> Guowei(on behalf of the Flink PMC)
>


Re: [ANNOUNCE] New Apache Flink Committer - Weijie Guo

2023-02-12 Thread Junrui Lee
Congratulations, Weijie!

Best,
Junrui


[RESULT][VOTE] FLIP-283: Use adaptive batch scheduler as default scheduler for batch jobs

2023-01-12 Thread Junrui Lee
Hi all:
FLIP-283: Use adaptive batch scheduler as default scheduler for batch jobs
[1] has been accepted. The FLIP was voted in this thread[2].

There are 3 bindings, and 2 non-bindings as follows:

Lijie Wang  (binding)
Zhu Zhu (binding)
yuxia (non-binding)
weijie guo (non-binding)
Xintong Song (binding)
There are no disapproving votes.

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-283%3A+Use+adaptive+batch+scheduler+as+default+scheduler+for+batch+jobs
[2] https://lists.apache.org/thread/gdymm7pr2slzy9gqkfo97vn73496w0cj

Best regards,
Junrui


Re: [DISCUSS] FLIP-283: Use adaptive batch scheduler as default scheduler for batch jobs

2023-01-10 Thread Junrui Lee
Hi, all. I have started voting[1] for this FLIP. If you have any questions,
you can reply in this discussion thread.

[1]: https://lists.apache.org/thread/gdymm7pr2slzy9gqkfo97vn73496w0cj

Best regards,
Junrui

Xintong Song  于2023年1月4日周三 18:16写道:

> Thanks for the proposal.
>
> Another potential benefit I see in this FLIP is that it may reduce the
> complexity and maintenance overhead of the scheduler. During developing
> hybrid shuffle, we had to re-implement some similar logic to make both
> default and adaptive batch schedulers support the new shuffle type. For
> batch workloads, the functionality provided by adaptive batch scheduler
> should already be a superset of those provided by the default scheduler. If
> that can be confirmed from user feedback after making the adaptive batch
> scheduler default, we may consider making default scheduler streaming
> dedicated. That should significantly simplify the default scheduler,
> benefiting both the maintenance of the default scheduler and the
> development of new features in the batch area.
>
> So +1 from my side.
>
> Best,
>
> Xintong
>
>
>
> On Wed, Jan 4, 2023 at 1:14 PM JunRui Lee  wrote:
>
> > Hi, Yuxia
> >
> > Thanks for the reply.
> >
> > > 1. About the configuration renaming. The old configurations ...
> >
> > Thanks for your suggestion. And this explanation has been updated
> > to the FLIP to describe this change more clearly. The old configurations
> > will be deprecated after the configuration renaming, and it still will be
> > considered to guarantee compatibility.
> >
> > > 2. I'm cursion in which case users will disable auto parallelism...
> >
> > Actually, the adaptive batch scheduler is not only used for automatic
> > parallelism derivation, but also for speculative execution. If users
> > want to use speculative execution and do not enable automatic
> > parallelism, they can configure the configuration
> > "execution.batch.adaptive.auto-parallelism.enabled" instead of
> > configuring the configuration "jobmanager.scheduler".
> >
> > In addition, “adaptive” represents a function, not just the automatic
> > derivation of parallelism, and more functions may be added in the
> > future. So we want users to focus on enabling or disabling the
> > parallelism derivation function itself rather than replacing the
> > scheduler type.
> >
> > Best,
> > JunRui
> >
> > Lijie Wang  于2023年1月3日周二 11:03写道:
> >
> > > Hi Junrui,
> > >
> > > Thanks for driving this FLIP, + 1 for this proposal. I believe it will
> > > greatly improve the experiences of batch users.
> > >
> > > Best,
> > > Lijie
> > >
> > > Zhu Zhu  于2022年12月30日周五 12:40写道:
> > >
> > > > Hi Junrui,
> > > >
> > > > Thanks for creating this FLIP!
> > > >
> > > > AdaptiveBatchScheduler is more powerful than DefaultScheduler in
> batch
> > > > scheduling, also with some must-have features like speculative
> > execution.
> > > > It will be great that users can easily use it, without required to
> > > knowing
> > > > the underlying scheduler and configuring some advanced items.
> > > >
> > > > So generally +1 for this proposal.
> > > >
> > > > Regarding the configuration key renaming, like yuxia mentioned, we
> > should
> > > > deprecate the old ones and add new ones with new names, to guarantee
> > > > compatibility.
> > > >
> > > > Thanks,
> > > > Zhu
> > > >
> > > > yuxia  于2022年12月30日周五 11:10写道:
> > > > >
> > > > > Hi, JunRui Lee.
> > > > >
> > > > > Thanks for driving this FLIP. It must a good improvement for batch
> > > > users' experiences.
> > > > > I have few questions about this FLIP:
> > > > > 1: About the configuration renaming. The old configurations will be
> > > > deprecated or removed directly? if user upgrade their Flink version,
> > > these
> > > > old configuration will still be considered or just ignored?  If
> ignore,
> > > the
> > > > users may need to modify their configurations after they upgrade
> their
> > > > Flink.
> > > > >
> > > > > 2: I'm cursion in which case users will disable auto parallelism
> > > > derivation if they have enabled adaptive batch scheduler.  IIUC, auto
> > > > parallelism derivation is what adaptive batch sched

Re: Re: [ANNOUNCE] New Apache Flink Committer - Lincoln Lee

2023-01-09 Thread Junrui Lee
Congratulations, Lincoln!

Best regards,
Junrui

Wencong Liu  于2023年1月10日周二 14:40写道:

> Congratulations, Lincoln!
>
> Best regards,
> Wencong
>
>
>
>
>
>
>
>
>
>
>
>
>
>
> 在 2023-01-10 13:25:09,"Yanfei Lei"  写道:
> >Congratulations, well deserved!
> >
> >Best,
> >Yanfei
> >
> >Yuan Mei  于2023年1月10日周二 13:16写道:
> >
> >> Congratulations, Lincoln!
> >>
> >> Best,
> >> Yuan
> >>
> >> On Tue, Jan 10, 2023 at 12:23 PM Lijie Wang 
> >> wrote:
> >>
> >> > Congratulations, Lincoln!
> >> >
> >> > Best,
> >> > Lijie
> >> >
> >> > Jingsong Li  于2023年1月10日周二 12:07写道:
> >> >
> >> > > Congratulations, Lincoln!
> >> > >
> >> > > Best,
> >> > > Jingsong
> >> > >
> >> > > On Tue, Jan 10, 2023 at 11:56 AM Leonard Xu 
> wrote:
> >> > > >
> >> > > > Congratulations, Lincoln!
> >> > > >
> >> > > > Impressive work in streaming semantics, well deserved!
> >> > > >
> >> > > >
> >> > > > Best,
> >> > > > Leonard
> >> > > >
> >> > > >
> >> > > > > On Jan 10, 2023, at 11:52 AM, Jark Wu  wrote:
> >> > > > >
> >> > > > > Hi everyone,
> >> > > > >
> >> > > > > On behalf of the PMC, I'm very happy to announce Lincoln Lee as
> a
> >> new
> >> > > Flink
> >> > > > > committer.
> >> > > > >
> >> > > > > Lincoln Lee has been a long-term Flink contributor since 2017.
> He
> >> > > mainly
> >> > > > > works on Flink
> >> > > > > SQL parts and drives several important FLIPs, e.g., FLIP-232
> (Retry
> >> > > Async
> >> > > > > I/O), FLIP-234 (
> >> > > > > Retryable Lookup Join), FLIP-260 (TableFunction Finish).
> Besides,
> >> He
> >> > > also
> >> > > > > contributed
> >> > > > > much to Streaming Semantics, including the non-determinism
> problem
> >> > and
> >> > > the
> >> > > > > message
> >> > > > > ordering problem.
> >> > > > >
> >> > > > > Please join me in congratulating Lincoln for becoming a Flink
> >> > > committer!
> >> > > > >
> >> > > > > Cheers,
> >> > > > > Jark Wu
> >> > > >
> >> > >
> >> >
> >>
>


[VOTE] FLIP-283: Use adaptive batch scheduler as default scheduler for batch jobs

2023-01-09 Thread Junrui Lee
Hi all,

Thanks for all the feedback so far.
Based on the discussion[1], we have come to a consensus,
so I would like to start a vote on FLIP-283: Use adaptive
batch scheduler as default scheduler for batch jobs[2].

The vote will last for at least 72 hours (Jan 12th at 12:00 GMT)
unless there is an objection or insufficient votes.

[1] https://lists.apache.org/thread/qwrh22do8scghz79vy852pqx2ny4jqv6
[2]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-283%3A+Use+adaptive+batch+scheduler+as+default+scheduler+for+batch+jobs


Re: [DISCUSS] FLIP-283: Use adaptive batch scheduler as default scheduler for batch jobs

2023-01-03 Thread JunRui Lee
Hi, Yuxia

Thanks for the reply.

> 1. About the configuration renaming. The old configurations ...

Thanks for your suggestion. And this explanation has been updated
to the FLIP to describe this change more clearly. The old configurations
will be deprecated after the configuration renaming, and it still will be
considered to guarantee compatibility.

> 2. I'm cursion in which case users will disable auto parallelism...

Actually, the adaptive batch scheduler is not only used for automatic
parallelism derivation, but also for speculative execution. If users
want to use speculative execution and do not enable automatic
parallelism, they can configure the configuration
"execution.batch.adaptive.auto-parallelism.enabled" instead of
configuring the configuration "jobmanager.scheduler".

In addition, “adaptive” represents a function, not just the automatic
derivation of parallelism, and more functions may be added in the
future. So we want users to focus on enabling or disabling the
parallelism derivation function itself rather than replacing the
scheduler type.

Best,
JunRui

Lijie Wang  于2023年1月3日周二 11:03写道:

> Hi Junrui,
>
> Thanks for driving this FLIP, + 1 for this proposal. I believe it will
> greatly improve the experiences of batch users.
>
> Best,
> Lijie
>
> Zhu Zhu  于2022年12月30日周五 12:40写道:
>
> > Hi Junrui,
> >
> > Thanks for creating this FLIP!
> >
> > AdaptiveBatchScheduler is more powerful than DefaultScheduler in batch
> > scheduling, also with some must-have features like speculative execution.
> > It will be great that users can easily use it, without required to
> knowing
> > the underlying scheduler and configuring some advanced items.
> >
> > So generally +1 for this proposal.
> >
> > Regarding the configuration key renaming, like yuxia mentioned, we should
> > deprecate the old ones and add new ones with new names, to guarantee
> > compatibility.
> >
> > Thanks,
> > Zhu
> >
> > yuxia  于2022年12月30日周五 11:10写道:
> > >
> > > Hi, JunRui Lee.
> > >
> > > Thanks for driving this FLIP. It must a good improvement for batch
> > users' experiences.
> > > I have few questions about this FLIP:
> > > 1: About the configuration renaming. The old configurations will be
> > deprecated or removed directly? if user upgrade their Flink version,
> these
> > old configuration will still be considered or just ignored?  If ignore,
> the
> > users may need to modify their configurations after they upgrade their
> > Flink.
> > >
> > > 2: I'm cursion in which case users will disable auto parallelism
> > derivation if they have enabled adaptive batch scheduler.  IIUC, auto
> > parallelism derivation is what adaptive batch scheduler aim to do. If use
> > want to diable auto parallelism derivation, can they just disable
> adaptive
> > batch scheduler.?
> > >
> > > Best regards,
> > > Yuxia
> > >
> > > - 原始邮件 -
> > > 发件人: "JunRui Lee" 
> > > 收件人: "dev" 
> > > 发送时间: 星期四, 2022年 12 月 29日 下午 7:45:36
> > > 主题: [DISCUSS] FLIP-283: Use adaptive batch scheduler as default
> > scheduler for batch jobs
> > >
> > > Hi, devs,
> > >
> > > I'd like to start a discussion about FLIP-283: Use adaptive batch
> > > scheduler as default scheduler for batch jobs[1].
> > >
> > > In FLIP-187, we introduced an adaptive batch scheduler. The adaptive
> > > batch scheduler has stronger batch scheduling capabilities, including
> > > automatically deciding parallelisms of job vertices for batch
> > > jobs (FLIP-187)[2], data balanced distribution (FLINK-29663)[3],
> > > and speculative execution (FLIP-168)[4]. To further use the adaptive
> > > batch scheduler to improve flink's batch capability, in this FLIP
> > > we aim to make the adaptive batch scheduler as the default batch
> > > scheduler.
> > >
> > > Currently, users have to set some configuration of the adaptive
> > > batch scheduler, which is not very convenient. To use the adaptive
> > > batch scheduler as the default batch scheduler, we need to improve
> > > the user's out-of-the-box experience. Therefore,  we also need to
> > > optimize the current adaptive batch scheduler configuration.
> > >
> > > Looking forward to your feedback.
> > >
> > > [1]:
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-283%3A+Use+adaptive+batch+scheduler+as+default+scheduler+for+batch+jobs
> > > [2]:
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-187%3A+Adaptive+Batch+Scheduler
> > > [3]:https://issues.apache.org/jira/browse/FLINK-29663
> > > [4]:
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-168%3A+Speculative+Execution+for+Batch+Job
> > >
> > > Best regards,
> > > JunRui Lee
> >
>


[DISCUSS] FLIP-283: Use adaptive batch scheduler as default scheduler for batch jobs

2022-12-29 Thread JunRui Lee
Hi, devs,

I'd like to start a discussion about FLIP-283: Use adaptive batch
scheduler as default scheduler for batch jobs[1].

In FLIP-187, we introduced an adaptive batch scheduler. The adaptive
batch scheduler has stronger batch scheduling capabilities, including
automatically deciding parallelisms of job vertices for batch
jobs (FLIP-187)[2], data balanced distribution (FLINK-29663)[3],
and speculative execution (FLIP-168)[4]. To further use the adaptive
batch scheduler to improve flink's batch capability, in this FLIP
we aim to make the adaptive batch scheduler as the default batch
scheduler.

Currently, users have to set some configuration of the adaptive
batch scheduler, which is not very convenient. To use the adaptive
batch scheduler as the default batch scheduler, we need to improve
the user's out-of-the-box experience. Therefore,  we also need to
optimize the current adaptive batch scheduler configuration.

Looking forward to your feedback.

[1]:
https://cwiki.apache.org/confluence/display/FLINK/FLIP-283%3A+Use+adaptive+batch+scheduler+as+default+scheduler+for+batch+jobs
[2]:
https://cwiki.apache.org/confluence/display/FLINK/FLIP-187%3A+Adaptive+Batch+Scheduler
[3]:https://issues.apache.org/jira/browse/FLINK-29663
[4]:
https://cwiki.apache.org/confluence/display/FLINK/FLIP-168%3A+Speculative+Execution+for+Batch+Job

Best regards,
JunRui Lee


Re: [DISCUSS] FLIP-271: Autoscaling

2022-11-07 Thread JunRui Lee
@Guyla,

Thanks for the explanation and the follow up actions. That sounds good to
me.

Thanks,
JunRui Lee

Yanfei Lei  于2022年11月7日周一 12:20写道:

> Hi Max,
>
> Thanks for the proposal. This proposal makes Flink better adapted to
> cloud-native applications!
>
> After reading the FLIP, I'm curious about some points:
>
> 1) It's said that "The first step is collecting metrics for all JobVertices
> by combining metrics from all the runtime subtasks and computing the
> *average*". When the load of the subtasks of an operator is not balanced,
> do we need to trigger autoScaling? Has the median or some percentiles been
> considered?
> 2) IIUC, "FLIP-159: Reactive Mode" is somewhat similar to this proposal,
> will we reuse some logic from Reactive Mode?
>
> Best,
> Yanfei
>
> Gyula Fóra  于2022年11月7日周一 02:33写道:
>
> > Hi Dong!
> >
> > Let me try to answer the questions :)
> >
> > 1 : busyTimeMsPerSecond is not specific for CPU, it measures the time
> spent
> > in the main record processing loop for an operator if I
> > understand correctly. This includes IO operations too.
> >
> > 2: We should add this to the FLIP I agree. It would be a Duration config
> > with the expected catch up time after rescaling (let's say 5 minutes). It
> > could be computed based on the current data rate and the calculated max
> > processing rate after the rescale.
> >
> > 3: In the current proposal we don't have per operator configs. Target
> > utilization would apply to all operators uniformly.
> >
> > 4: It should be configurable, yes.
> >
> > 5,6: The names haven't been finalized but I think these are minor
> details.
> > We could add concrete names to the FLIP :)
> >
> > Cheers,
> > Gyula
> >
> >
> > On Sun, Nov 6, 2022 at 5:19 PM Dong Lin  wrote:
> >
> > > Hi Max,
> > >
> > > Thank you for the proposal. The proposal tackles a very important issue
> > > for Flink users and the design looks promising overall!
> > >
> > > I have some questions to better understand the proposed public
> interfaces
> > > and the algorithm.
> > >
> > > 1) The proposal seems to assume that the operator's busyTimeMsPerSecond
> > > could reach 1 sec. I believe this is mostly true for cpu-bound
> operators.
> > > Could you confirm that this can also be true for io-bound operators
> such
> > as
> > > sinks? For example, suppose a Kafka Sink subtask has reached I/O
> > bottleneck
> > > when flushing data out to the Kafka clusters, will busyTimeMsPerSecond
> > > reach 1 sec?
> > >
> > > 2) It is said that "users can configure a maximum time to fully process
> > > the backlog". The configuration section does not seem to provide this
> > > config. Could you specify this? And any chance this proposal can
> provide
> > > the formula for calculating the new processing rate?
> > >
> > > 3) How are users expected to specify the per-operator configs (e.g.
> > target
> > > utilization)? For example, should users specify it programmatically in
> a
> > > DataStream/Table/SQL API?
> > >
> > > 4) How often will the Flink Kubernetes operator query metrics from
> > > JobManager? Is this configurable?
> > >
> > > 5) Could you specify the config name and default value for the proposed
> > > configs?
> > >
> > > 6) Could you add the name/mbean/type for the proposed metrics?
> > >
> > >
> > > Cheers,
> > > Dong
> > >
> > >
> > >
> >
>


Re: [DISCUSS] FLIP-271: Autoscaling

2022-11-05 Thread JunRui Lee
Hi Max,

Thanks for writing this FLIP and initiating the discussion.

I just have a small question after reading the FLIP:

In the document, I didn't find the definition of when to trigger
autoScaling after some jobVertex reach the threshold. If I missed is,
please let me know.
IIUC, the proper triggering rules are necessary to avoid unnecessary
autoscaling caused by temporary large changes in data,
and in this case, it will lead to at least two meaningless resubmissions of
jobs, which will negatively affect users.

Thanks,
JunRui Lee

Gyula Fóra  于2022年11月5日周六 20:38写道:

> Hey!
>
> Thanks for the input!
>
> The algorithm does not really differentiate between scaling up or down as
> it’s concerned about finding the right parallelism to match the target
> processing rate with just enough spare capacity.
>
> Let me try to address your specific points:
>
> 1. The backlog growth rate only matters for computing the target processing
> rate for the sources. If the parallelism is high enough and there is no
> back pressure it will be close to 0 so the target rate is the source read
> rate. This is as intended. If we see that the sources are not busy and they
> can read more than enough the algorithm would scale them down.
>
> 2. You are right , it’s dangerous to scale in too much, so we already
> thought about limiting the scale down amount per scaling step/time window
> to give more safety. But we can definitely think about different strategies
> in the future!
>
> The observation regarding max parallelism is very important and we should
> always take that into consideration.
>
> Cheers
> Gyula
>
> On Sat, 5 Nov 2022 at 11:46, Biao Geng  wrote:
>
> > Hi Max,
> >
> > Thanks a lot for the FLIP. It is an extremely attractive feature!
> >
> > Just some follow up questions/thoughts after reading the FLIP:
> > In the doc, the discussion of  the strategy of “scaling out” is thorough
> > and convincing to me but it seems that “scaling down” is less discussed.
> I
> > have 2 cents for this aspect:
> >
> >   1.  For source parallelisms, if the user configure a much larger value
> > than normal, there should be very little pending records though it is
> > possible to get optimized. But IIUC, in current algorithm, we will not
> take
> > actions for this case as the backlog growth rate is almost zero. Is the
> > understanding right?
> >   2.  Compared with “scaling out”, “scaling in” is usually more dangerous
> > as it is more likely to lead to negative influence to the downstream
> jobs.
> > The min/max load bounds should be useful. I am wondering if it is
> possible
> > to have different strategy for “scaling in” to make it more conservative.
> > Or more eagerly, allow custom autoscaling strategy(e.g. time-based
> > strategy).
> >
> > Another side thought is that to recover a job from checkpoint/savepoint,
> > the new parallelism cannot be larger than max parallelism defined in the
> > checkpoint(see this<
> >
> https://github.com/apache/flink/blob/17a782c202c93343b8884cb52f4562f9c4ba593f/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java#L128
> >).
> > Not sure if this limit should be mentioned in the FLIP.
> >
> > Again, thanks for the great work and looking forward to using flink k8s
> > operator with it!
> >
> > Best,
> > Biao Geng
> >
> > From: Maximilian Michels 
> > Date: Saturday, November 5, 2022 at 2:37 AM
> > To: dev 
> > Cc: Gyula Fóra , Thomas Weise ,
> > Marton Balassi , Őrhidi Mátyás <
> > matyas.orh...@gmail.com>
> > Subject: [DISCUSS] FLIP-271: Autoscaling
> > Hi,
> >
> > I would like to kick off the discussion on implementing autoscaling for
> > Flink as part of the Flink Kubernetes operator. I've outlined an approach
> > here which I find promising:
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-271%3A+Autoscaling
> >
> > I've been discussing this approach with some of the operator
> contributors:
> > Gyula, Marton, Matyas, and Thomas (all in CC). We started prototyping an
> > implementation based on the current FLIP design. If that goes well, we
> > would like to contribute this to Flink based on the results of the
> > discussion here.
> >
> > I'm curious to hear your thoughts.
> >
> > -Max
> >
>


Re: [VOTE] Release 1.16.0, release candidate #2

2022-10-24 Thread JunRui Lee
+1 (non-binding) for this candidate

  *   Built from the source code.
  *   Ran batch wordcount jobs with slow nodes of different source types on
the yarn cluster.
  *   The new source speculative execution works as expected, the result is
expected, no suspicious log output.
  *   Slow nodes are successfully added to the blocklist and subsequently
removed as expected.
  *   Ran large parallelism batch jobs and performance does not degrade.

Best,
JunRui

yuxia  于2022年10月25日周二 09:23写道:

> +1 (non-binding)
> * Build from source
> * Use Flink Sql client create catalog/tables
> * Use Hive dialect to run some queries and insert statements
>
> Best regards,
> Yuxia
>
> - 原始邮件 -
> 发件人: "Teoh, Hong" 
> 收件人: "dev" 
> 发送时间: 星期二, 2022年 10 月 25日 上午 4:35:39
> 主题: Re: [VOTE] Release 1.16.0, release candidate #2
>
> +1 (non-binding)
>
> * Hashes and Signatures look good
> * All required files on dist.apache.org
> * Tag is present in Github
> * Verified source archive does not contain any binary files
> * Source archive builds using maven
> * Deployed standalone session cluster and ran TopSpeedWindowing example in
> streaming with checkpointing enabled. Looks ok
>
> Cheers,
> Hong
>
> On 24/10/2022, 16:06, "Gyula Fóra"  wrote:
>
> CAUTION: This email originated from outside of the organization. Do
> not click links or open attachments unless you can confirm the sender and
> know the content is safe.
>
>
>
> +1 (binding)
>
> * Verified checksums/GPG signatures
> * Built from source
> * Tested with Kubernetes operator, including simple jobs,
> checkpointing etc.
> * Metrics, logs look good.
>
> Gyula
>
> On Mon, Oct 24, 2022 at 4:54 PM Matthias Pohl
>  wrote:
>
> > +1 (non-binding)
> >
> > * Downloaded artifacts
> > * Verified checksums/GPG signatures
> > * Compared checkout with provided sources
> > * Verified pom file versions
> > * Went over NOTICE file/pom files changes without finding anything
> > suspicious
> > * Build Flink from sources
> > * Deployed standalone session cluster and ran WordCount example in
> batch
> > and streaming: Nothing suspicious in log files found
> >
> > On Mon, Oct 24, 2022 at 3:51 PM Sergey Nuyanzin  >
> > wrote:
> >
> > > +1 (non-binding)
> > >
> > > - checked hashes and signatures
> > > - built from sources
> > > - started cluster, ran different simple jobs
> > > - checked sql client
> > >
> > >
> > > On Mon, Oct 24, 2022 at 3:14 PM Leonard Xu 
> wrote:
> > >
> > > > +1 (non-binding)
> > > >
> > > > - verified signatures and hashsums
> > > > - built from source code succeeded
> > > > - checked all dependency artifacts are 1.16
> > > > - started a cluster, ran a wordcount job, the result is
> expected, no
> > > > suspicious log output
> > > > - started SQL Gateway, tested several rest APIs, the SQL query
> results
> > > are
> > > > expected
> > > >
> > > > Best,
> > > > Leonard Xu
> > > >
> > > >
> > > > > 2022年10月24日 下午8:49,Xingbo Huang  写道:
> > > > >
> > > > > +1 (non-binding)
> > > > >
> > > > > - verify signatures and checksums
> > > > > - no binaries found in source archive
> > > > > - build from source code
> > > > > - verify python wheel package contents
> > > > > - pip install apache-flink-libraries and apache-flink wheel
> packages
> > > > > - thread mode works as expected in Python DataStream API
> > > > > - the Python DataStream Window works as expected
> > > > > - the Python Sideoutput works as expected
> > > > >
> > > > > Best,
> > > > > Xingbo
> > > > >
> > > > > Lijie Wang  于2022年10月24日周一 16:09写道:
> > > > >
> > > > >> +1 (non-binding) for this candidate
> > > > >>
> > > > >>  *   Built from the source code.
> > > > >>  *   Verified the signature and checksum
> > > > >>  *   Ran both streaming/batch jobs on yarn cluster
> > > > >>  *   The new speculative execution works as expected
> > > > >>
> > > > >> Best,
> > > > >> Lijie
> > > > >>
> > > > >> Yun Tang  于2022年10月22日周六 15:20写道:
> > > > >>
> > > > >>> +1 (non-binding)
> > > > >>>
> > > > >>>
> > > > >>>  *   build from the source code.
> > > > >>>  *   verified the signature of the release binaries.
> > > > >>>  *   launched a standalone cluster with examples for both
> > > > >> streaming/batch
> > > > >>> execution modes.
> > > > >>>  *   checked rocksdb's log would exist under Flink's default
> log
> > > > folder.
> > > > >>>  *   reviewed the flink-web PR
> > > > >>>
> > > > >>> Best
> > > > >>> Yun Tang
> > > > >>> 
> > > > >>> From: Xintong Song 
> > > > >>> Sent: Friday, October 21, 2022 18:29
> > > > >>> To: dev@flink.apache.org 
> > > > >>> Subject: Re: [VOTE] Release 1.16.0, release 

Re: [ANNOUNCE] New Apache Flink Committer - Lijie Wang

2022-08-17 Thread JunRui Lee
Congratulations, Lijie!

Best,
JunRui

Timo Walther  于2022年8月17日周三 19:30写道:

> Congratulations and welcome to the committer team :-)
>
> Regards,
> Timo
>
>
> On 17.08.22 12:50, Yuxin Tan wrote:
> > Congratulations, Lijie!
> >
> > Best,
> > Yuxin
> >
> >
> > Guowei Ma  于2022年8月17日周三 18:42写道:
> >
> >> Congratulations, Lijie. Welcome on board~!
> >> Best,
> >> Guowei
> >>
> >>
> >> On Wed, Aug 17, 2022 at 6:25 PM Zhu Zhu  wrote:
> >>
> >>> Hi everyone,
> >>>
> >>> On behalf of the PMC, I'm very happy to announce Lijie Wang as
> >>> a new Flink committer.
> >>>
> >>> Lijie has been contributing to Flink project for more than 2 years.
> >>> He mainly works on the runtime/coordination part, doing feature
> >>> development, problem debugging and code reviews. He has also
> >>> driven the work of FLIP-187(Adaptive Batch Scheduler) and
> >>> FLIP-224(Blocklist for Speculative Execution), which are important
> >>> to run batch jobs.
> >>>
> >>> Please join me in congratulating Lijie for becoming a Flink committer!
> >>>
> >>> Cheers,
> >>> Zhu
> >>>
> >>
> >
>
>