Hi Pritam,

The legacy config option `taskmanager.network.numberOfBuffers` is
deprecated and will be ignored, so please do not refer to it.
The exact size of network memory calculated through
`taskmanager.memory.network.fraction`, while also ensuring that it is
within the constraints of min and max value. So it can be explicitly
specified by setting the min & max to the same value if you want. Another
thing to note is that the fraction is relative to flink memory, not process
memory(8gb in your example), and there is a difference of
jvmMetaspaceAndOverload memory part between them.


Best regards,

Weijie


Hangxiang Yu <master...@gmail.com> 于2023年6月7日周三 10:44写道:

> Hi, Pritam.
> IIUC, the number is TM scope and just calculated by "available network
> memory / buffer size".
> For example, if the fraction is 0.1, the number may be about ( 8 * 0.1 *
> 1024 * 1024 * 1024 / 32768).
>
>
> On Tue, Jun 6, 2023 at 3:14 PM Pritam Agarwala <
> pritamagarwala...@gmail.com> wrote:
>
>> Thanks for answering Hangxiang!
>>
>> Still confused. How did Flink get this number 22773 ? Couldn't find the
>> default value of "taskmanager.network.numberOfBuffers" config.
>> and According to the formula it should be around 1000. ( #slots-per-TM^2
>> * #TMs * 4 = 4^2 * 16 * 4 = 1024)
>>
>> I have a total of 6000 tasks with 16 TM , 4 cores each with
>> jobmanger/taskmanger.momry.process.size = 8 gb .
>>
>>
>> Thanks & Regards,
>> Pritam
>>
>>
>>
>> On Tue, Jun 6, 2023 at 9:02 AM Hangxiang Yu <master...@gmail.com> wrote:
>>
>>> Hi, Pritam.
>>> This error message indicates that the current configuration of the
>>> network buffer is not enough to handle the current workload.
>>>
>>>> What is the meaning of this exception (The total number of network
>>>> buffers is currently set to 22773 of 32768 bytes each)?
>>>>
>>> This just provides some information about the current status of network
>>> buffers (22773 * 32768 bytes ~= 711MB).
>>>
>>> How to figure out a good combination of
>>>> ('taskmanager.memory.network.fraction', 'taskmanager.memory.network.min',
>>>> and 'taskmanager.memory.network.max'.)
>>>> for this issue ?
>>>>
>>> IIUC,  There is no absolute standard for setting these parameters.
>>> These parameters may be affected by many factors, such as the data flow
>>> rate, computational complexity, and memory usage of your job.
>>>
>>> Some steps to setup and adjust these parameters:
>>> 1. Check the available memory on your job.
>>> 2. Evaluate the network usage and consider how much memory could be
>>> used for network buffers.
>>> 3. Monitor the system and collect metrics such as network throughput,
>>> memory usage.
>>> 4. Adjust these parameters if the job has a high network usage or
>>> memory-intensive.
>>>
>>> Just a personal and immature suggestion about how to adjust when it's
>>> not enough:  1. Increase taskmanager.memory.network.fraction from 0.1
>>> to 0.2, or just increase taskmanager.memory.network.max slightly.
>>> 2. If the buffer size is too large, it may affect checkpoints. So it's
>>> recommended to combine with buffer debloating.
>>> <https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/memory/network_mem_tuning/#the-buffer-debloating-mechanism>
>>>
>>>
>>> On Tue, Jun 6, 2023 at 2:44 AM Pritam Agarwala <
>>> pritamagarwala...@gmail.com> wrote:
>>>
>>>> Hi All,
>>>>
>>>>
>>>> java.io.IOException: Insufficient number of network buffers: required
>>>> 2, but only 0 available. The total number of network buffers is currently
>>>> set to 22773 of 32768 bytes each.
>>>>
>>>> What is the meaning of this exception (The total number of network
>>>> buffers is currently set to 22773 of 32768 bytes each)?
>>>>
>>>> How to figure out a good combination of
>>>> ('taskmanager.memory.network.fraction', 'taskmanager.memory.network.min',
>>>> and 'taskmanager.memory.network.max'.)
>>>> for this issue ?
>>>>
>>>>
>>>> Thanks & Regards,
>>>> Pritam
>>>>
>>>
>>>
>>> --
>>> Best,
>>> Hangxiang.
>>>
>>
>
> --
> Best,
> Hangxiang.
>

Reply via email to