You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/21 11:39:41 UTC

[GitHub] [flink] rkhachatryan opened a new pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

rkhachatryan opened a new pull request #13723:
URL: https://github.com/apache/flink/pull/13723


   ## What is the purpose of the change
   
   This is a fix for the following case(s):
   1. clientA requests a connection
   2. clientB  requests the same connection and waits on the same future
   3. clientA is interrupted; `PartitionRequestClientFactory` caches the `InterruptedException`
   4. clientB as well as all subsequent clients get cached `InterruptedException`
   
   ## Verifying this change
   
   Added `PartitionRequestClientFactoryTest.testInterruptsNotCached`
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? no
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] AHeise commented on a change in pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13723:
URL: https://github.com/apache/flink/pull/13723#discussion_r509264809



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
##########
@@ -58,6 +59,38 @@
 
 	private static final int SERVER_PORT = NetUtils.getAvailablePort();
 
+	@Test
+	public void testInterruptsNotCached() throws Exception {
+		ConnectionID connectionId = new ConnectionID(new InetSocketAddress(InetAddress.getLocalHost(), 8080), 0);

Review comment:
       Sorry for the stupid question, but is this actually using the port for network traffic or is it just a dummy value?
   If it's actually using, I'd propose to use port 0 (random free port).

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NeverCompletingChannelFuture.java
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.netty;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.Future;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.GenericFutureListener;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+@SuppressWarnings({"InfiniteLoopStatement", "BusyWait"})
+class NeverCompletingChannelFuture implements ChannelFuture {

Review comment:
       Could we use `CompleteChannelFuture` as a base to avoid all the no-ops?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
##########
@@ -75,12 +73,24 @@ NettyPartitionRequestClient createPartitionRequestClient(ConnectionID connection
 				return new CompletableFuture<>();
 			});
 			if (isTheFirstOne.get()) {
-				completeFromCallable(clientFuture, () -> connectWithRetries(connectionId));
+				try {
+					clientFuture.complete(connectWithRetries(connectionId));
+				} catch (InterruptedException e) {
+					clientFuture.complete(null); // let know others waiting that they should retry

Review comment:
       nit: let others know




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] AHeise commented on a change in pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13723:
URL: https://github.com/apache/flink/pull/13723#discussion_r509969318



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
##########
@@ -58,6 +59,38 @@
 
 	private static final int SERVER_PORT = NetUtils.getAvailablePort();
 
+	@Test
+	public void testInterruptsNotCached() throws Exception {
+		ConnectionID connectionId = new ConnectionID(new InetSocketAddress(InetAddress.getLocalHost(), 8080), 0);

Review comment:
       Okay if the test is not affected by that I'll merge.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13723:
URL: https://github.com/apache/flink/pull/13723#issuecomment-713525013


   I have verified that the change is fixing the problem I initially reported with my test setup.
   
   I'll leave a full review to somebody more familiar with the network stack.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13723:
URL: https://github.com/apache/flink/pull/13723#issuecomment-713526541


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8014",
       "triggerID" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "triggerType" : "PUSH"
     }, {
       "hash" : "71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 18772dddf0714c8ee47354a40d70f70ab9ac6387 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8014) 
   * 71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rkhachatryan commented on a change in pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #13723:
URL: https://github.com/apache/flink/pull/13723#discussion_r509616571



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
##########
@@ -58,6 +59,38 @@
 
 	private static final int SERVER_PORT = NetUtils.getAvailablePort();
 
+	@Test
+	public void testInterruptsNotCached() throws Exception {
+		ConnectionID connectionId = new ConnectionID(new InetSocketAddress(InetAddress.getLocalHost(), 8080), 0);

Review comment:
       The request will likely fail (usually nothing is running on 8080), but the result is ignored, so it won't affect the test.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13723:
URL: https://github.com/apache/flink/pull/13723#issuecomment-713526541


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 18772dddf0714c8ee47354a40d70f70ab9ac6387 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13723:
URL: https://github.com/apache/flink/pull/13723#issuecomment-713508588


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 18772dddf0714c8ee47354a40d70f70ab9ac6387 (Wed Oct 21 11:41:40 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13723:
URL: https://github.com/apache/flink/pull/13723#issuecomment-713526541


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8014",
       "triggerID" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 18772dddf0714c8ee47354a40d70f70ab9ac6387 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8014) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] AHeise merged pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
AHeise merged pull request #13723:
URL: https://github.com/apache/flink/pull/13723


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rkhachatryan commented on a change in pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #13723:
URL: https://github.com/apache/flink/pull/13723#discussion_r509316058



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NeverCompletingChannelFuture.java
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.netty;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.Future;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.GenericFutureListener;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+@SuppressWarnings({"InfiniteLoopStatement", "BusyWait"})
+class NeverCompletingChannelFuture implements ChannelFuture {

Review comment:
       It requires a non-null channel and I wanted to avoid creating a channel unnecessarily. I also couldn't find any suitable channel implementation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] AHeise commented on a change in pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13723:
URL: https://github.com/apache/flink/pull/13723#discussion_r509609977



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
##########
@@ -58,6 +59,38 @@
 
 	private static final int SERVER_PORT = NetUtils.getAvailablePort();
 
+	@Test
+	public void testInterruptsNotCached() throws Exception {
+		ConnectionID connectionId = new ConnectionID(new InetSocketAddress(InetAddress.getLocalHost(), 8080), 0);

Review comment:
       I'm more worried that the test will fail if something else is running on that port.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13723:
URL: https://github.com/apache/flink/pull/13723#issuecomment-713526541


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8014",
       "triggerID" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "triggerType" : "PUSH"
     }, {
       "hash" : "71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8032",
       "triggerID" : "71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8032) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rkhachatryan commented on pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on pull request #13723:
URL: https://github.com/apache/flink/pull/13723#issuecomment-713508011


   cc: @rmetzger 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13723:
URL: https://github.com/apache/flink/pull/13723#issuecomment-713526541


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8014",
       "triggerID" : "18772dddf0714c8ee47354a40d70f70ab9ac6387",
       "triggerType" : "PUSH"
     }, {
       "hash" : "71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8032",
       "triggerID" : "71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 18772dddf0714c8ee47354a40d70f70ab9ac6387 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8014) 
   * 71c15ebc8f5ee9d06956bdb4e0f7ff36b7a3a656 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8032) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rkhachatryan commented on a change in pull request #13723: [FlINK-19688][network] Don't cache InterruptedExceptions in PartitionRequestClientFactory

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #13723:
URL: https://github.com/apache/flink/pull/13723#discussion_r509315036



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
##########
@@ -58,6 +59,38 @@
 
 	private static final int SERVER_PORT = NetUtils.getAvailablePort();
 
+	@Test
+	public void testInterruptsNotCached() throws Exception {
+		ConnectionID connectionId = new ConnectionID(new InetSocketAddress(InetAddress.getLocalHost(), 8080), 0);

Review comment:
       It's a port netty will to connect to. So it doesn't matter which port to use (as long we aren't interested in any results).
   This port is also used throughout the class.




----------------------------------------------------------------
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:
users@infra.apache.org