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

ASF GitHub Bot commented on FLINK-4545:
---------------------------------------

Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3721#discussion_r111994207
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
 ---
    @@ -376,6 +392,169 @@ private static NetworkEnvironment 
createNetworkEnvironment(
        }
     
        /**
    +    * Calculates the amount of memory used for network buffers based on 
the total memory to use and
    +    * the according configuration parameters.
    +    *
    +    * The following configuration parameters are involved:
    +    * <ul>
    +    *  <li>{@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_FRACTION},</li>
    +    *      <li>{@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_MIN},</li>
    +    *      <li>{@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_MAX}, 
and</li>
    +    *  <li>{@link TaskManagerOptions#NETWORK_NUM_BUFFERS} (fallback if the 
ones above do not exist)</li>
    +    * </ul>.
    +    *
    +    * @param totalJavaMemorySize
    +    *              overall available memory to use (heap and off-heap, in 
bytes)
    +    * @param config
    +    *              configuration object
    +    *
    +    * @return memory to use for network buffers (in bytes)
    +    */
    +   public static long calculateNetworkBuf(long totalJavaMemorySize, 
Configuration config) {
    --- End diff --
    
    how about a slightly longer method name?


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

Reply via email to