You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Yingjie Cao (Jira)" <ji...@apache.org> on 2022/01/11 08:37:00 UTC

[jira] [Commented] (FLINK-25606) Requesting exclusive buffers timeout when recovering from unaligned checkpoint under fine-grained resource mode

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

Yingjie Cao commented on FLINK-25606:
-------------------------------------

After some analysis, we find that both recovered channel and remote channel can occupy exclusive buffers at the same time, which may increase the number of network buffers needed, that is to say, the total number of total network buffers is enough for job running but may be not enough for recovering from checkpoint. Under fine-grained resource mode, the number of network buffers is calculated which is limited and can exactly support job running.

To fix the issue, we need to first free the exclusive buffers taken by recovered channel and then initialize the remote channel.

(I am not sure if there are other reasons which caused the exception.)

> Requesting exclusive buffers timeout when recovering from unaligned checkpoint under fine-grained resource mode
> ---------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-25606
>                 URL: https://issues.apache.org/jira/browse/FLINK-25606
>             Project: Flink
>          Issue Type: Bug
>            Reporter: Yingjie Cao
>            Priority: Major
>
> When converting the RecoveredInputChannel to RemoteInputChannel, the network buffer is not enough to initialize input channel exclusive buffers. Here is the exception stack:
> {code:java}
> java.io.IOException: Timeout triggered when requesting exclusive buffers: The total number of network buffers is currently set to 6144 of 32768 bytes each. You can increase this number by setting the configuration keys 'taskmanager.memory.network.fraction', 'taskmanager.memory.network.min', and 'taskmanager.memory.network.max',  or you may increase the timeout which is 30000ms by setting the key 'taskmanager.network.memory.exclusive-buffers-request-timeout-ms'.
>   at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.requestMemorySegments(NetworkBufferPool.java:205)
>   at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.requestMemorySegments(NetworkBufferPool.java:60)
>   at org.apache.flink.runtime.io.network.partition.consumer.BufferManager.requestExclusiveBuffers(BufferManager.java:133)
>   at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.setup(RemoteInputChannel.java:157)
>   at org.apache.flink.runtime.io.network.partition.consumer.RemoteRecoveredInputChannel.toInputChannelInternal(RemoteRecoveredInputChannel.java:77)
>   at org.apache.flink.runtime.io.network.partition.consumer.RecoveredInputChannel.toInputChannel(RecoveredInputChannel.java:106)
>   at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.convertRecoveredInputChannels(SingleInputGate.java:307)
>   at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.requestPartitions(SingleInputGate.java:290)
>   at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.requestPartitions(InputGateWithMetrics.java:94)
>   at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50)
>   at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90)
>   at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsNonBlocking(MailboxProcessor.java:359)
>   at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:323)
>   at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:202)
>   at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:684)
>   at org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:639)
>   at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:650)
>   at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:623)
>   at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:779)
>   at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
>   at java.lang.Thread.run(Thread.java:834) {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)