rkhachatryan commented on a change in pull request #11541:
URL: https://github.com/apache/flink/pull/11541#discussion_r442869876



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
##########
@@ -131,6 +134,42 @@ void destroyPartitionRequestClient(ConnectionID 
connectionId, PartitionRequestCl
                clients.remove(connectionId, client);
        }
 
+       private NettyPartitionRequestClient 
connectChannelWithRetry(ConnectingChannel connectingChannel,
+                                                                               
 ConnectionID connectionId, boolean needConnect)
+               throws IOException, InterruptedException {
+               int count = 0;
+               Exception exception = null;
+               do {
+                       try {
+                               if (needConnect) {
+                                       LOG.info("Connecting to {} at {} 
attempt", connectionId.getAddress(), count);
+                                       
nettyClient.connect(connectionId.getAddress()).addListener(connectingChannel);
+                               }
+
+                               NettyPartitionRequestClient client = 
connectingChannel.waitForChannel();
+                               clients.replace(connectionId, 
connectingChannel, client);
+                               return client;
+                       } catch (IOException | ChannelException e) {
+                               LOG.error("Failed {} times to connect to {}", 
count, connectionId.getAddress(), e);

Review comment:
       I think it will print `0 times` for the 1st failure and `1 times` for 
the 2nd. Should  `count + 1` be used?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
##########
@@ -131,6 +134,42 @@ void destroyPartitionRequestClient(ConnectionID 
connectionId, PartitionRequestCl
                clients.remove(connectionId, client);
        }
 
+       private NettyPartitionRequestClient 
connectChannelWithRetry(ConnectingChannel connectingChannel,
+                                                                               
 ConnectionID connectionId, boolean needConnect)
+               throws IOException, InterruptedException {
+               int count = 0;
+               Exception exception = null;
+               do {
+                       try {
+                               if (needConnect) {
+                                       LOG.info("Connecting to {} at {} 
attempt", connectionId.getAddress(), count);
+                                       
nettyClient.connect(connectionId.getAddress()).addListener(connectingChannel);
+                               }
+
+                               NettyPartitionRequestClient client = 
connectingChannel.waitForChannel();
+                               clients.replace(connectionId, 
connectingChannel, client);
+                               return client;
+                       } catch (IOException | ChannelException e) {
+                               LOG.error("Failed {} times to connect to {}", 
count, connectionId.getAddress(), e);
+                               ConnectingChannel newConnectingChannel = new 
ConnectingChannel(connectionId, this);
+                               clients.replace(connectionId, 
connectingChannel, newConnectingChannel);

Review comment:
       If the exception is thrown by `connect` and not `waitForChannel` then we 
use a new channel.
   But other threads are waiting on the old channel - which will never be 
completed, right?
   
   I guess this is the reason of the deadlock I see in Azure and locally.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to