Caizhi Weng created FLINK-18821:
-----------------------------------

             Summary: 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.11.0
            Reporter: Caizhi Weng


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)

Reply via email to