Re: [VOTE] FLIP-380: Support Full Partition Processing On Non-keyed DataStream

2023-12-19 Thread Yingjie Cao
+1 (binding) Best, Yingjie Xintong Song 于2023年12月18日周一 15:27写道: > +1 (binding) > > > Best, > > Xintong > > > > On Fri, Dec 15, 2023 at 5:15 PM weijie guo > wrote: > > > +1 (binding) > > > > Best regards, > > > > Weijie > > > > > > Wencong Liu 于2023年12月15日周五 09:15写道: > > > > > Hi dev, > > > >

Re: [ANNOUNCE] Release 1.17.0, release candidate #1

2023-03-10 Thread Yingjie Cao
> more blockers to be found, we can fix them with the next RC and have > (hopefully) a successful vote on it. > > Best regards, > > Martijn > > On Thu, Mar 9, 2023 at 4:54 PM Yingjie Cao > wrote: > > > Hi community and release managers: > > > > Wh

Re: [ANNOUNCE] Release 1.17.0, release candidate #1

2023-03-09 Thread Yingjie Cao
Hi community and release managers: When testing the release candidate #1 for batch scenario, I found a potential deadlock issue of blocking shuffle. I have created a ticket [1] for it and marked it as blocker. I will fix it no later than tomorrow. [1]

[jira] [Created] (FLINK-31386) Fix the potential deadlock issue of blocking shuffle

2023-03-09 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-31386: --- Summary: Fix the potential deadlock issue of blocking shuffle Key: FLINK-31386 URL: https://issues.apache.org/jira/browse/FLINK-31386 Project: Flink Issue

[jira] [Created] (FLINK-29351) Enable input buffer floating for blocking shuffle

2022-09-20 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-29351: --- Summary: Enable input buffer floating for blocking shuffle Key: FLINK-29351 URL: https://issues.apache.org/jira/browse/FLINK-29351 Project: Flink Issue Type

[jira] [Created] (FLINK-29299) Fix the network memory size calculation issue in fine-grained resource mode

2022-09-13 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-29299: --- Summary: Fix the network memory size calculation issue in fine-grained resource mode Key: FLINK-29299 URL: https://issues.apache.org/jira/browse/FLINK-29299 Project

[jira] [Created] (FLINK-28663) Allow multiple downstream consumer job vertices sharing the same intermediate dataset at scheduler side

2022-07-25 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28663: --- Summary: Allow multiple downstream consumer job vertices sharing the same intermediate dataset at scheduler side Key: FLINK-28663 URL: https://issues.apache.org/jira/browse/FLINK

[jira] [Created] (FLINK-28561) Merge subpartition shuffle data read request for better sequential IO

2022-07-15 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28561: --- Summary: Merge subpartition shuffle data read request for better sequential IO Key: FLINK-28561 URL: https://issues.apache.org/jira/browse/FLINK-28561 Project: Flink

[jira] [Created] (FLINK-28556) Extract header fields of Buffer into a BufferHeader class for blocking shuffle file IO

2022-07-14 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28556: --- Summary: Extract header fields of Buffer into a BufferHeader class for blocking shuffle file IO Key: FLINK-28556 URL: https://issues.apache.org/jira/browse/FLINK-28556

[jira] [Created] (FLINK-28551) Store the number of bytes instead of the number of buffers in index entry for sort-shuffle

2022-07-14 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28551: --- Summary: Store the number of bytes instead of the number of buffers in index entry for sort-shuffle Key: FLINK-28551 URL: https://issues.apache.org/jira/browse/FLINK-28551

[jira] [Created] (FLINK-28550) Remove the unused field in SortMergeSubpartitionReader

2022-07-14 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28550: --- Summary: Remove the unused field in SortMergeSubpartitionReader Key: FLINK-28550 URL: https://issues.apache.org/jira/browse/FLINK-28550 Project: Flink Issue

[jira] [Created] (FLINK-28519) Fix the bug that SortMergeResultPartitionReadScheduler may not read data sequentially

2022-07-12 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28519: --- Summary: Fix the bug that SortMergeResultPartitionReadScheduler may not read data sequentially Key: FLINK-28519 URL: https://issues.apache.org/jira/browse/FLINK-28519

[jira] [Created] (FLINK-28514) Remove data flush in SortMergeResultPartition

2022-07-12 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28514: --- Summary: Remove data flush in SortMergeResultPartition Key: FLINK-28514 URL: https://issues.apache.org/jira/browse/FLINK-28514 Project: Flink Issue Type: Sub

[jira] [Created] (FLINK-28512) Select HashBasedDataBuffer and SortBasedDataBuffer dynamically based on the number of network buffers can be allocated for

2022-07-12 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28512: --- Summary: Select HashBasedDataBuffer and SortBasedDataBuffer dynamically based on the number of network buffers can be allocated for Key: FLINK-28512 URL: https://issues.apache.org

[jira] [Created] (FLINK-28382) Introduce new compression algorithms of higher compression ratio

2022-07-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28382: --- Summary: Introduce new compression algorithms of higher compression ratio Key: FLINK-28382 URL: https://issues.apache.org/jira/browse/FLINK-28382 Project: Flink

[jira] [Created] (FLINK-28380) Produce one intermediate dataset for multiple consumers consuming the same data

2022-07-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28380: --- Summary: Produce one intermediate dataset for multiple consumers consuming the same data Key: FLINK-28380 URL: https://issues.apache.org/jira/browse/FLINK-28380

[jira] [Created] (FLINK-28378) Use larger data reading buffer size for sort-shuffle

2022-07-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28378: --- Summary: Use larger data reading buffer size for sort-shuffle Key: FLINK-28378 URL: https://issues.apache.org/jira/browse/FLINK-28378 Project: Flink Issue

[jira] [Created] (FLINK-28377) Enable to request less number of data reading buffers for sort-shuffle if there is no enough data

2022-07-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28377: --- Summary: Enable to request less number of data reading buffers for sort-shuffle if there is no enough data Key: FLINK-28377 URL: https://issues.apache.org/jira/browse/FLINK-28377

[jira] [Created] (FLINK-28376) Reduce the number of IO threads for sort-shuffle

2022-07-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28376: --- Summary: Reduce the number of IO threads for sort-shuffle Key: FLINK-28376 URL: https://issues.apache.org/jira/browse/FLINK-28376 Project: Flink Issue Type

[jira] [Created] (FLINK-28374) Some further improvements of sort-shuffle

2022-07-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28374: --- Summary: Some further improvements of sort-shuffle Key: FLINK-28374 URL: https://issues.apache.org/jira/browse/FLINK-28374 Project: Flink Issue Type

[jira] [Created] (FLINK-28373) Read larger size of data sequentially for sort-shuffle

2022-07-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-28373: --- Summary: Read larger size of data sequentially for sort-shuffle Key: FLINK-28373 URL: https://issues.apache.org/jira/browse/FLINK-28373 Project: Flink Issue

Re: [VOTE] FLIP-235: Hybrid Shuffle Mode

2022-05-31 Thread Yingjie Cao
+1 Best, Yingjie Jing Ge 于2022年5月30日周一 05:02写道: > +1 (non-binding) > > Best Regards, > Jing > > On Sun, May 29, 2022 at 5:16 AM Aitozi wrote: > > > +1 (non-binding) > > > > Best, > > Aitozi > > > > Yangze Guo 于2022年5月27日周五 11:17写道: > > > > > +1 > > > > > > Best, > > > Yangze Guo > > > > > >

[jira] [Created] (FLINK-26347) Should use Flink system Classloader (AppClassloader) when deserializing RPC message

2022-02-24 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-26347: --- Summary: Should use Flink system Classloader (AppClassloader) when deserializing RPC message Key: FLINK-26347 URL: https://issues.apache.org/jira/browse/FLINK-26347

[jira] [Created] (FLINK-25960) Distribute the data read buffers more fairly among result partitions for sort-shuffle

2022-02-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25960: --- Summary: Distribute the data read buffers more fairly among result partitions for sort-shuffle Key: FLINK-25960 URL: https://issues.apache.org/jira/browse/FLINK-25960

[jira] [Created] (FLINK-25959) Add a micro-benchmark for the sort-based blocking shuffle

2022-02-04 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25959: --- Summary: Add a micro-benchmark for the sort-based blocking shuffle Key: FLINK-25959 URL: https://issues.apache.org/jira/browse/FLINK-25959 Project: Flink

[jira] [Created] (FLINK-25860) Move read buffer allocation and output file creation to setup method for sort-shuffle result partition to avoid blocking main thread

2022-01-27 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25860: --- Summary: Move read buffer allocation and output file creation to setup method for sort-shuffle result partition to avoid blocking main thread Key: FLINK-25860 URL: https

[jira] [Created] (FLINK-25796) Avoid record copy for result partition of sort-shuffle if there are enough buffers for better performance

2022-01-24 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25796: --- Summary: Avoid record copy for result partition of sort-shuffle if there are enough buffers for better performance Key: FLINK-25796 URL: https://issues.apache.org/jira/browse/FLINK

[jira] [Created] (FLINK-25786) Adjust the generation of subpartition data storage order for sort-shuffle from random shuffle to random shift

2022-01-24 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25786: --- Summary: Adjust the generation of subpartition data storage order for sort-shuffle from random shuffle to random shift Key: FLINK-25786 URL: https://issues.apache.org/jira/browse

[jira] [Created] (FLINK-25781) Adjust the maximum number of buffers can be used per result partition for shuffle data read

2022-01-24 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25781: --- Summary: Adjust the maximum number of buffers can be used per result partition for shuffle data read Key: FLINK-25781 URL: https://issues.apache.org/jira/browse/FLINK-25781

[jira] [Created] (FLINK-25780) Reduce the maximum number of data output buffers for sort-shuffle

2022-01-24 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25780: --- Summary: Reduce the maximum number of data output buffers for sort-shuffle Key: FLINK-25780 URL: https://issues.apache.org/jira/browse/FLINK-25780 Project: Flink

[jira] [Created] (FLINK-25774) Restrict the maximum number of buffers can be used per result partition for blocking shuffle

2022-01-24 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25774: --- Summary: Restrict the maximum number of buffers can be used per result partition for blocking shuffle Key: FLINK-25774 URL: https://issues.apache.org/jira/browse/FLINK-25774

[jira] [Created] (FLINK-25654) Remove the redundant lock in SortMergeResultPartition

2022-01-13 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25654: --- Summary: Remove the redundant lock in SortMergeResultPartition Key: FLINK-25654 URL: https://issues.apache.org/jira/browse/FLINK-25654 Project: Flink Issue

[jira] [Created] (FLINK-25653) Move buffer recycle in SortMergeSubpartitionReader out of lock to avoid deadlock

2022-01-13 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25653: --- Summary: Move buffer recycle in SortMergeSubpartitionReader out of lock to avoid deadlock Key: FLINK-25653 URL: https://issues.apache.org/jira/browse/FLINK-25653

[jira] [Created] (FLINK-25640) Enhance the document for blocking shuffle

2022-01-12 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25640: --- Summary: Enhance the document for blocking shuffle Key: FLINK-25640 URL: https://issues.apache.org/jira/browse/FLINK-25640 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-25639) Increase the default read buffer size of sort-shuffle to 64M

2022-01-12 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25639: --- Summary: Increase the default read buffer size of sort-shuffle to 64M Key: FLINK-25639 URL: https://issues.apache.org/jira/browse/FLINK-25639 Project: Flink

[jira] [Created] (FLINK-25638) Increase the default write buffer size of sort-shuffle to 16M

2022-01-12 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25638: --- Summary: Increase the default write buffer size of sort-shuffle to 16M Key: FLINK-25638 URL: https://issues.apache.org/jira/browse/FLINK-25638 Project: Flink

[jira] [Created] (FLINK-25637) Make sort-shuffle the default shuffle implementation for batch jobs

2022-01-12 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25637: --- Summary: Make sort-shuffle the default shuffle implementation for batch jobs Key: FLINK-25637 URL: https://issues.apache.org/jira/browse/FLINK-25637 Project: Flink

[jira] [Created] (FLINK-25636) FLIP-199: Change some default config values of blocking shuffle for better usability

2022-01-12 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25636: --- Summary: FLIP-199: Change some default config values of blocking shuffle for better usability Key: FLINK-25636 URL: https://issues.apache.org/jira/browse/FLINK-25636

[jira] [Created] (FLINK-25607) Sorting by duration on Flink Web UI does not work correctly

2022-01-11 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25607: --- Summary: Sorting by duration on Flink Web UI does not work correctly Key: FLINK-25607 URL: https://issues.apache.org/jira/browse/FLINK-25607 Project: Flink

[jira] [Created] (FLINK-25606) Requesting exclusive buffers timeout when recovering from unaligned checkpoint under fine-grained resource mode

2022-01-11 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25606: --- Summary: Requesting exclusive buffers timeout when recovering from unaligned checkpoint under fine-grained resource mode Key: FLINK-25606 URL: https://issues.apache.org/jira/browse

[VOTE] FLIP-199: Change some default config values of blocking shuffle for better usability

2022-01-10 Thread Yingjie Cao
Hi all, I'd like to start a vote on FLIP-199: Change some default config values of blocking shuffle for better usability [1] which has been discussed in this thread [2]. The vote will be open for at least 72 hours unless there is an objection or not enough votes. [1]

[DISCUSS] FLIP-209: Support to run multiple shuffle plugins in one session cluster

2022-01-07 Thread Yingjie Cao
Hi dev, I'd like to start a discussion for FLIP-209 [1] which aims to support to run multiple shuffle plugins in one session cluster. Currently, one Flink cluster can only use one shuffle service plugin configured by 'shuffle-service-factory.class'. It is not flexible enough and cannot support

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

2022-01-06 Thread Yingjie Cao
some default 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
mited 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 for the

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: >&g

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

2021-12-13 Thread Yingjie Cao
e 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. Will th

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
config values 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+bet

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

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

2021-12-01 Thread Yingjie Cao
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 happy t

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

2021-11-30 Thread Yingjie Cao
; > > > Best regards, > > > > Martijn > > > > On Tue, 30 Nov 2021 at 17:31, Till Rohrmann > wrote: > > > > > Great news, Yingjie. Thanks a lot for sharing this information with the > > > community and kudos to all the contributors of the e

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

2021-11-30 Thread Yingjie Cao
ue, 30 Nov 2021 at 17:31, Till Rohrmann wrote: > > > Great news, Yingjie. Thanks a lot for sharing this information with the > > community and kudos to all the contributors of the external shuffle > service > > :-) > > > > Cheers, > > Till > > > &

[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

Re: [DISCUSS] Releasing Flink 1.14.1

2021-11-24 Thread Yingjie Cao
Hi Martijn, I moved the fix version of "FLINK-21788 - Throw PartitionNotFoundException if the partition file has been lost for blocking shuffle" to 1.15.0 Best, Yingjie Martijn Visser 于2021年11月25日周四 上午2:40写道: > Hi all, > > I would like to

[jira] [Created] (FLINK-24954) Reset read buffer request timeout on buffer recycling for sort-shuffle

2021-11-18 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-24954: --- Summary: Reset read buffer request timeout on buffer recycling for sort-shuffle Key: FLINK-24954 URL: https://issues.apache.org/jira/browse/FLINK-24954 Project: Flink

[jira] [Created] (FLINK-24901) Some further improvements of the pluggable shuffle framework

2021-11-15 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-24901: --- Summary: Some further improvements of the pluggable shuffle framework Key: FLINK-24901 URL: https://issues.apache.org/jira/browse/FLINK-24901 Project: Flink

[jira] [Created] (FLINK-24900) Support to run multiple shuffle plugins in one session cluster

2021-11-15 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-24900: --- Summary: Support to run multiple shuffle plugins in one session cluster Key: FLINK-24900 URL: https://issues.apache.org/jira/browse/FLINK-24900 Project: Flink

[jira] [Created] (FLINK-24899) Enable data compression for blocking shuffle by default

2021-11-15 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-24899: --- Summary: Enable data compression for blocking shuffle by default Key: FLINK-24899 URL: https://issues.apache.org/jira/browse/FLINK-24899 Project: Flink Issue

[jira] [Created] (FLINK-24898) Some further improvements of sort-shuffle

2021-11-15 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-24898: --- Summary: Some further improvements of sort-shuffle Key: FLINK-24898 URL: https://issues.apache.org/jira/browse/FLINK-24898 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-24035) Notify the buffer listeners when the local buffer pool receives available notification from the global pool

2021-08-27 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-24035: --- Summary: Notify the buffer listeners when the local buffer pool receives available notification from the global pool Key: FLINK-24035 URL: https://issues.apache.org/jira/browse

[jira] [Created] (FLINK-23724) Network buffer leak when ResultPartition is released (failover)

2021-08-11 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-23724: --- Summary: Network buffer leak when ResultPartition is released (failover) Key: FLINK-23724 URL: https://issues.apache.org/jira/browse/FLINK-23724 Project: Flink

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

2021-07-16 Thread Yingjie Cao
Hi all, I'd like to start a vote on FLIP-184 [1] which was discussed in [2] [3]. The vote will be open for at least 72 hours until 7.21 unless there is an objection. [1]

Re: [DISCUSS] FLIP-183: Dynamic buffer size adjustment

2021-07-13 Thread Yingjie Cao
Hi, Thanks for driving this, I think it is really helpful for jobs suffering from backpressure. Best, Yingjie Anton,Kalashnikov 于2021年7月9日周五 下午10:59写道: > Hey! > > There is a wish to decrease amount of in-flight data which can improve > aligned checkpoint time(fewer in-flight data to process

[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

Re: [DISCUSS] Lifecycle of ShuffleMaster and its Relationship with JobMaster and PartitionTracker

2021-07-09 Thread Yingjie Cao
be from different jobs. But only > one single ShuffleEnvironment will be created on each task manager. > > Thanks, > Zhu > > Yingjie Cao 于2021年7月8日周四 上午11:43写道: > > > Hi, > > > > Thanks for the reply. > > > > @Guowei > > I agree that we ca

Re: [DISCUSS] Lifecycle of ShuffleMaster and its Relationship with JobMaster and PartitionTracker

2021-07-07 Thread Yingjie Cao
e ShuffleMasterContext at the beginning; > at > > the same time, JobShuffleContext::getConfiguration/listPartitions should > > not be needed. > > > > [1] > > > > > https://docs.google.com/document/d/1_cHoapNbx_fJ7ZNraSqw4ZK1hMRiWWJDITuSZrdMDDs/edit > >

[jira] [Created] (FLINK-23275) Support to release cluster partitions stored externally

2021-07-06 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-23275: --- Summary: Support to release cluster partitions stored externally Key: FLINK-23275 URL: https://issues.apache.org/jira/browse/FLINK-23275 Project: Flink Issue

[jira] [Created] (FLINK-23249) Introduce ShuffleMasterContext to ShuffleMaster

2021-07-05 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-23249: --- Summary: Introduce ShuffleMasterContext to ShuffleMaster Key: FLINK-23249 URL: https://issues.apache.org/jira/browse/FLINK-23249 Project: Flink Issue Type

[jira] [Created] (FLINK-23214) Make ShuffleMaster a cluster level shared service

2021-07-02 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-23214: --- Summary: Make ShuffleMaster a cluster level shared service Key: FLINK-23214 URL: https://issues.apache.org/jira/browse/FLINK-23214 Project: Flink Issue Type

[DISCUSS] Lifecycle of ShuffleMaster and its Relationship with JobMaster and PartitionTracker

2021-06-11 Thread Yingjie Cao
Hi devs, I'd like to start a discussion about "Lifecycle of ShuffleMaster and its Relationship with JobMaster and PartitionTracker". (These are things we found when moving our external shuffle to the pluggable shuffle service framework.) The mail client may fail to display the right format. If

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 >> >

[jira] [Created] (FLINK-22910) ShuffleMaster enhancement for pluggable shuffle service framework

2021-06-07 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-22910: --- Summary: ShuffleMaster enhancement for pluggable shuffle service framework Key: FLINK-22910 URL: https://issues.apache.org/jira/browse/FLINK-22910 Project: Flink

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

[jira] [Created] (FLINK-22307) Increase the data writing cache size of sort-merge blocking shuffle

2021-04-16 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-22307: --- Summary: Increase the data writing cache size of sort-merge blocking shuffle Key: FLINK-22307 URL: https://issues.apache.org/jira/browse/FLINK-22307 Project: Flink

[jira] [Created] (FLINK-22305) Increase the default value of taskmanager.network.sort-shuffle.min-buffers

2021-04-16 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-22305: --- Summary: Increase the default value of taskmanager.network.sort-shuffle.min-buffers Key: FLINK-22305 URL: https://issues.apache.org/jira/browse/FLINK-22305 Project

[jira] [Created] (FLINK-22156) HiveDialectQueryITCase fails on Azure because of no output for 900 seconds

2021-04-08 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-22156: --- Summary: HiveDialectQueryITCase fails on Azure because of no output for 900 seconds Key: FLINK-22156 URL: https://issues.apache.org/jira/browse/FLINK-22156 Project

[jira] [Created] (FLINK-22153) Manually test the sort-merge blocking shuffle

2021-04-08 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-22153: --- Summary: Manually test the sort-merge blocking shuffle Key: FLINK-22153 URL: https://issues.apache.org/jira/browse/FLINK-22153 Project: Flink Issue Type: Task

[jira] [Created] (FLINK-22127) Enrich error message of read buffer request timeout exception

2021-04-06 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-22127: --- Summary: Enrich error message of read buffer request timeout exception Key: FLINK-22127 URL: https://issues.apache.org/jira/browse/FLINK-22127 Project: Flink

[jira] [Created] (FLINK-21951) Fix wrong if condition in BufferReaderWriterUtil#writeBuffers

2021-03-24 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21951: --- Summary: Fix wrong if condition in BufferReaderWriterUtil#writeBuffers Key: FLINK-21951 URL: https://issues.apache.org/jira/browse/FLINK-21951 Project: Flink

[jira] [Created] (FLINK-21859) Batch job fails due to "Could not mark slot 61a637e3977c58a0e6b73533c419297d active"

2021-03-18 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21859: --- Summary: Batch job fails due to "Could not mark slot 61a637e3977c58a0e6b73533c419297d active" Key: FLINK-21859 URL: https://issues.apache.org/jira/browse/F

[jira] [Created] (FLINK-21857) StackOverflow for large parallelism jobs when processing EndOfChannelStateEvent

2021-03-18 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21857: --- Summary: StackOverflow for large parallelism jobs when processing EndOfChannelStateEvent Key: FLINK-21857 URL: https://issues.apache.org/jira/browse/FLINK-21857

[jira] [Created] (FLINK-21850) Improve document and config description of sort-merge blocking shuffle

2021-03-17 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21850: --- Summary: Improve document and config description of sort-merge blocking shuffle Key: FLINK-21850 URL: https://issues.apache.org/jira/browse/FLINK-21850 Project: Flink

[jira] [Created] (FLINK-21790) Shuffle data directories to make data directory section of different TaskManagers fairer

2021-03-15 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21790: --- Summary: Shuffle data directories to make data directory section of different TaskManagers fairer Key: FLINK-21790 URL: https://issues.apache.org/jira/browse/FLINK-21790

[jira] [Created] (FLINK-21789) Make FileChannelManagerImpl#getNextPathNum select data directory fairly

2021-03-15 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21789: --- Summary: Make FileChannelManagerImpl#getNextPathNum select data directory fairly Key: FLINK-21789 URL: https://issues.apache.org/jira/browse/FLINK-21789 Project: Flink

[jira] [Created] (FLINK-21788) Throw PartitionNotFoundException if the partition file has been lost for blocking shuffle

2021-03-15 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21788: --- Summary: Throw PartitionNotFoundException if the partition file has been lost for blocking shuffle Key: FLINK-21788 URL: https://issues.apache.org/jira/browse/FLINK-21788

[jira] [Created] (FLINK-21778) Use heap memory instead of direct memory as index entry cache for sort-merge shuffle

2021-03-13 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21778: --- Summary: Use heap memory instead of direct memory as index entry cache for sort-merge shuffle Key: FLINK-21778 URL: https://issues.apache.org/jira/browse/FLINK-21778

[jira] [Created] (FLINK-21777) Replace the 4M data writing cache of sort-merge shuffle with writev system call

2021-03-13 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-21777: --- Summary: Replace the 4M data writing cache of sort-merge shuffle with writev system call Key: FLINK-21777 URL: https://issues.apache.org/jira/browse/FLINK-21777

[jira] [Created] (FLINK-20758) Use region file mechanism for shuffle data reading before we switch to managed memory

2020-12-23 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-20758: --- Summary: Use region file mechanism for shuffle data reading before we switch to managed memory Key: FLINK-20758 URL: https://issues.apache.org/jira/browse/FLINK-20758

[jira] [Created] (FLINK-20757) Optimize data broadcast for sort-merge shuffle

2020-12-23 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-20757: --- Summary: Optimize data broadcast for sort-merge shuffle Key: FLINK-20757 URL: https://issues.apache.org/jira/browse/FLINK-20757 Project: Flink Issue Type: Sub

[jira] [Created] (FLINK-20740) Use managed memory to avoid direct memory OOM error

2020-12-22 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-20740: --- Summary: Use managed memory to avoid direct memory OOM error Key: FLINK-20740 URL: https://issues.apache.org/jira/browse/FLINK-20740 Project: Flink Issue Type

[jira] [Created] (FLINK-20013) BoundedBlockingSubpartition may leak network buffer if task is failed or canceled

2020-11-05 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-20013: --- Summary: BoundedBlockingSubpartition may leak network buffer if task is failed or canceled Key: FLINK-20013 URL: https://issues.apache.org/jira/browse/FLINK-20013

[jira] [Created] (FLINK-20010) SinkITCase.writerAndCommitterAndGlobalCommitterExecuteInStreamingMode fails on Azure Pipeline

2020-11-05 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-20010: --- Summary: SinkITCase.writerAndCommitterAndGlobalCommitterExecuteInStreamingMode fails on Azure Pipeline Key: FLINK-20010 URL: https://issues.apache.org/jira/browse/FLINK-20010

[jira] [Created] (FLINK-19991) UnalignedCheckpointITCase#shouldPerformUnalignedCheckpointMassivelyParallel fails on Azure Pipeline

2020-11-05 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-19991: --- Summary: UnalignedCheckpointITCase#shouldPerformUnalignedCheckpointMassivelyParallel fails on Azure Pipeline Key: FLINK-19991 URL: https://issues.apache.org/jira/browse/FLINK

[jira] [Created] (FLINK-19938) Implement shuffle data read scheduling for sort-merge blocking shuffle

2020-11-02 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-19938: --- Summary: Implement shuffle data read scheduling for sort-merge blocking shuffle Key: FLINK-19938 URL: https://issues.apache.org/jira/browse/FLINK-19938 Project: Flink

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

2020-10-25 Thread Yingjie Cao
Hi devs, I'd like to start a vote for FLIP-148: Introduce Sort-Merge Based Blocking Shuffle to Flink [1] which is discussed in discussion thread [2]. The vote will last for at least 72 hours until a consensus voting. [1]

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

2020-10-23 Thread Yingjie Cao
Hi devs, This discussion thread has been opened for over a week. If there is no other concerns, I'd like to open a voting thread soon. Best, Yingjie Yingjie Cao 于2020年10月23日周五 上午11:56写道: > Hi Zhijiang, > > Thanks for your reply and suggestions. > > 1. For > taskmanager.

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

2020-10-22 Thread Yingjie Cao
huffle to Flink > > Thanks for sharing the preliminary numbers with us Yingjie. The numbers > look quite impressive :-) > > Cheers, > Till > > On Thu, Oct 15, 2020 at 5:25 PM Yingjie Cao > wrote: > > > Hi Till, > > > > Thanks for your rep

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

2020-10-15 Thread Yingjie Cao
; up when using the sort-merge based shuffle? > > Cheers, > Till > > On Thu, Oct 15, 2020 at 5:03 AM Yingjie Cao > wrote: > > > Hi devs, > > > > Currently, Flink adopts a hash-style blocking shuffle implementation > which > > writes data sent to differ

  1   2   >