flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "zhijiang (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (FLINK-4545) Flink automatically manages TM network buffer
Date Fri, 17 Mar 2017 09:58:41 GMT

    [ https://issues.apache.org/jira/browse/FLINK-4545?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15929676#comment-15929676
] 

zhijiang edited comment on FLINK-4545 at 3/17/17 9:58 AM:
----------------------------------------------------------

[~NicoK], [~StephanEwen], thank you for your replies!

As I know, we already introduced another two parameters for users to control the core buffer
amount in {{LocalBufferPool}}, {{taskmanager.net.memory.buffers-per-channel}}(default: 2)
and {{taskmanager.net.memory.extra-buffers-per-gate}}(default: 8) separately. And it is easy
for users to understand them.

In my opinion , the {{JobManager}} or {{ResourceManager}} can decide the total buffer amount
in {{NetworkBufferPool}} based on two conditions:
   - The number of tasks would be deployed into this {{TaskManager}}.
   - The core number of buffers in {{LocalBufferPool}}, maybe extra consider {{ResultPartitionType}}.

So the framework can calculate the total buffer amount and corresponding memory usage before
starting the {{TaskManager}}. The buffer amount will be set onto {{TaskManager}} and memory
usage will be aggregated into the total resource request. It seems consistent.

Have you considered not to expose the global network memory parameters to users, and just
allow the parameters in {{LocalBufferPool}}.Otherwise the users should consider both global
parameters and local parameters, and may also understand the total number of tasks in {{TaskManager}}.
It is easy to cause conflict.  The less parameters users know, the better.

Maybe you have other concerns that I have not covered yet, wish your advices and I am very
willing to do something for it if needed.


was (Author: zjwang):
[~NicoK], [~StephanEwen], thank you for your replies!

As I know, we already introduced another two parameters for users to control the core buffer
amount in {{LocalBufferPool}}, {{taskmanager.net.memory.buffers-per-channel}}(default: 2)
and {{taskmanager.net.memory.extra-buffers-per-gate}}(default: 8) separately. And it is easy
for users to understand them.

In my opinion , the {{JobManager}} or {{ResourceManager}} can decide the total buffer amount
in {{NetworkBufferPool}} based on two conditions:
   - The number of tasks would be deployed into this {{TaskManager}}.
   - The core number of buffers in {{LocalBufferPool}}, maybe extra consider {{ResultPartitionType}}.

So the framework can calculate the total buffer amount and corresponding memory usage before
starting the {{TaskManager}}. The buffer amount will be set onto {{TaskManager}} and memory
usage will be aggregated into the total resource request. It seems consistent.

Have you considered not to expose the global network memory parameters to users, and just
allow the parameters in {{LocalBufferPool}}.
Otherwise the users should consider both global parameters and local parameters, and may also
understand the total number of tasks in {{TaskManager}}. It is easy to cause conflict.  The
less parameters users know, the better.

Maybe you have other concerns that I have not covered yet, wish your advices and I am very
willing to do something for it if needed.

> Flink automatically manages TM network buffer
> ---------------------------------------------
>
>                 Key: FLINK-4545
>                 URL: https://issues.apache.org/jira/browse/FLINK-4545
>             Project: Flink
>          Issue Type: Wish
>          Components: Network
>            Reporter: Zhenzhong Xu
>
> Currently, the number of network buffer per task manager is preconfigured and the memory
is pre-allocated through taskmanager.network.numberOfBuffers config. In a Job DAG with shuffle
phase, this number can go up very high depends on the TM cluster size. The formula for calculating
the buffer count is documented here (https://ci.apache.org/projects/flink/flink-docs-master/setup/config.html#configuring-the-network-buffers).
 
> #slots-per-TM^2 * #TMs * 4
> In a standalone deployment, we may need to control the task manager cluster size dynamically
and then leverage the up-coming Flink feature to support scaling job parallelism/rescaling
at runtime. 
> If the buffer count config is static at runtime and cannot be changed without restarting
task manager process, this may add latency and complexity for scaling process. I am wondering
if there is already any discussion around whether the network buffer should be automatically
managed by Flink or at least expose some API to allow it to be reconfigured. Let me know if
there is any existing JIRA that I should follow.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message