You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Zhijiang (Jira)" <ji...@apache.org> on 2020/08/12 08:31:00 UTC

[jira] [Closed] (FLINK-18821) Netty client retry mechanism may cause PartitionRequestClientFactory#createPartitionRequestClient to wait infinitely

     [ https://issues.apache.org/jira/browse/FLINK-18821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Zhijiang closed FLINK-18821.
----------------------------
    Resolution: Fixed

Merged in release-1.11: ded539e85444f9c8d34f88da31784775ed281aba
Merged in master: 33bdc978a059c50ec55b8f24d40039d13a6b78e7

> Netty client retry mechanism may cause PartitionRequestClientFactory#createPartitionRequestClient to wait infinitely
> --------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-18821
>                 URL: https://issues.apache.org/jira/browse/FLINK-18821
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Network
>    Affects Versions: 1.10.1, 1.12.0, 1.11.1
>            Reporter: Caizhi Weng
>            Assignee: Roman Khachatryan
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.12.0, 1.11.2
>
>
> When running TPCDS 10T benchmark on Flink I found some of the task slots stuck. After some investigation there seems to be a bug inĀ {{PartitionRequestClientFactory}}.
> When a task tries to require a partition of data from its upstream task but fails, {{PartitionRequestClientFactory#connect}} will throw {{RemoteTransportException}} and {{PartitionRequestClientFactory#connectWithRetries}} will throw {{CompletionException}}. However this exception is not caught by {{PartitionRequestClientFactory#connect}} and it will eventually fail the task.
> But {{PartitionRequestClientFactory}} lives in a task manager not in a task. In {{PartitionRequestClientFactory}} a {{ConcurrentHashMap}} named {{clients}} is maintained for reusing {{NettyPartitionRequestClient}}. When the above exception happens, {{clients}} is not cleaned up; When the next call to {{PartitionRequestClientFactory#connect}} with the same connection id comes, it will use the invalid {{CompletableFuture}} in {{clients}} and this future will never complete, causing the task to stuck forever.
> Exception stack:
> {code}
> 2020-08-05 03:37:07,539 ERROR org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory [] - Failed 1 times to connect to <host-name>/<ip>:<port>
> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException: Connecting to remote task manager '<host-name>/<ip>:<port>' has failed. This might indicate that the remote task manager has been lost.
> 	at org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory.connect(PartitionRequestClientFactory.java:120) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory.connectWithRetries(PartitionRequestClientFactory.java:99) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory.createPartitionRequestClient(PartitionRequestClientFactory.java:76) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.io.network.netty.NettyConnectionManager.createPartitionRequestClient(NettyConnectionManager.java:67) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.requestSubpartition(RemoteInputChannel.java:146) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.internalRequestPartitions(SingleInputGate.java:329) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.requestPartitions(SingleInputGate.java:301) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.requestPartitions(InputGateWithMetrics.java:95) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.requestPartitions(StreamTask.java:514) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.readRecoveredChannelState(StreamTask.java:484) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:475) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:469) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:522) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at java.lang.Thread.run(Thread.java:834) [?:1.8.0_102]
> Caused by: java.lang.NullPointerException
> 	at org.apache.flink.util.Preconditions.checkNotNull(Preconditions.java:58) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.io.network.netty.NettyPartitionRequestClient.<init>(NettyPartitionRequestClient.java:73) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	at org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory.connect(PartitionRequestClientFactory.java:114) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> 	... 16 more
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)