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/08/05 09:27:22 UTC

[GitHub] [flink] rkhachatryan opened a new pull request #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   ## What is the purpose of the change
   
   Currently, an exception in `PartitionRequestClientFactory` is propagated to the client directly. 
   The future in the internal `PartitionRequestClientFactory` map is not completed exceptionally.
   This causes subsequent calls to wait indefinitely for the future to complete (see FLINK-18821).
   
   This PR fixes it by completing the future exceptionally in case of error.
   
   ## Verifying this change
   
   Added unit test: `PartitionRequestClientFactoryTest.testFailureReportedToSubsequentRequests`
   
   ## 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] wsry commented on pull request #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   The fix itself LGTM, but I am afraid there is another issue we may also need to fix. It seems to me the exceptionally completed future in the ```clients``` map is not cleaned up correctly. Am I right? If so, I think it is better to also fix it.


----------------------------------------------------------------
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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -90,6 +90,21 @@
 		}
 	}
 
+	/**
+	 * Fakes asynchronous execution by immediately executing the operation and completing the supplied future
+	 * either noramlly or exceptionally.
+	 *
+	 * @param operation to executed
+	 * @param <T> type of the result
+	 */
+	public static <T> void completeFromCallable(CompletableFuture<T> future, Callable<T> operation) {

Review comment:
       Wanted to avoid extra CI run :)
   OK, added this commit.




----------------------------------------------------------------
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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -90,6 +90,21 @@
 		}
 	}
 
+	/**
+	 * Fakes asynchronous execution by immediately executing the operation and completing the supplied future
+	 * either noramlly or exceptionally.
+	 *
+	 * @param operation to executed
+	 * @param <T> type of the result
+	 */
+	public static <T> void completeFromCallable(CompletableFuture<T> future, Callable<T> operation) {

Review comment:
       I added a similar method to `FutureUtils` called `runSync` (but it creates a new future). Would it make sense to keep the names in sync? (I have no hard feeling about renaming `runSync`).




----------------------------------------------------------------
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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202",
       "triggerID" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3ad391b48b7ffc4a5ebd3717e0f99808522cb28 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202) 
   
   <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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3ad391b48b7ffc4a5ebd3717e0f99808522cb28 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 edited a comment on pull request #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202",
       "triggerID" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9912d6fd64757bce996b438befee462f1debc2ea",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9912d6fd64757bce996b438befee462f1debc2ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3ad391b48b7ffc4a5ebd3717e0f99808522cb28 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202) 
   * 9912d6fd64757bce996b438befee462f1debc2ea 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] AHeise commented on a change in pull request #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -90,6 +90,21 @@
 		}
 	}
 
+	/**
+	 * Fakes asynchronous execution by immediately executing the operation and completing the supplied future
+	 * either noramlly or exceptionally.
+	 *
+	 * @param operation to executed
+	 * @param <T> type of the result
+	 */
+	public static <T> void completeFromCallable(CompletableFuture<T> future, Callable<T> operation) {

Review comment:
       I'd probably go by hotfix commit, doesn't deserve a separate PR.




----------------------------------------------------------------
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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   Yes, you are right, exceptionally completed future stays in the map.
   But I think this was the original intention in `PartitionRequestClientFactory`. Otherwise, new requests would retry again almost immediately.
   
   I think ideally such a future should be removed after some delay (regardless of client requests). But this seems to be out of scope of this PR (which removes indefinite wait). WDYT about addressing it separately?
   


----------------------------------------------------------------
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] wsry commented on pull request #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   Addressing the issue separately is also reasonable to me.


----------------------------------------------------------------
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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202",
       "triggerID" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9912d6fd64757bce996b438befee462f1debc2ea",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5247",
       "triggerID" : "9912d6fd64757bce996b438befee462f1debc2ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3ad391b48b7ffc4a5ebd3717e0f99808522cb28 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202) 
   * 9912d6fd64757bce996b438befee462f1debc2ea Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5247) 
   
   <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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -90,6 +90,21 @@
 		}
 	}
 
+	/**
+	 * Fakes asynchronous execution by immediately executing the operation and completing the supplied future
+	 * either noramlly or exceptionally.
+	 *
+	 * @param operation to executed
+	 * @param <T> type of the result
+	 */
+	public static <T> void completeFromCallable(CompletableFuture<T> future, Callable<T> operation) {

Review comment:
       I checked out `FutureUtils.runSyn` but I couldn't find any usages of it (now and in the original commit too).
   So I think we should just remove it, but also in a separate PR.
   WDYT @AHeise?




----------------------------------------------------------------
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] zhijiangW merged pull request #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   


----------------------------------------------------------------
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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   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 e3ad391b48b7ffc4a5ebd3717e0f99808522cb28 (Wed Aug 05 09:29: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] AHeise commented on a change in pull request #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
##########
@@ -90,6 +90,21 @@
 		}
 	}
 
+	/**
+	 * Fakes asynchronous execution by immediately executing the operation and completing the supplied future
+	 * either noramlly or exceptionally.
+	 *
+	 * @param operation to executed
+	 * @param <T> type of the result
+	 */
+	public static <T> void completeFromCallable(CompletableFuture<T> future, Callable<T> operation) {

Review comment:
       `runSync` should probably also use your method.




----------------------------------------------------------------
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 #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202",
       "triggerID" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9912d6fd64757bce996b438befee462f1debc2ea",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5247",
       "triggerID" : "9912d6fd64757bce996b438befee462f1debc2ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9912d6fd64757bce996b438befee462f1debc2ea Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5247) 
   
   <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 edited a comment on pull request #13067: [FLINK-18821][network] Report PartitionRequest failures to subsequent…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202",
       "triggerID" : "e3ad391b48b7ffc4a5ebd3717e0f99808522cb28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3ad391b48b7ffc4a5ebd3717e0f99808522cb28 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5202) 
   
   <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