Yangze, Thank you for the feedback!
Here's my thoughts for your questions:

>>> How do we decide the size of the buffer pool in MemoryDataManager and
the read buffers in FileDataManager?
The BufferPool in MemoryDataManager is the LocalBufferPool used by
ResultPartition, and the size is the same as the current implementation of
sort-merge shuffle. In other words, the minimum value of BufferPool is a
configurable fixed value, and the maximum value is Math.max(min, 4 *
numSubpartitions). The default value can be determined by running the
TPC-DS tests.
Read buffers in FileDataManager are requested from the
BatchShuffleReadBufferPool shared by TaskManager, it's size controlled by
*taskmanager.memory.framework.off-heap.batch-shuffle.size*, the default
value is 32M, which is consistent with the current sort-merge shuffle logic.

>>> Is there an upper limit for the sum of them? If there is, how does
MemoryDataManager and FileDataManager sync the memory usage?
The buffers of the MemoryDataManager are limited by the size of the
LocalBufferPool, and the upper limit is the size of the Network Memory. The
buffers of the FileDataManager are directly requested from
UnpooledOffHeapMemory, and are also limited by the size of the framework
off-heap memory. I think there should be no need for additional
synchronization mechanisms.

>>> How do you disable the slot sharing? If user configures both the slot
sharing group and hybrid shuffle, what will happen to that job?
I think we can print a warning log when Hybrid Shuffle is enabled and SSG
is configured during the JobGraph compilation stage, and fallback to the
region slot sharing group by default. Of course, it will be emphasized in
the document that we do not currently support SSG, If configured, it will
fall back to the default.


Best regards,

Weijie


Yangze Guo <karma...@gmail.com> 于2022年5月19日周四 16:25写道:

> Thanks for driving this. Xintong and Weijie.
>
> I believe this feature will make Flink a better batch/OLAP engine. +1
> for the overall design.
>
> Some questions:
> 1. How do we decide the size of the buffer pool in MemoryDataManager
> and the read buffers in FileDataManager?
> 2. Is there an upper limit for the sum of them? If there is, how does
> MemoryDataManager and FileDataManager sync the memory usage?
> 3. How do you disable the slot sharing? If user configures both the
> slot sharing group and hybrid shuffle, what will happen to that job?
>
>
> Best,
> Yangze Guo
>
> On Thu, May 19, 2022 at 2:41 PM Xintong Song <tonysong...@gmail.com>
> wrote:
> >
> > Thanks for preparing this FLIP, Weijie.
> >
> > I think this is a good improvement on batch resource elasticity. Looking
> > forward to the community feedback.
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Thu, May 19, 2022 at 2:31 PM weijie guo <guoweijieres...@gmail.com>
> > wrote:
> >
> > > Hi all,
> > >
> > >
> > > I’d like to start a discussion about FLIP-235[1], which introduce a
> new shuffle mode
> > >  can overcome some of the problems of Pipelined Shuffle and Blocking
> Shuffle in batch scenarios.
> > >
> > >
> > > Currently in Flink, task scheduling is more or less constrained by the
> shuffle implementations.
> > > This will bring the following disadvantages:
> > >
> > >    1. Pipelined Shuffle:
> > >     For pipelined shuffle, the upstream and downstream tasks are
> required to be deployed at the same time, to avoid upstream tasks being
> blocked forever. This is fine when there are enough resources for both
> upstream and downstream tasks to run simultaneously, but will cause the
> following problems otherwise:
> > >    1.
> > >       Pipelined shuffle connected tasks (i.e., a pipelined region)
> cannot be executed until obtaining resources for all of them, resulting in
> longer job finishing time and poorer resource efficiency due to holding
> part of the resources idle while waiting for the rest.
> > >       2.
> > >       More severely, if multiple jobs each hold part of the cluster
> resources and are waiting for more, a deadlock would occur. The chance is
> not trivial, especially for scenarios such as OLAP where concurrent job
> submissions are frequent.
> > >       2. Blocking Shuffle:
> > >     For blocking shuffle, execution of downstream tasks must wait for
> all upstream tasks to finish, despite there might be more resources
> available. The sequential execution of upstream and downstream tasks
> significantly increase the job finishing time, and the disk IO workload for
> spilling and loading full intermediate data also affects the performance.
> > >
> > >
> > > We believe the root cause of the above problems is that shuffle
> implementations put unnecessary constraints on task scheduling.
> > >
> > >
> > > To solve this problem, Xintong Song and I propose to introduce hybrid
> shuffle to minimize the scheduling constraints. With Hybrid Shuffle, Flink
> should:
> > >
> > >    1. Make best use of available resources.
> > >     Ideally, we want Flink to always make progress if possible. That
> is to say, it should always execute a pending task if there are resources
> available for that task.
> > >    2. Minimize disk IO load.
> > >     In-flight data should be consumed directly from memory as much as
> possible. Only data that is not consumed timely should be spilled to disk.
> > >
> > > You can find more details in FLIP-235. Looking forward to your
> feedback.
> > >
> > >
> > > [1]
> > >
> > >
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-235%3A+Hybrid+Shuffle+Mode
> > >
> > >
> > >
> > > Best regards,
> > >
> > > Weijie
> > >
>

Reply via email to