Hi Thomas,

The current calculation for network buffer size is 
"Math.min(taskmanager.network.memory.max, 
Math.max(taskmanager.network.memory.min, fraction * totalMem))".
Based on your below configuration, the result is just 32768 bytes (8 buffers) 
from taskmanager.network.memory.min.
If you want to config the fixed network buffers, you can set the same values 
for min and max parameters, ignore the fraction value in the configuration.
BTW, you can set the unit for these paramenters, such as min : 32kb.

As for the thoughts of min and max setting,  it is difficult to exactly know 
how many network buffers are needed in TaskManager startup and which kinds of 
tasks would be deployed to run in this TaskManager later.  For example, the 
batch jobs can make use of as many network buffers as system can spare. But for 
stream jobs, the spare buffers can be used for other places for possible 
improvements. In order to keep the possibility of future improments and not 
change the configuration setting, we retain these current parameters.

For you job I think you should increase the min value for more network buffers, 
the current 4 buffers are indeed not enough for common jobs.

Best,
Zhijiang
------------------------------------------------------------------
发件人:Thomas Weise <t...@apache.org>
发送时间:2018年11月13日(星期二) 13:11
收件人:dev <dev@flink.apache.org>
主 题:Memory size for network buffers

Hi,

I'm trying to understand the intention behind the size parameters for
network buffers, specifically max, min and fraction. The reason we are
looking at it is an attempt to make the memory allocation elastic, so that
memory is allocated according to the actual number of buffers required
(within a range), without the need to tune this for every deployment.

As of Flink 1.5, there are 3 parameters, but they all result in a fixed
allocation, which is not what we were looking for.

Here is an example just to illustrate it:

taskmanager.network.memory.fraction: 0.000001
taskmanager.network.memory.min: 32768
taskmanager.network.memory.max: 1073741824
taskmanager.memory.segment-size: 8192

I wanted fraction to be out of the picture (but 0 isn't an acceptable
value).

Then set min to something tiny that my job will exceed and max to something
too large to reach. Unfortunately, that fails:

java.io.IOException: Insufficient number of network buffers: required 8,
but only 0 available. The total number of network buffers is currently set
to 4 of 8192 bytes each. You can increase this number by setting the
configuration keys 'taskmanager.network.memory.fraction',
'taskmanager.network.memory.min', and 'taskmanager.network.memory.max'.

So the question then is, why have min and max? Or is the intention to have
a different implementation in the future?

Thanks,
Thomas

Reply via email to