Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-18821

Netty client retry mechanism may cause PartitionRequestClientFactory#createPartitionRequestClient to wait infinitely

    XMLWordPrintableJSON

Details

    Description

      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:

      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
      

      Attachments

        Activity

          People

            roman Roman Khachatryan
            TsReaper Caizhi Weng
            Votes:
            0 Vote for this issue
            Watchers:
            8 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: