Re: [DISCUSS] Change some default config values of blocking shuffle

2022-01-06 Thread Yingjie Cao
ault config values of blocking shuffle > > Thanks for the experiment. +1 for the changes. > > Yingjie Cao 于2022年1月4日周二 17:35写道: > > > Hi all, > > > > After running some tests with the proposed default value ( > > taskmanager.network.sort-shuffle.min-parallelism

Re: [DISCUSS] Change some default config values of blocking shuffle

2022-01-04 Thread Yingjie Cao
my limited knowledge, I guess that increasing the total TaskManager size and network memory size is important for performance, because more memory (managed and network) can make operators and shuffle faster. Best, Yingjie Yingjie Cao 于2021年12月15日周三 12:19写道: > Hi Till, > > Thanks fo

Re: [DISCUSS] Change some default config values of blocking shuffle

2021-12-14 Thread Yingjie Cao
shuffle/#sort-shuffle > > Cheers, > Till > > On Tue, Dec 14, 2021 at 8:43 AM Jingsong Li > wrote: > >> Hi Yingjie, >> >> Thanks for your explanation. I have no more questions. +1 >> >> On Tue, Dec 14, 2021 at 3:31 PM Yingjie Cao >> wrote: >>

Re: [DISCUSS] Change some default config values of blocking shuffle

2021-12-13 Thread Yingjie Cao
uld be best if a rough table could be provided. > > Best, > Jingsong > > On Tue, Dec 14, 2021 at 2:16 PM Yingjie Cao > wrote: > > > > Hi Jiangang, > > > > Thanks for your suggestion. > > > > >>> The config can affect the memory usage. Wi

Re: [DISCUSS] Change some default config values of blocking shuffle

2021-12-13 Thread Yingjie Cao
rg> >> Subject:Re: [DISCUSS] Change some default config values of blocking >> shuffle >> >> Hi dev & users: >> >> I have created a FLIP [1] for it, feedbacks are highly appreciated. >> >> Best, >> Yingjie >> >> [1] >> http

Re: [DISCUSS] Change some default config values of blocking shuffle

2021-12-13 Thread Yingjie Cao
alues of blocking shuffle > > Hi dev & users: > > I have created a FLIP [1] for it, feedbacks are highly appreciated. > > Best, > Yingjie > > [1] > https://cwiki.apache.org/confluence/display/FLINK/FLIP-199%3A+Change+some+default+config+values+of+blocking+shuffle+for+better

Re: [DISCUSS] Change some default config values of blocking shuffle

2021-12-10 Thread Yingjie Cao
Hi dev & users: I have created a FLIP [1] for it, feedbacks are highly appreciated. Best, Yingjie [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-199%3A+Change+some+default+config+values+of+blocking+shuffle+for+better+usability Yingjie Cao 于2021年12月3日周五 17:02写道: > Hi dev

Re: [DISCUSS] Change some default config values of blocking shuffle

2021-12-06 Thread Yingjie Cao
ources and, thus, will it risk destabilizing our testing infrastructure? > > I would propose to create a FLIP for these changes since you propose to > change the default behaviour. It can be a very short one, though. > > Cheers, > Till > > On Fri, Dec 3, 2021 at 10:02 AM Yingjie Cao &g

[DISCUSS] Change some default config values of blocking shuffle

2021-12-03 Thread Yingjie Cao
Hi dev & users, We propose to change some default values of blocking shuffle to improve the user out-of-box experience (not influence streaming). The default values we want to change are as follows: 1. Data compression (taskmanager.network.blocking-shuffle.compression.enabled): Currently, the def

Re: [ANNOUNCE] Open source of remote shuffle project for Flink batch data processing

2021-12-01 Thread Yingjie Cao
once failover. We have investigated the > component a lot and welcome Flink's native solution. We will try it and > help improve it. > > Thanks, > Liu Jiangang > > Yingjie Cao 于2021年11月30日周二 下午9:33写道: > > > Hi dev & users, > > > > We are happ

[ANNOUNCE] Open source of remote shuffle project for Flink batch data processing

2021-11-30 Thread Yingjie Cao
Hi dev & users, We are happy to announce the open source of remote shuffle project [1] for Flink. The project is originated in Alibaba and the main motivation is to improve batch data processing for both performance & stability and further embrace cloud native. For more features about the project,

[DISCUSS] FLIP-184: Refine ShuffleMaster lifecycle management for pluggable shuffle service framework

2021-07-13 Thread Yingjie Cao
Hi devs and users, This topic was originally discussed and reached a consensus in [1]. Because the change touches the pluggable shuffle interface, though not annotated as public currently, some users may be using it already. To avoid bring compatibility issues to customized shuffle plugins already

Re: flink job exception analysis (netty related, readAddress failed. connection timed out)

2021-06-16 Thread Yingjie Cao
etwork.retries which is default > > 0, and taskmanager.network.netty.client.connectTimeoutSec 's default > > is 120s。 > > 5: I checked the net fd number of the taskmanager, it is about 1000+, > > so I think it is a reasonable value. > > > > 1: can not be sure. > > > > Yingjie Cao

Re: flink job exception analysis (netty related, readAddress failed. connection timed out)

2021-06-16 Thread Yingjie Cao
Hi yidan, 1. Is the network stable? 2. Is there any GC problem? 3. Is it a batch job? If so, please use sort-shuffle, see [1] for more information. 4. You may try to config these two options: taskmanager.network.retries, taskmanager.network.netty.client.connectTimeoutSec. More relevant options can

Re: [DISCUSS] FLIP-148: Introduce Sort-Merge Based Blocking Shuffle to Flink

2021-06-08 Thread Yingjie Cao
his could be interesting to our users. > > Cheers, > Till > > On Mon, Jun 7, 2021 at 4:49 AM Jingsong Li wrote: > >> Thanks Yingjie for the great effort! >> >> This is really helpful to Flink Batch users! >> >> Best, >> Jingsong >> >

Re: [DISCUSS] FLIP-148: Introduce Sort-Merge Based Blocking Shuffle to Flink

2021-06-06 Thread Yingjie Cao
nt[2] which contains more implementation details. FYI. [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-148%3A+Introduce+Sort-Based+Blocking+Shuffle+to+Flink [2] https://docs.google.com/document/d/1j12TkSqgf6dg3J48udA2MFrDOQccW24tzjn5pJlTQaQ/edit?usp=sharing Best, Yingjie Yingjie Ca

Re: With the checkpoint interval of the same size, the Flink 1.12 version of the job checkpoint time-consuming increase and production failure, the Flink1.9 job is running normally

2021-03-30 Thread Yingjie Cao
Hi Haihang, After scanning the user mailing list, I found some users have reported checkpoint timeout when using unaligned checkpoint, can you share which checkpoint mode do you use? (The information can be found in log or the checkpoint -> configuration tab in webui) Best, Yingjie Yingjie

Re: With the checkpoint interval of the same size, the Flink 1.12 version of the job checkpoint time-consuming increase and production failure, the Flink1.9 job is running normally

2021-03-30 Thread Yingjie Cao
Hi Haihang, I think your issue is not related to FLINK-16404 , because that change should have small impact on checkpoint time, we already have a micro benchmark for that change (1s checkpoint interval) and no regression is seen. Could you share

Re: What happens to the channels when there is backpressure?

2019-11-27 Thread yingjie cao
Hi Felipe, That depends on what do you mean by 'bandwidth'. If you mean the capability of the network stack, the answer would be no. Here is a post about Flink network stack which may help: https://flink.apache.org/2019/06/05/flink-network-stack.html. Thanks, Yingjie Felipe Gutierrez 于2019年11月