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/24 12:21:23 UTC

[GitHub] [flink] AHeise opened a new pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

AHeise opened a new pull request #13228:
URL: https://github.com/apache/flink/pull/13228


   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   Currently, netty threads and task thread are both accessing Unaligner. Netty thread access it indirectly through `BufferReceivedListener`. Thus, we need some costly synchronization and weaken the threading model of the Unaligner.
   
   This PR mitigates it by letting priority buffers use the same handover protocol as regular buffers. Instead buffers are reordered where needed an additional wake-up mechanism is used for priority events, such that they can interrupt regular buffer processing.
   
   ## Brief change log
   
   - Adding specific buffer data type for priority events.
   - Adding priority flags to hand-over protocols (`BufferAndAvailability`, `BufferAndBacklog`, `InputWithData`, ...) directly or indirectly.
   - Reordering buffers, channels, and gates on priority buffers, such that they are pulled before any non-priority buffer.
   - Adding a data structure `PriorityDeque` to support such reordering.
   - Adding priority notifications on input/output.
   - Handling priority notifications in `CheckpointedInputGate` (moved out from Unaligner). Generalized hand-over of events from `CheckpointedInputGate` to `StreamTaskNetworkInput`, such that all events are forwarded.
   - Removing synchronizations from Unaligner.
   - Moving spill handling from `StreamTaskNetworkInput` to Unaligner.
   - Removing `BufferReceivedListener` and letting Unaligner react on in-flight data.
   
   ## Verifying this change
   
   - Added/modified a bunch of unit tests.
   - UnalignedCheckpointITCase still covering along with all other ITCases and e2e (using unaligned checkpoints by default).
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (**yes** / no / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no**  /don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented)
   


----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 67942d0ea538985ec25cecf7cefdc1bec0c1b53b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 755c3410f798bf7b3fffef91b7f2349b021613a9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301) 
   * 4af565f8d262b2ed2ebfb7614e0f24494eb22186 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 40c76a4ce74a5bbc800cf9833b96bb0156cacc41 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376) 
   * 407169bc192b5d05bfba2c264849f1a6e7b81415 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -454,42 +431,106 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 				}
 
 				wasEmpty = receivedBuffers.isEmpty();
-				receivedBuffers.add(buffer);
 
-				if (listener != null && buffer.isBuffer() && receivedCheckpointId < lastRequestedCheckpointId) {
-					notifyReceivedBuffer = buffer.retainBuffer();
+				AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+				if (priorityEvent != null) {
+					receivedBuffers.addPriorityElement(buffer);
+					final int pos = receivedBuffers.getNumPriorityElements();
+					if (priorityEvent instanceof CheckpointBarrier) {
+						final long barrierId = ((CheckpointBarrier) priorityEvent).getId();
+						// don't spill future buffers for this checkpoint
+						if (!pendingCheckpointBarriers.remove(barrierId)) {
+							// checkpoint was not yet started by task thread,
+							// so remember the numbers of buffers to spill for the time when it will be started
+							numBuffersOvertaken.put(barrierId, receivedBuffers.size() - pos);
+						}
+					}
+					firstPriorityEvent = pos == 1;
 				} else {
-					notifyReceivedBuffer = null;
+					receivedBuffers.add(buffer);
+					if (buffer.isBuffer()) {
+						for (final long checkpointId : pendingCheckpointBarriers) {
+							channelStateWriter.addInputData(
+								checkpointId,
+								channelInfo,
+								sequenceNumber,
+								CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+						}
+					}

Review comment:
       Much smaller now thanks to the helper 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



[GitHub] [flink] flinkbot edited a comment on pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a6a2d36afcce1758e885f0751359ccf035ae2378 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -621,61 +626,84 @@ public boolean isFinished() {
 		return Optional.of(transformToBufferOrEvent(
 			inputWithData.data.buffer(),
 			inputWithData.moreAvailable,
-			inputWithData.input));
+			inputWithData.input,
+			inputWithData.morePriorityEvents));
 	}
 
 	private Optional<InputWithData<InputChannel, BufferAndAvailability>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<InputChannel> inputChannel = getChannel(blocking);
-			if (!inputChannel.isPresent()) {
+			Optional<InputChannel> inputChannelOpt = getChannel(blocking);
+			if (!inputChannelOpt.isPresent()) {
 				return Optional.empty();
 			}
 
 			// Do not query inputChannel under the lock, to avoid potential deadlocks coming from
 			// notifications.
-			Optional<BufferAndAvailability> result = inputChannel.get().getNextBuffer();
+			final InputChannel inputChannel = inputChannelOpt.get();
+			Optional<BufferAndAvailability> bufferAndAvailabilityOpt = inputChannel.getNextBuffer();
 
 			synchronized (inputChannelsWithData) {
-				if (result.isPresent() && result.get().moreAvailable()) {
+				if (!bufferAndAvailabilityOpt.isPresent()) {
+					if (inputChannelsWithData.isEmpty()) {
+						availabilityHelper.resetUnavailable();
+					}
+					continue;

Review comment:
       Sry, I will split.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   * cb585f45dd49a291daefda4b4eb847262b75856e 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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java
##########
@@ -120,67 +112,10 @@ public void testNoDataProcessedAfterCheckpointBarrier() throws Exception {
 		assertEquals(0, output.getNumberOfEmittedRecords());
 	}
 
-	@Test
-	public void testSnapshotWithTwoInputGates() throws Exception {

Review comment:
       why was this test removed?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -62,9 +65,37 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		if (inputGate.getPriorityEventAvailableFuture().isDone()) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}
+		}
+

Review comment:
       add `checkState(!inputGate.getPriorityEventAvailableFuture().isDone())`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
##########
@@ -812,242 +788,13 @@ public void testQueuedBuffers() throws Exception {
 		}
 	}
 
-	@Test
-	public void testBufferReceivedListener() throws Exception {

Review comment:
       Can you double check if indeed all of those test should be removed? I don't see how things like `testPartitionNotFoundExceptionWhileGetNextBuffer` should be related to this commit/PR?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -361,4 +360,89 @@ public String toString() {
 				'}';
 		}
 	}
+
+	/**
+	 * Helper class for persisting channel state via {@link ChannelStateWriter}.
+	 */
+	@NotThreadSafe
+	protected final class ChannelStatePersister {
+		private static final long CHECKPOINT_COMPLETED = -1;
+
+		private static final long BARRIER_RECEIVED = -2;
+
+		/** All started checkpoints where a barrier has not been received yet. */
+		private long pendingCheckpointBarrierId = CHECKPOINT_COMPLETED;
+
+		/** Writer must be initialized before usage. {@link #startPersisting(long, List)} enforces this invariant. */
+		@Nullable
+		private final ChannelStateWriter channelStateWriter;
+
+		public ChannelStatePersister(@Nullable ChannelStateWriter channelStateWriter) {
+			this.channelStateWriter = channelStateWriter;
+		}
+
+		protected void startPersisting(long barrierId, List<Buffer> knownBuffers) {
+			checkState(isInitialized(), "Channel state writer not injected");
+
+			if (pendingCheckpointBarrierId != BARRIER_RECEIVED) {
+				pendingCheckpointBarrierId = barrierId;
+			}
+			if (knownBuffers.size() > 0) {
+				channelStateWriter.addInputData(
+					barrierId,
+					channelInfo,
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.fromList(knownBuffers, Buffer::recycleBuffer));
+			}
+		}
+
+		protected boolean isInitialized() {
+			return channelStateWriter != null;
+		}
+
+		protected void stopPersisting() {
+			pendingCheckpointBarrierId = CHECKPOINT_COMPLETED;
+		}
+
+		protected void maybePersist(Buffer buffer) {
+			if (pendingCheckpointBarrierId >= 0 && buffer.isBuffer()) {
+				channelStateWriter.addInputData(
+					pendingCheckpointBarrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+			}
+		}
+
+		protected boolean checkForBarrier(Buffer buffer) throws IOException {
+			final AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+			if (priorityEvent instanceof CheckpointBarrier) {
+				pendingCheckpointBarrierId = BARRIER_RECEIVED;
+				return true;
+			}
+			return false;
+		}
+
+		/**
+		 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
+		 * returns null in all other cases.
+		 */
+		@Nullable
+		protected AbstractEvent parsePriorityEvent(Buffer buffer) throws IOException {
+			if (buffer.isBuffer() || !buffer.getDataType().hasPriority()) {
+				return null;
+			}
+
+			AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+			// reset the buffer because it would be deserialized again in SingleInputGate while getting next buffer.
+			// we can further improve to avoid double deserialization in the future.
+			buffer.setReaderIndex(0);
+			return event;
+		}

Review comment:
       This doesn't seem to fit in the `ChannelStatePersister` interface. Maybe move those methods somewhere else? As static methods to `InputChannel`? 
   
   Also, aren't they duplicating the same logic as somewhere on the output sides?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -63,9 +66,38 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		final boolean hasPriorityEvents = inputGate.getPriorityEventAvailableFuture().isDone();
+		if (hasPriorityEvents) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}

Review comment:
       maybe add a `checkState`, that we are not loosing some unexpected data?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -177,32 +187,53 @@ private boolean addBuffer(BufferConsumer bufferConsumer) {
 			buffers.add(bufferConsumer);
 			return false;
 		}
-		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-			"checkpoints");
 
 		buffers.addPriorityElement(bufferConsumer);
 		final int numPriorityElements = buffers.getNumPriorityElements();
 
-		// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-		final Iterator<BufferConsumer> iterator = buffers.iterator();
-		Iterators.advance(iterator, numPriorityElements);
-		while (iterator.hasNext()) {
-			BufferConsumer buffer = iterator.next();
-
-			if (buffer.isBuffer()) {
-				try (BufferConsumer bc = buffer.copy()) {
-					inflightBufferSnapshot.add(bc.build());
+		CheckpointBarrier barrier = parseCheckpointBarrier(bufferConsumer);
+		if (barrier != null) {
+			checkState(
+				barrier.getCheckpointOptions().isUnalignedCheckpoint(),
+				"Only unaligned checkpoints should be priority events");
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, numPriorityElements);
+			List<Buffer> inflightBuffers = new ArrayList<>();
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
+						inflightBuffers.add(bc.build());
+					}
 				}
 			}
+			if (!inflightBuffers.isEmpty()) {
+				channelStateWriter.addOutputData(
+					barrier.getId(),
+					subpartitionInfo,
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					inflightBuffers.toArray(new Buffer[0]));
+			}
 		}

Review comment:
       nit: move to a separate `processCheckpointBarrier` method?
   
   or maybe also one step further:
   ```
   	private boolean addBuffer(BufferConsumer bufferConsumer) {
   		assert Thread.holdsLock(buffers);	
   		if (bufferConsumer.getDataType().hasPriority()) {
   			processPriorityBuffer(bufferConsumer);
   		}
   		buffers.add(bufferConsumer);
   		return false;
   	}
   ```
   
   to highlight that on the hot path, (without priority) we are skipping this whole code?




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 407169bc192b5d05bfba2c264849f1a6e7b81415 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402) 
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a6a2d36afcce1758e885f0751359ccf035ae2378 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 pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   > CI shows a timeout in {{DataSinkTaskTest.testDataSinkTask}}.
   
   Thanks. I converted to draft until I resolved it.
   
   I also wanted to collect some feedback anyways before considering it a serious PR (in the sense of being mergeable).


----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -63,9 +66,38 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		final boolean hasPriorityEvents = inputGate.getPriorityEventAvailableFuture().isDone();
+		if (hasPriorityEvents) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}

Review comment:
       Yes, first this method checks if there is at least one priority event (priority future completed). If there is at least one, it starts processing the first one. At this point, it relies on `BufferOrEvent::morePriorityEvents` to be correct in both directions (no false positives or negatives; although a false negative would just be a tad slower).




----------------------------------------------------------------
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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -133,14 +136,14 @@ public boolean isAvailable() {
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	@Nullable
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!
-		if (numCreditsAvailable > 0) {
-			return bufferAndBacklog.isDataAvailable();
-		}
-		else {
-			return bufferAndBacklog.isEventAvailable();
+		final Buffer.DataType nextDataType = bufferAndBacklog.getNextDataType();
+		if (numCreditsAvailable > 0 || (nextDataType != null && nextDataType.isEvent())) {
+			return nextDataType;
 		}
+		return null;

Review comment:
       hmmm, maybe add another enum type for this purpose, instead of having `null`? (I'm not sure, just brain storming)




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 11cb1939f8a98340acab9b795c6f1894808fb606 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351) 
   * 19c4f0d1d1710b829946a78cb58aff768baab684 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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -147,6 +148,14 @@ else if (configuredInput instanceof SourceInputConfig) {
 		return anyInputAvailable;
 	}
 
+	@Override
+	public CompletableFuture<?> getPriorityEventAvailableFuture() {
+		return CompletableFuture.anyOf(
+			Arrays.stream(inputProcessors)
+				.map(inputProcessor -> inputProcessor.taskInput.getPriorityEventAvailableFuture())
+				.toArray(CompletableFuture[]::new));

Review comment:
       This is an intermediate code that is being replaced later. Can you squash those changes?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -371,12 +371,16 @@ protected void processInput(MailboxDefaultAction.Controller controller) throws E
 	 */
 	@VisibleForTesting
 	CompletableFuture<?> getInputOutputJointFuture(InputStatus status) {
+		final CompletableFuture<?> priorityEventAvailableFuture = inputProcessor.getPriorityEventAvailableFuture();
 		if (status == InputStatus.NOTHING_AVAILABLE && !recordWriter.isAvailable()) {
-			return CompletableFuture.allOf(inputProcessor.getAvailableFuture(), recordWriter.getAvailableFuture());
+			return CompletableFuture.anyOf(
+				priorityEventAvailableFuture,
+				CompletableFuture.allOf(inputProcessor.getAvailableFuture(), recordWriter.getAvailableFuture()));
 		} else if (status == InputStatus.NOTHING_AVAILABLE) {
-			return inputProcessor.getAvailableFuture();
+			return CompletableFuture.anyOf(inputProcessor.getAvailableFuture(),
+				priorityEventAvailableFuture);
 		} else {
-			return recordWriter.getAvailableFuture();
+			return CompletableFuture.anyOf(priorityEventAvailableFuture, recordWriter.getAvailableFuture());

Review comment:
       As we discussed online there is a bit of duplicated code here.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
##########
@@ -232,19 +232,16 @@ public int getInputIndex() {
 			ChannelStateWriter channelStateWriter,
 			long checkpointId) throws IOException {
 		for (int channelIndex = 0; channelIndex < recordDeserializers.length; channelIndex++) {
-			final InputChannel channel = checkpointedInputGate.getChannel(channelIndex);
-
-			// Assumption for retrieving buffers = one concurrent checkpoint
 			RecordDeserializer<?> deserializer = recordDeserializers[channelIndex];
 			if (deserializer != null) {
+				final InputChannel channel = checkpointedInputGate.getChannel(channelIndex);
+
 				channelStateWriter.addInputData(
 					checkpointId,
 					channel.getChannelInfo(),
 					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
 					deserializer.getUnconsumedBuffer());
 			}
-
-			checkpointedInputGate.spillInflightBuffers(checkpointId, channelIndex, channelStateWriter);

Review comment:
       Are you sure it's going in the right direction this change? Previously spilling was explicit on demand, now it's happening magically (implicitly) inside `LocalInputChannel`.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -171,10 +183,36 @@ public InputStatus processInput() throws Exception {
 		}
 
 		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		updatePriorityAvailability();
 		checkFinished(inputStatus, readingInputIndex);
 		return inputSelectionHandler.updateStatus(inputStatus, readingInputIndex);
 	}
 
+	private void updatePriorityAvailability() {
+		if (lastPriorityInputIndex != InputSelection.NONE_AVAILABLE) {
+			final CompletableFuture<?> priorityEventAvailableFuture =
+				inputProcessors[lastPriorityInputIndex].taskInput.getPriorityEventAvailableFuture();
+			// no more priority events for the input
+			if (!priorityEventAvailableFuture.isDone()) {
+				prioritySelectionHandler.setUnavailableInput(lastPriorityInputIndex);
+				if (!prioritySelectionHandler.isAnyInputAvailable()) {
+					priorityAvailability.resetUnavailable();
+				}
+				priorityEventAvailableFuture.thenRun(onPriorityEvent(lastPriorityInputIndex));
+			}
+		}
+	}
+
+	private Runnable onPriorityEvent(int index) {
+		return () -> {
+			// set the priority flag in a mail before notifying StreamTask of availability
+			mainMailboxExecutor.execute(() -> {
+				prioritySelectionHandler.setAvailableInput(index);
+				priorityAvailability.getUnavailableToResetAvailable().complete(null);
+			}, "priority event {}", index);

Review comment:
       nit: add `StreamMultipleInputProcessor` to the mail's name?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
##########
@@ -113,7 +113,11 @@ public void processBarrier(CheckpointBarrier barrier, InputChannelInfo channelIn
 
 			if (++numBarriersReceived == numOpenChannels) {
 				allBarriersReceivedFuture.complete(null);
-				resetPendingCheckpoint(barrierId);
+				for (final InputGate gate : inputGates) {
+					for (int index = 0, numChannels = gate.getNumberOfInputChannels(); index < numChannels; index++) {
+						gate.getChannel(index).checkpointStopped(currentCheckpointId);
+					}
+				}

Review comment:
       What's the story behind this change? 
   1. it seems it differs only by a single line `numBarriersReceived = 0;`, so at the very least we should deduplicate some code here
   2. can you explain what's the functional change?
   3. aren't we missing a unit test for that? It would help answer point 2., and if there was a bug discovered in e2e test, it would be nice to have a faster unit test for that as well.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -173,50 +181,51 @@ private boolean addBuffer(BufferConsumer bufferConsumer) {
 			buffers.add(bufferConsumer);
 			return false;
 		}
-		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-			"checkpoints");
 
 		final int pos = buffers.getNumPriorityElements();
 		buffers.addPriorityElement(bufferConsumer);
 
-		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
-		if (unalignedCheckpoint) {
+		CheckpointBarrier barrier = parseCheckpointBarrier(bufferConsumer);
+		if (barrier != null) {
+			checkState(
+				barrier.getCheckpointOptions().isUnalignedCheckpoint(),
+				"Only unaligned checkpoints should be priority events");
 			final Iterator<BufferConsumer> iterator = buffers.iterator();
 			Iterators.advance(iterator, pos + 1);
+			List<Buffer> inflightBuffers = new ArrayList<>();
 			while (iterator.hasNext()) {
 				BufferConsumer buffer = iterator.next();
 
 				if (buffer.isBuffer()) {
 					try (BufferConsumer bc = buffer.copy()) {
-						inflightBufferSnapshot.add(bc.build());
+						inflightBuffers.add(bc.build());
 					}
 				}
 			}
+			channelStateWriter.addOutputData(
+				barrier.getId(),
+				subpartitionInfo,
+				ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+				inflightBuffers.toArray(new Buffer[0]));
 		}
 		return pos == 0;
 	}
 
-	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
-		boolean unalignedCheckpoint;
+	@Nullable
+	private CheckpointBarrier parseCheckpointBarrier(BufferConsumer bufferConsumer) {

Review comment:
       This is again modifying code that I have already reviewed in the previous commit :( 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -173,50 +181,51 @@ private boolean addBuffer(BufferConsumer bufferConsumer) {
 			buffers.add(bufferConsumer);
 			return false;
 		}
-		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-			"checkpoints");
 
 		final int pos = buffers.getNumPriorityElements();
 		buffers.addPriorityElement(bufferConsumer);
 
-		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
-		if (unalignedCheckpoint) {
+		CheckpointBarrier barrier = parseCheckpointBarrier(bufferConsumer);
+		if (barrier != null) {
+			checkState(
+				barrier.getCheckpointOptions().isUnalignedCheckpoint(),
+				"Only unaligned checkpoints should be priority events");
 			final Iterator<BufferConsumer> iterator = buffers.iterator();
 			Iterators.advance(iterator, pos + 1);
+			List<Buffer> inflightBuffers = new ArrayList<>();
 			while (iterator.hasNext()) {
 				BufferConsumer buffer = iterator.next();
 
 				if (buffer.isBuffer()) {
 					try (BufferConsumer bc = buffer.copy()) {
-						inflightBufferSnapshot.add(bc.build());
+						inflightBuffers.add(bc.build());
 					}
 				}
 			}
+			channelStateWriter.addOutputData(
+				barrier.getId(),
+				subpartitionInfo,
+				ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+				inflightBuffers.toArray(new Buffer[0]));

Review comment:
       Again I would point to the previous comment:
   https://github.com/apache/flink/pull/13228#discussion_r487038327
   
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   might be a better option. (It might not, as I haven't tried to implement it)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -89,17 +96,15 @@
 	/** The number of available buffers that have not been announced to the producer yet. */
 	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
 
-	/**
-	 * The latest already triggered checkpoint id which would be updated during
-	 * {@link #spillInflightBuffers(long, ChannelStateWriter)}.
-	 */
-	@GuardedBy("receivedBuffers")
-	private long lastRequestedCheckpointId = -1;
+	private final BufferManager bufferManager;
 
-	/** The current received checkpoint id from the network. */
-	private long receivedCheckpointId = -1;
+	/** Stores #overtaken buffers when a checkpoint barrier is received before task thread started checkpoint. */
+	@GuardedBy("receivedBuffers")
+	private Map<Long, Integer> numBuffersOvertaken = new HashMap<>();
 
-	private final BufferManager bufferManager;
+	/** All started checkpoints where a barrier has not been received yet. */
+	@GuardedBy("receivedBuffers")
+	private Deque<Long> pendingCheckpointBarriers = new ArrayDeque<>(2);

Review comment:
       I don't think it's worth complicating the code with support for multiple concurrent checkpoints. It's not likely to be implemented soon, if ever.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
##########
@@ -210,15 +221,25 @@ public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelSt
 		}
 
 		Buffer buffer = next.buffer();
-		CheckpointBarrier notifyReceivedBarrier = parseCheckpointBarrierOrNull(buffer);
-		if (notifyReceivedBarrier != null) {
-			receivedCheckpointId = notifyReceivedBarrier.getId();
-		} else if (receivedCheckpointId < lastRequestedCheckpointId && buffer.isBuffer()) {
-			inputGate.getBufferReceivedListener().notifyBufferReceived(buffer.retainBuffer(), channelInfo);
-		}
 
 		numBytesIn.inc(buffer.getSize());
 		numBuffersIn.inc();
+		if (buffer.isBuffer()) {
+			for (final long barrierId : pendingCheckpointBarriers) {
+				channelStateWriter.addInputData(
+					barrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));

Review comment:
       Are you spilling only on polling the buffer? (I think `RemoteInputChannel` is working better in this regard)

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -171,10 +183,36 @@ public InputStatus processInput() throws Exception {
 		}
 
 		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		updatePriorityAvailability();
 		checkFinished(inputStatus, readingInputIndex);
 		return inputSelectionHandler.updateStatus(inputStatus, readingInputIndex);
 	}
 
+	private void updatePriorityAvailability() {
+		if (lastPriorityInputIndex != InputSelection.NONE_AVAILABLE) {
+			final CompletableFuture<?> priorityEventAvailableFuture =
+				inputProcessors[lastPriorityInputIndex].taskInput.getPriorityEventAvailableFuture();
+			// no more priority events for the input
+			if (!priorityEventAvailableFuture.isDone()) {
+				prioritySelectionHandler.setUnavailableInput(lastPriorityInputIndex);
+				if (!prioritySelectionHandler.isAnyInputAvailable()) {
+					priorityAvailability.resetUnavailable();
+				}
+				priorityEventAvailableFuture.thenRun(onPriorityEvent(lastPriorityInputIndex));
+			}
+		}
+	}
+
+	private Runnable onPriorityEvent(int index) {
+		return () -> {
+			// set the priority flag in a mail before notifying StreamTask of availability
+			mainMailboxExecutor.execute(() -> {
+				prioritySelectionHandler.setAvailableInput(index);
+				priorityAvailability.getUnavailableToResetAvailable().complete(null);

Review comment:
       Can not you maybe handle the priority message directly here, in this mail? Instead of relaying on the `processDefaultAction` to pick this up? 
   
   (I'm asking/loudly thinking)

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -314,6 +315,21 @@ protected StreamTask(
 		}
 
 		this.channelIOExecutor = Executors.newSingleThreadExecutor(new ExecutorThreadFactory("channel-state-unspilling"));
+
+		injectChannelStateWriterIntoChannels();
+	}
+
+	private void injectChannelStateWriterIntoChannels() {
+		final Environment env = getEnvironment();
+		final ChannelStateWriter channelStateWriter = subtaskCheckpointCoordinator.getChannelStateWriter();
+		for (final InputGate gate : env.getAllInputGates()) {
+			gate.setChannelStateWriter(channelStateWriter);
+		}
+		for (ResultPartitionWriter writer : env.getAllWriters()) {
+			if (writer instanceof ChannelStateHolder) {
+				((ChannelStateHolder) writer).setChannelStateWriter(channelStateWriter);
+			}
+		}

Review comment:
       ❤️ 

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
##########
@@ -22,13 +22,10 @@
 import org.apache.flink.annotation.VisibleForTesting;

Review comment:
       Is the code in a working state before:
   > Remove synchronization from CheckpointBarrierUnaligner.
   
   commit? It looks like data are spilled in two places, right?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/InputGateWithMetrics.java
##########
@@ -122,8 +121,8 @@ public void close() throws Exception {
 	}
 
 	@Override
-	public void registerBufferReceivedListener(BufferReceivedListener listener) {
-		inputGate.registerBufferReceivedListener(listener);
+	public CompletableFuture<?> getPriorityEventAvailableFuture() {
+		return inputGate.getPriorityEventAvailableFuture();

Review comment:
       A minor rebasing/squashing mistake?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
##########
@@ -159,10 +158,12 @@ public InputStatus emitNext(DataOutput<T> output) throws Exception {
 			if (bufferOrEvent.isPresent()) {
 				// return to the mailbox after receiving a checkpoint barrier to avoid processing of
 				// data after the barrier before checkpoint is performed for unaligned checkpoint mode
-				if (bufferOrEvent.get().isEvent() && bufferOrEvent.get().getEvent() instanceof CheckpointBarrier) {
+				if (bufferOrEvent.get().isBuffer()) {
+					processBuffer(bufferOrEvent.get());
+				} else {
+					processEvent(bufferOrEvent.get());

Review comment:
       Could you ether pull it to another commit or revert? This
   > Use futures to listen to priority events and handle them in StreamTaskNetworkInput.
   
   commit has a couple of other irrelevant changes

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerTestBase.java
##########
@@ -369,9 +369,7 @@ public void testMissingCancellationBarriers() throws Exception {
 		inputGate = createBarrierBuffer(2, sequence, validator);

Review comment:
       Why has this test and `CheckpointBarrierTrackerTest.java` changed in this commit? Rebasing/squashing mistake, or am I missing something about this commit (I thought it's a pure clean up without functional changes).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -454,42 +431,106 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 				}
 
 				wasEmpty = receivedBuffers.isEmpty();
-				receivedBuffers.add(buffer);
 
-				if (listener != null && buffer.isBuffer() && receivedCheckpointId < lastRequestedCheckpointId) {
-					notifyReceivedBuffer = buffer.retainBuffer();
+				AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+				if (priorityEvent != null) {
+					receivedBuffers.addPriorityElement(buffer);
+					final int pos = receivedBuffers.getNumPriorityElements();
+					if (priorityEvent instanceof CheckpointBarrier) {
+						final long barrierId = ((CheckpointBarrier) priorityEvent).getId();
+						// don't spill future buffers for this checkpoint
+						if (!pendingCheckpointBarriers.remove(barrierId)) {
+							// checkpoint was not yet started by task thread,
+							// so remember the numbers of buffers to spill for the time when it will be started
+							numBuffersOvertaken.put(barrierId, receivedBuffers.size() - pos);
+						}
+					}
+					firstPriorityEvent = pos == 1;
 				} else {
-					notifyReceivedBuffer = null;
+					receivedBuffers.add(buffer);
+					if (buffer.isBuffer()) {
+						for (final long checkpointId : pendingCheckpointBarriers) {
+							channelStateWriter.addInputData(
+								checkpointId,
+								channelInfo,
+								sequenceNumber,
+								CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+						}
+					}
 				}
-				notifyReceivedBarrier = listener != null ? parseCheckpointBarrierOrNull(buffer) : null;
 			}
 			recycleBuffer = false;
 
 			++expectedSequenceNumber;
 
+			if (firstPriorityEvent) {
+				notifyPriorityEvent();
+			}
 			if (wasEmpty) {
 				notifyChannelNonEmpty();
 			}
 
 			if (backlog >= 0) {
 				onSenderBacklog(backlog);
 			}
-
-			if (notifyReceivedBarrier != null) {
-				receivedCheckpointId = notifyReceivedBarrier.getId();
-				if (notifyReceivedBarrier.isCheckpoint()) {
-					listener.notifyBarrierReceived(notifyReceivedBarrier, channelInfo);
-				}
-			} else if (notifyReceivedBuffer != null) {
-				listener.notifyBufferReceived(notifyReceivedBuffer, channelInfo);
-			}
 		} finally {
 			if (recycleBuffer) {
 				buffer.recycleBuffer();
 			}
 		}
 	}
 
+	/**
+	 * Spills all queued buffers on checkpoint start. If barrier has already been received (and reordered), spill only
+	 * the overtaken buffers.
+	 */
+	public void checkpointStarted(CheckpointBarrier barrier) {
+		checkState(channelStateWriter != null, "Channel state writer not injected");
+		synchronized (receivedBuffers) {
+			final Integer numBuffers = numBuffersOvertaken.get(barrier.getId());
+			if (numBuffers != null) {
+				// already received barrier before the task thread picked up the barrier of this or another channel
+				spillBuffers(barrier.getId(), numBuffers);
+			} else {
+				// barrier not yet received, spill all current and future buffers
+				spillBuffers(barrier.getId(), receivedBuffers.getNumUnprioritizedElements());
+				pendingCheckpointBarriers.add(barrier.getId());
+			}
+		}
+	}
+
+	public void checkpointStopped(long checkpointId) {
+		synchronized (receivedBuffers) {
+			numBuffersOvertaken.remove(checkpointId);
+			pendingCheckpointBarriers.remove(checkpointId);
+		}
+	}
+
+	private void spillBuffers(long checkpointId, int numBuffers) {

Review comment:
       nitty nit: `writeInFlightBuffers`? (`write` as Romand picked `writer` for the "spilling" nomenclature)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -454,42 +431,106 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 				}
 
 				wasEmpty = receivedBuffers.isEmpty();
-				receivedBuffers.add(buffer);
 
-				if (listener != null && buffer.isBuffer() && receivedCheckpointId < lastRequestedCheckpointId) {
-					notifyReceivedBuffer = buffer.retainBuffer();
+				AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+				if (priorityEvent != null) {
+					receivedBuffers.addPriorityElement(buffer);
+					final int pos = receivedBuffers.getNumPriorityElements();
+					if (priorityEvent instanceof CheckpointBarrier) {
+						final long barrierId = ((CheckpointBarrier) priorityEvent).getId();
+						// don't spill future buffers for this checkpoint
+						if (!pendingCheckpointBarriers.remove(barrierId)) {
+							// checkpoint was not yet started by task thread,
+							// so remember the numbers of buffers to spill for the time when it will be started
+							numBuffersOvertaken.put(barrierId, receivedBuffers.size() - pos);
+						}
+					}
+					firstPriorityEvent = pos == 1;
 				} else {
-					notifyReceivedBuffer = null;
+					receivedBuffers.add(buffer);
+					if (buffer.isBuffer()) {
+						for (final long checkpointId : pendingCheckpointBarriers) {
+							channelStateWriter.addInputData(
+								checkpointId,
+								channelInfo,
+								sequenceNumber,
+								CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+						}
+					}

Review comment:
       This method has grown too large.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 40c76a4ce74a5bbc800cf9833b96bb0156cacc41 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376) 
   * 407169bc192b5d05bfba2c264849f1a6e7b81415 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * b3a1520089c241fc74837902b6440d84a9636c14 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 4af565f8d262b2ed2ebfb7614e0f24494eb22186 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311) 
   * b3a1520089c241fc74837902b6440d84a9636c14 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 pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   >     * https://github.com/apache/flink/pull/13228/files#r493039856 is still not addressed I think?
   :white_check_mark:
   >     * azure green
   :white_check_mark:
   >     * benchmark request will not show a visible regression :)
   :white_check_mark:
   


----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724) 
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740) 
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 1a86b34a3cd22d0ae34d16e9fa96036650dd6323 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * fa3e495b6ba201769d908adea420e4944ddd7643 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459) 
   * a1fb1c3114305ed5fce898dfcc03abed4d6963d4 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
##########
@@ -113,7 +113,11 @@ public void processBarrier(CheckpointBarrier barrier, InputChannelInfo channelIn
 
 			if (++numBarriersReceived == numOpenChannels) {
 				allBarriersReceivedFuture.complete(null);
-				resetPendingCheckpoint(barrierId);
+				for (final InputGate gate : inputGates) {
+					for (int index = 0, numChannels = gate.getNumberOfInputChannels(); index < numChannels; index++) {
+						gate.getChannel(index).checkpointStopped(currentCheckpointId);
+					}
+				}

Review comment:
       Sorry that was just a test commit to see if the stuck e2e failed because of this change. I removed it. The original change is covered by a few unit tests already.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 407169bc192b5d05bfba2c264849f1a6e7b81415 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402) 
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 19c4f0d1d1710b829946a78cb58aff768baab684 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365) 
   * 40c76a4ce74a5bbc800cf9833b96bb0156cacc41 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 273cc6bf445b4a44daca0e98d49aa0745a36a534 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225) 
   * f1bfd2b98e8665739bd07ae828d2ca73e2e927fc 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   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 434413607c62242a6ab818ae9c5891ad45122757 (Mon Aug 24 12:23:22 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fce9f56a12b7c28a827c85669ce4bb0a8d31a48b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290) 
   * d141f10ef060063162dc073b1cd66729f5f75a3b 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fce9f56a12b7c28a827c85669ce4bb0a8d31a48b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290) 
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * ef2c7ecea91a30e4d721efa759ca1b15728a24cb 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * 834c28da93d4b24e562e017979b483e288db02a2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570) 
   * a22657562a42dda270f89c176e5c260ee73c6698 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -171,19 +175,42 @@ private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAs
 			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
 				"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
-						inflightBufferSnapshot.add(bc.build());
+			final int pos = buffers.getNumPriorityElements();
+			buffers.addPriorityElement(bufferConsumer);
+
+			boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+			if (unalignedCheckpoint) {
+				final Iterator<BufferConsumer> iterator = buffers.iterator();
+				Iterators.advance(iterator, pos + 1);
+				while (iterator.hasNext()) {
+					BufferConsumer buffer = iterator.next();
+
+					if (buffer.isBuffer()) {
+						try (BufferConsumer bc = buffer.copy()) {
+							inflightBufferSnapshot.add(bc.build());
+						}
 					}
 				}
 			}
+			return;
+		}

Review comment:
       In general, I wanted to drop the assumption that there is only one priority event going on at any given time. That's especially true when we make cancellation events also a priority and we have a more or less fully blocked channel.
   
   Specifically, this change had following motivations:
   * drop the assumption that all priority events are unaligned checkpoints.
   * drop the assumption that the new priority event is always at position 0.
   * a small performance improvement where buffers are only copied after it's clear that they are not containing an event.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724) 
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740) 
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 1a86b34a3cd22d0ae34d16e9fa96036650dd6323 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778) 
   * 74094b3c0206de0bad49bdb7526534a8c43a5934 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   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 36c366400126cb68f0a8c8cf14b5c5e98ef2b58d (Fri Feb 19 07:24:38 UTC 2021)
   
   **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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724) 
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740) 
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * ef2c7ecea91a30e4d721efa759ca1b15728a24cb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297) 
   * 755c3410f798bf7b3fffef91b7f2349b021613a9 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 273cc6bf445b4a44daca0e98d49aa0745a36a534 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -171,19 +175,42 @@ private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAs
 			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
 				"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
-						inflightBufferSnapshot.add(bc.build());
+			final int pos = buffers.getNumPriorityElements();
+			buffers.addPriorityElement(bufferConsumer);
+
+			boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+			if (unalignedCheckpoint) {
+				final Iterator<BufferConsumer> iterator = buffers.iterator();
+				Iterators.advance(iterator, pos + 1);
+				while (iterator.hasNext()) {
+					BufferConsumer buffer = iterator.next();
+
+					if (buffer.isBuffer()) {
+						try (BufferConsumer bc = buffer.copy()) {
+							inflightBufferSnapshot.add(bc.build());
+						}
 					}
 				}
 			}
+			return;
+		}
+		buffers.add(bufferConsumer);

Review comment:
       Yes, good idea. In general that change looks a bit odd, because it's isolated from the upcoming changes (I had to split somewhere and probably didn't hit the sweet spot everywhere).




----------------------------------------------------------------
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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -133,14 +136,14 @@ public boolean isAvailable() {
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	@Nullable

Review comment:
       nit: add a javadoc explaining the returned value?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -171,19 +175,42 @@ private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAs
 			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
 				"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
-						inflightBufferSnapshot.add(bc.build());
+			final int pos = buffers.getNumPriorityElements();
+			buffers.addPriorityElement(bufferConsumer);
+
+			boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+			if (unalignedCheckpoint) {
+				final Iterator<BufferConsumer> iterator = buffers.iterator();
+				Iterators.advance(iterator, pos + 1);
+				while (iterator.hasNext()) {
+					BufferConsumer buffer = iterator.next();
+
+					if (buffer.isBuffer()) {
+						try (BufferConsumer bc = buffer.copy()) {
+							inflightBufferSnapshot.add(bc.build());
+						}
 					}
 				}
 			}
+			return;
+		}

Review comment:
       Why do we need this change? In what scenarios are you expecting more than one priority event in the output buffer?  (if there is a reason that I'm forgetting about, please add it to the commit message)
   
   edit: (after reading commit message a couple of times) Or you are just re-using here a class, that you are mostly intending to use later in the future (on the inputs?)? If so maybe it needs some more explanation in the commit message?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -195,28 +193,27 @@ void retriggerSubpartitionRequest(int subpartitionIndex) throws IOException {
 	}
 
 	@Override
-	public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelStateWriter) throws IOException {
+	public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelStateWriter) {
 		synchronized (receivedBuffers) {
-			checkState(checkpointId > lastRequestedCheckpointId, "Need to request the next checkpointId");
-
-			final List<Buffer> inflightBuffers = new ArrayList<>(receivedBuffers.size());
-			for (Buffer buffer : receivedBuffers) {
-				CheckpointBarrier checkpointBarrier = parseCheckpointBarrierOrNull(buffer);
-				if (checkpointBarrier != null && checkpointBarrier.getId() >= checkpointId) {
-					break;
+			final Integer numRecords = numRecordsOvertaken.remove(checkpointId);

Review comment:
       shouldn't we remove also obsolete values from this map? (to prevent a potential memory leak?)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -171,19 +175,42 @@ private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAs
 			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
 				"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
-						inflightBufferSnapshot.add(bc.build());
+			final int pos = buffers.getNumPriorityElements();
+			buffers.addPriorityElement(bufferConsumer);
+
+			boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+			if (unalignedCheckpoint) {
+				final Iterator<BufferConsumer> iterator = buffers.iterator();
+				Iterators.advance(iterator, pos + 1);
+				while (iterator.hasNext()) {
+					BufferConsumer buffer = iterator.next();
+
+					if (buffer.isBuffer()) {
+						try (BufferConsumer bc = buffer.copy()) {
+							inflightBufferSnapshot.add(bc.build());
+						}
 					}
 				}
 			}
+			return;
+		}
+		buffers.add(bufferConsumer);

Review comment:
       nit: it was a bit confusing for me for a moment which code paths are doing what. IMO it would be easier to follow it, if the shorter branch would be first, and if the more complicated part would have one lever of nesting less:
   ```
   if (!insertAsHead) {
     buffers.add(bufferConsumer);
     return;
   }
   //rest of the code
   ```
   in that case it's more obvious that `!insertAsHead` is a trivial case and that it doesn't interact with the other branch at all.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -195,28 +193,27 @@ void retriggerSubpartitionRequest(int subpartitionIndex) throws IOException {
 	}
 
 	@Override
-	public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelStateWriter) throws IOException {
+	public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelStateWriter) {
 		synchronized (receivedBuffers) {
-			checkState(checkpointId > lastRequestedCheckpointId, "Need to request the next checkpointId");
-
-			final List<Buffer> inflightBuffers = new ArrayList<>(receivedBuffers.size());
-			for (Buffer buffer : receivedBuffers) {
-				CheckpointBarrier checkpointBarrier = parseCheckpointBarrierOrNull(buffer);
-				if (checkpointBarrier != null && checkpointBarrier.getId() >= checkpointId) {
-					break;
+			final Integer numRecords = numRecordsOvertaken.remove(checkpointId);

Review comment:
       Do I understand it correctly? Currently there is a fragile contract, that `numRecordsOvertaken` value wouldn't change between `onBuffer(...)` where we are setting it and this `spillInflightBuffers(...)` call? In other words, it assumes that between enqueueing of the priority event and the `spillInflightBuffers(...)`, task thread is not allowed to process any buffers?
   
   Maybe it would be better to embed the `numRecordsOvertaken` value in the priority event that would be processed by the task thread?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -74,34 +106,34 @@ public CheckpointedInputGate(
 	}
 
 	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {

Review comment:
       Isn't it changing the semantic slightly? Am I right, that the only case on the master branch which actually causes another iteration of this loop is 
   ```
   barrierHandler.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent());
   ```
   and all of the other cases were exiting the loop? Are now all of the cases exiting always?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -195,28 +193,27 @@ void retriggerSubpartitionRequest(int subpartitionIndex) throws IOException {
 	}
 
 	@Override
-	public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelStateWriter) throws IOException {
+	public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelStateWriter) {
 		synchronized (receivedBuffers) {
-			checkState(checkpointId > lastRequestedCheckpointId, "Need to request the next checkpointId");
-
-			final List<Buffer> inflightBuffers = new ArrayList<>(receivedBuffers.size());
-			for (Buffer buffer : receivedBuffers) {
-				CheckpointBarrier checkpointBarrier = parseCheckpointBarrierOrNull(buffer);
-				if (checkpointBarrier != null && checkpointBarrier.getId() >= checkpointId) {
-					break;
+			final Integer numRecords = numRecordsOvertaken.remove(checkpointId);

Review comment:
       `numRecordsOvertaken` -> `numBuffersOvertaken`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -770,34 +808,50 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {

Review comment:
       Heh, there are quite a bit more of corner/edge cases now.
   
   I wonder if there is maybe some other way to express the priority events, that would simplify the input gates code?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
##########
@@ -159,10 +158,12 @@ public InputStatus emitNext(DataOutput<T> output) throws Exception {
 			if (bufferOrEvent.isPresent()) {
 				// return to the mailbox after receiving a checkpoint barrier to avoid processing of
 				// data after the barrier before checkpoint is performed for unaligned checkpoint mode
-				if (bufferOrEvent.get().isEvent() && bufferOrEvent.get().getEvent() instanceof CheckpointBarrier) {
+				if (bufferOrEvent.get().isBuffer()) {
+					processBuffer(bufferOrEvent.get());
+				} else {
+					processEvent(bufferOrEvent.get());

Review comment:
       Is this spit into `processBuffer` and `processEvent` relevant?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -133,14 +136,14 @@ public boolean isAvailable() {
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	@Nullable
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!
-		if (numCreditsAvailable > 0) {
-			return bufferAndBacklog.isDataAvailable();
-		}
-		else {
-			return bufferAndBacklog.isEventAvailable();
+		final Buffer.DataType nextDataType = bufferAndBacklog.getNextDataType();
+		if (numCreditsAvailable > 0 || (nextDataType != null && nextDataType.isEvent())) {
+			return nextDataType;
 		}
+		return null;

Review comment:
       hmmm, maybe add another enum type for this purpose? (I'm not sure, just brain storming)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -621,61 +626,84 @@ public boolean isFinished() {
 		return Optional.of(transformToBufferOrEvent(
 			inputWithData.data.buffer(),
 			inputWithData.moreAvailable,
-			inputWithData.input));
+			inputWithData.input,
+			inputWithData.morePriorityEvents));
 	}
 
 	private Optional<InputWithData<InputChannel, BufferAndAvailability>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<InputChannel> inputChannel = getChannel(blocking);
-			if (!inputChannel.isPresent()) {
+			Optional<InputChannel> inputChannelOpt = getChannel(blocking);
+			if (!inputChannelOpt.isPresent()) {
 				return Optional.empty();
 			}
 
 			// Do not query inputChannel under the lock, to avoid potential deadlocks coming from
 			// notifications.
-			Optional<BufferAndAvailability> result = inputChannel.get().getNextBuffer();
+			final InputChannel inputChannel = inputChannelOpt.get();
+			Optional<BufferAndAvailability> bufferAndAvailabilityOpt = inputChannel.getNextBuffer();
 
 			synchronized (inputChannelsWithData) {
-				if (result.isPresent() && result.get().moreAvailable()) {
+				if (!bufferAndAvailabilityOpt.isPresent()) {
+					if (inputChannelsWithData.isEmpty()) {
+						availabilityHelper.resetUnavailable();
+					}
+					continue;

Review comment:
       maybe if the `result` variable rename and adding `continue` branch had happened in an independent "refactor" commit, It would have saved me a couple of minutes while reading this code while I was trying to understand the change :( 
   
   maybe, as I can see how the changes are a bit interconnected.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -63,9 +66,38 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		final boolean hasPriorityEvents = inputGate.getPriorityEventAvailableFuture().isDone();
+		if (hasPriorityEvents) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}
+		}
+
+		// re-enqueue mail to process priority events
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	private void waitForPriorityEvents(InputGate inputGate, MailboxExecutor mailboxExecutor) {
+		final CompletableFuture<?> priorityEventAvailableFuture = inputGate.getPriorityEventAvailableFuture();
+		priorityEventAvailableFuture.thenRun(() -> {
+			mailboxExecutor.execute(this::processPriorityEvents, "process priority even @ gate %s", inputGate);
+		});

Review comment:
       1. Again, do I understand this correctly? Is this assuming that nobody polls anything between completing `getPriorityEventAvailableFuture` and executing `this::processPriorityEvents`? Isn't that a bit fragile?
   
   2. What was the motivation for no passing the priority events to the `StreamTaskNetworkInput`?
   
   3. What about processing priority events as part of `pollNext()`?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -74,34 +106,34 @@ public CheckpointedInputGate(
 	}
 
 	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {
-			Optional<BufferOrEvent> next = inputGate.pollNext();
+	public Optional<BufferOrEvent> pollNext() throws IOException, InterruptedException {
+		Optional<BufferOrEvent> next = inputGate.pollNext();
 
-			if (!next.isPresent()) {
-				return handleEmptyBuffer();
-			}
+		if (!next.isPresent()) {
+			return handleEmptyBuffer();
+		}
 
-			BufferOrEvent bufferOrEvent = next.get();
-			checkState(!barrierHandler.isBlocked(bufferOrEvent.getChannelInfo()));
+		BufferOrEvent bufferOrEvent = next.get();
+		checkState(!barrierHandler.isBlocked(bufferOrEvent.getChannelInfo()));
 
-			if (bufferOrEvent.isBuffer()) {
-				return next;
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) {
-				CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent();
-				barrierHandler.processBarrier(checkpointBarrier, bufferOrEvent.getChannelInfo());
-				return next;
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
-				barrierHandler.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent());
-			}
-			else {
-				if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) {
-					barrierHandler.processEndOfPartition();
-				}
-				return next;
-			}
+		if (bufferOrEvent.isEvent()) {
+			handleEvent(bufferOrEvent);
+		} else {
+			barrierHandler.processBuffer(bufferOrEvent.getBuffer(), bufferOrEvent.getChannelInfo());

Review comment:
       for now in this commit, this is just a NO-OP call?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -63,9 +66,38 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		final boolean hasPriorityEvents = inputGate.getPriorityEventAvailableFuture().isDone();
+		if (hasPriorityEvents) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}

Review comment:
       As I understand it, it assumes that this `pollNext()` can not return anything else besides a priority event?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
##########
@@ -92,306 +90,173 @@
 		super(toNotifyOnCheckpoint);
 
 		this.taskName = taskName;
-		hasInflightBuffers = Arrays.stream(inputGates)
+		this.inputGates = inputGates;
+		storeNewBuffers = Arrays.stream(inputGates)
 			.flatMap(gate -> gate.getChannelInfos().stream())
 			.collect(Collectors.toMap(Function.identity(), info -> false));
-		threadSafeUnaligner = new ThreadSafeUnaligner(checkNotNull(checkpointCoordinator), this, inputGates);
+		numOpenChannels = storeNewBuffers.size();
+		this.checkpointCoordinator = checkpointCoordinator;
 	}
 
-	/**
-	 * We still need to trigger checkpoint via {@link ThreadSafeUnaligner#notifyBarrierReceived(CheckpointBarrier, InputChannelInfo)}
-	 * while reading the first barrier from one channel, because this might happen
-	 * earlier than the previous async trigger via mailbox by netty thread.
-	 *
-	 * <p>Note this is also suitable for the trigger case of local input channel.
-	 */
 	@Override
-	public void processBarrier(CheckpointBarrier receivedBarrier, InputChannelInfo channelInfo) throws IOException {
-		long barrierId = receivedBarrier.getId();
-		if (currentConsumedCheckpointId > barrierId || (currentConsumedCheckpointId == barrierId && !isCheckpointPending())) {
+	public void processBarrier(CheckpointBarrier barrier, InputChannelInfo channelInfo) throws IOException {
+		long barrierId = barrier.getId();
+		if (currentCheckpointId > barrierId || (currentCheckpointId == barrierId && !isCheckpointPending())) {
 			// ignore old and cancelled barriers
 			return;
 		}
-		if (currentConsumedCheckpointId < barrierId) {
-			currentConsumedCheckpointId = barrierId;
-			numBarrierConsumed = 0;
-			hasInflightBuffers.entrySet().forEach(hasInflightBuffer -> hasInflightBuffer.setValue(true));
+		if (currentCheckpointId < barrierId) {
+			handleNewCheckpoint(barrier);
+			notifyCheckpoint(barrier, 0);
 		}
-		if (currentConsumedCheckpointId == barrierId) {
-			hasInflightBuffers.put(channelInfo, false);
-			numBarrierConsumed++;
+		if (currentCheckpointId == barrierId) {
+			if (storeNewBuffers.put(channelInfo, false)) {
+				LOG.debug("{}: Received barrier from channel {} @ {}.", taskName, channelInfo, barrierId);
+
+				inputGates[channelInfo.getGateIdx()].getChannel(channelInfo.getInputChannelIdx())
+					.spillInflightBuffers(barrierId, checkpointCoordinator.getChannelStateWriter());
+
+				if (++numBarriersReceived == numOpenChannels) {
+					allBarriersReceivedFuture.complete(null);
+				}
+			}
 		}
-		threadSafeUnaligner.notifyBarrierReceived(receivedBarrier, channelInfo);
 	}
 
 	@Override
 	public void abortPendingCheckpoint(long checkpointId, CheckpointException exception) throws IOException {
-		threadSafeUnaligner.tryAbortPendingCheckpoint(checkpointId, exception);
+		tryAbortPendingCheckpoint(checkpointId, exception);
 
-		if (checkpointId > currentConsumedCheckpointId) {
-			resetPendingCheckpoint(checkpointId);
+		if (checkpointId > currentCheckpointId) {
+			resetPendingCheckpoint();
 		}
 	}
 
 	@Override
 	public void processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws IOException {
 		final long cancelledId = cancelBarrier.getCheckpointId();
-		boolean shouldAbort = threadSafeUnaligner.setCancelledCheckpointId(cancelledId);
+		boolean shouldAbort = setCancelledCheckpointId(cancelledId);
 		if (shouldAbort) {
 			notifyAbort(
 				cancelledId,
 				new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER));
 		}
 
-		if (cancelledId >= currentConsumedCheckpointId) {
-			resetPendingCheckpoint(cancelledId);
-			currentConsumedCheckpointId = cancelledId;
+		if (cancelledId >= currentCheckpointId) {
+			resetPendingCheckpoint();
+			currentCheckpointId = cancelledId;
 		}
 	}
 
 	@Override
 	public void processEndOfPartition() throws IOException {
-		threadSafeUnaligner.onChannelClosed();
-		resetPendingCheckpoint(-1L);
+		numOpenChannels--;
+
+		resetPendingCheckpoint();
+		notifyAbort(
+			currentCheckpointId,
+			new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_INPUT_END_OF_STREAM));
 	}
 
-	private void resetPendingCheckpoint(long checkpointId) {
-		if (isCheckpointPending()) {
-			LOG.warn("{}: Received barrier or EndOfPartition(-1) {} before completing current checkpoint {}. " +
-					"Skipping current checkpoint.",
-				taskName,
-				checkpointId,
-				currentConsumedCheckpointId);
+	private void resetPendingCheckpoint() {
+		LOG.warn("{}: Received barrier or EndOfPartition(-1) before completing current checkpoint {}. " +
+				"Skipping current checkpoint.",
+			taskName,
+			currentCheckpointId);
 
-			hasInflightBuffers.entrySet().forEach(hasInflightBuffer -> hasInflightBuffer.setValue(false));
-			numBarrierConsumed = 0;
-		}
+		storeNewBuffers.entrySet().forEach(storeNewBuffer -> storeNewBuffer.setValue(false));
+		numBarriersReceived = 0;
 	}
 
 	@Override
 	public long getLatestCheckpointId() {
-		return currentConsumedCheckpointId;
+		return currentCheckpointId;
 	}
 
 	@Override
 	public String toString() {
-		return String.format("%s: last checkpoint: %d", taskName, currentConsumedCheckpointId);
+		return String.format("%s: last checkpoint: %d", taskName, currentCheckpointId);
 	}
 
 	@Override
 	public void close() throws IOException {
 		super.close();
-		threadSafeUnaligner.close();
-	}
-
-	@Override
-	public boolean hasInflightData(long checkpointId, InputChannelInfo channelInfo) {
-		if (checkpointId < currentConsumedCheckpointId) {
-			return false;
-		}
-		if (checkpointId > currentConsumedCheckpointId) {
-			return true;
-		}
-		return hasInflightBuffers.get(channelInfo);
-	}
-
-	@Override
-	public CompletableFuture<Void> getAllBarriersReceivedFuture(long checkpointId) {
-		return threadSafeUnaligner.getAllBarriersReceivedFuture(checkpointId);
-	}
-
-	@Override
-	public Optional<BufferReceivedListener> getBufferReceivedListener() {
-		return Optional.of(threadSafeUnaligner);
+		allBarriersReceivedFuture.cancel(false);
 	}
 
 	@Override
 	protected boolean isCheckpointPending() {
-		return numBarrierConsumed > 0;
-	}
-
-	@VisibleForTesting
-	int getNumOpenChannels() {
-		return threadSafeUnaligner.getNumOpenChannels();
-	}
-
-	@VisibleForTesting
-	ThreadSafeUnaligner getThreadSafeUnaligner() {
-		return threadSafeUnaligner;
+		return numBarriersReceived > 0;
 	}
 
-	private void notifyCheckpoint(CheckpointBarrier barrier) throws IOException {
-		// ignore the previous triggered checkpoint by netty thread if it was already canceled or aborted before.
-		if (barrier.getId() >= threadSafeUnaligner.getCurrentCheckpointId()) {
-			super.notifyCheckpoint(barrier, 0);
+	@Override
+	public void processBuffer(Buffer buffer, InputChannelInfo channelInfo) {
+		if (storeNewBuffers.get(channelInfo)) {
+			checkpointCoordinator.getChannelStateWriter().addInputData(
+				currentCheckpointId,
+				channelInfo,
+				ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+				ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));

Review comment:
       Here, we are persisting in-flight buffers, only as they are being processed? Doesn't it mean, that unaligned checkpoint will be completed only after we process all of the buffers, making unaligned checkpoint just as quick as aligned?




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 67942d0ea538985ec25cecf7cefdc1bec0c1b53b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833) 
   * 273cc6bf445b4a44daca0e98d49aa0745a36a534 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 407169bc192b5d05bfba2c264849f1a6e7b81415 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f0bb8a255816919a6578a6994499757075ab371b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f1bfd2b98e8665739bd07ae828d2ca73e2e927fc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229) 
   * fce9f56a12b7c28a827c85669ce4bb0a8d31a48b 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * 834c28da93d4b24e562e017979b483e288db02a2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570) 
   * a22657562a42dda270f89c176e5c260ee73c6698 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 40c76a4ce74a5bbc800cf9833b96bb0156cacc41 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -195,28 +193,27 @@ void retriggerSubpartitionRequest(int subpartitionIndex) throws IOException {
 	}
 
 	@Override
-	public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelStateWriter) throws IOException {
+	public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelStateWriter) {
 		synchronized (receivedBuffers) {
-			checkState(checkpointId > lastRequestedCheckpointId, "Need to request the next checkpointId");
-
-			final List<Buffer> inflightBuffers = new ArrayList<>(receivedBuffers.size());
-			for (Buffer buffer : receivedBuffers) {
-				CheckpointBarrier checkpointBarrier = parseCheckpointBarrierOrNull(buffer);
-				if (checkpointBarrier != null && checkpointBarrier.getId() >= checkpointId) {
-					break;
+			final Integer numRecords = numRecordsOvertaken.remove(checkpointId);

Review comment:
       Good catch, a leak could happen when the checkpoint is cancelled through another channel. The map itself is rather small, but it could add up over all channels and gates.
   I don't have a good idea on how to properly abstract this cleanup except by adding some kind of checkpoint-cancelled hook though. 
   Alternatively, checkpoint barrier handler becomes more aware of the buffers to be spilled. So instead of calling `channel.spillInflightBuffers`, it could be `channel.getSpilledBuffers().forEach(channelStateWriter::write)` on a good checkpoint and `channel.getSpilledBuffers().forEach(Buffer::recycle)` on cancelled checkpoints, where `getSpilledBuffers` always cleans up this map.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740) 
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 1a86b34a3cd22d0ae34d16e9fa96036650dd6323 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765) 
   * 3c429de7c86d271a8980a34759026e18d3cf24b8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775) 
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f0bb8a255816919a6578a6994499757075ab371b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819) 
   * 67942d0ea538985ec25cecf7cefdc1bec0c1b53b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
##########
@@ -159,10 +158,12 @@ public InputStatus emitNext(DataOutput<T> output) throws Exception {
 			if (bufferOrEvent.isPresent()) {
 				// return to the mailbox after receiving a checkpoint barrier to avoid processing of
 				// data after the barrier before checkpoint is performed for unaligned checkpoint mode
-				if (bufferOrEvent.get().isEvent() && bufferOrEvent.get().getEvent() instanceof CheckpointBarrier) {
+				if (bufferOrEvent.get().isBuffer()) {
+					processBuffer(bufferOrEvent.get());
+				} else {
+					processEvent(bufferOrEvent.get());

Review comment:
       Nope, it's a refactoring that I should move out. We could also revert back. I had an intermediate version where `processEvent` signaled if the loop should be broken or not and there the split made more sense.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -147,6 +148,14 @@ else if (configuredInput instanceof SourceInputConfig) {
 		return anyInputAvailable;
 	}
 
+	@Override
+	public CompletableFuture<?> getPriorityEventAvailableFuture() {
+		return CompletableFuture.anyOf(
+			Arrays.stream(inputProcessors)
+				.map(inputProcessor -> inputProcessor.taskInput.getPriorityEventAvailableFuture())
+				.toArray(CompletableFuture[]::new));

Review comment:
       Commit is removed.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * ef2c7ecea91a30e4d721efa759ca1b15728a24cb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297) 
   * 755c3410f798bf7b3fffef91b7f2349b021613a9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a6a2d36afcce1758e885f0751359ccf035ae2378 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446) 
   * fa3e495b6ba201769d908adea420e4944ddd7643 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 4af565f8d262b2ed2ebfb7614e0f24494eb22186 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311) 
   * b3a1520089c241fc74837902b6440d84a9636c14 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
##########
@@ -812,242 +788,13 @@ public void testQueuedBuffers() throws Exception {
 		}
 	}
 
-	@Test
-	public void testBufferReceivedListener() throws Exception {

Review comment:
       Good catch, my intent was to delete `testBufferReceivedListener` and `testPartitionNotFoundExceptionWhileGetNextBuffer` but not the test in between them.
   
   `testBufferReceivedListener` tests `BufferReceivedListener` which this commits renders useless (and is later removed).
   
   `testPartitionNotFoundExceptionWhileGetNextBuffer` tests concurrent spilling of lingering buffers and receiving of such lingering buffers. Both now happens in the same thread, so the test does not make any sense.




----------------------------------------------------------------
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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java
##########
@@ -120,67 +112,10 @@ public void testNoDataProcessedAfterCheckpointBarrier() throws Exception {
 		assertEquals(0, output.getNumberOfEmittedRecords());
 	}
 
-	@Test
-	public void testSnapshotWithTwoInputGates() throws Exception {

Review comment:
       why was this test removed?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -62,9 +65,37 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		if (inputGate.getPriorityEventAvailableFuture().isDone()) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}
+		}
+

Review comment:
       add `checkState(!inputGate.getPriorityEventAvailableFuture().isDone())`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
##########
@@ -812,242 +788,13 @@ public void testQueuedBuffers() throws Exception {
 		}
 	}
 
-	@Test
-	public void testBufferReceivedListener() throws Exception {

Review comment:
       Can you double check if indeed all of those test should be removed? I don't see how things like `testPartitionNotFoundExceptionWhileGetNextBuffer` should be related to this commit/PR?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -361,4 +360,89 @@ public String toString() {
 				'}';
 		}
 	}
+
+	/**
+	 * Helper class for persisting channel state via {@link ChannelStateWriter}.
+	 */
+	@NotThreadSafe
+	protected final class ChannelStatePersister {
+		private static final long CHECKPOINT_COMPLETED = -1;
+
+		private static final long BARRIER_RECEIVED = -2;
+
+		/** All started checkpoints where a barrier has not been received yet. */
+		private long pendingCheckpointBarrierId = CHECKPOINT_COMPLETED;
+
+		/** Writer must be initialized before usage. {@link #startPersisting(long, List)} enforces this invariant. */
+		@Nullable
+		private final ChannelStateWriter channelStateWriter;
+
+		public ChannelStatePersister(@Nullable ChannelStateWriter channelStateWriter) {
+			this.channelStateWriter = channelStateWriter;
+		}
+
+		protected void startPersisting(long barrierId, List<Buffer> knownBuffers) {
+			checkState(isInitialized(), "Channel state writer not injected");
+
+			if (pendingCheckpointBarrierId != BARRIER_RECEIVED) {
+				pendingCheckpointBarrierId = barrierId;
+			}
+			if (knownBuffers.size() > 0) {
+				channelStateWriter.addInputData(
+					barrierId,
+					channelInfo,
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.fromList(knownBuffers, Buffer::recycleBuffer));
+			}
+		}
+
+		protected boolean isInitialized() {
+			return channelStateWriter != null;
+		}
+
+		protected void stopPersisting() {
+			pendingCheckpointBarrierId = CHECKPOINT_COMPLETED;
+		}
+
+		protected void maybePersist(Buffer buffer) {
+			if (pendingCheckpointBarrierId >= 0 && buffer.isBuffer()) {
+				channelStateWriter.addInputData(
+					pendingCheckpointBarrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+			}
+		}
+
+		protected boolean checkForBarrier(Buffer buffer) throws IOException {
+			final AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+			if (priorityEvent instanceof CheckpointBarrier) {
+				pendingCheckpointBarrierId = BARRIER_RECEIVED;
+				return true;
+			}
+			return false;
+		}
+
+		/**
+		 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
+		 * returns null in all other cases.
+		 */
+		@Nullable
+		protected AbstractEvent parsePriorityEvent(Buffer buffer) throws IOException {
+			if (buffer.isBuffer() || !buffer.getDataType().hasPriority()) {
+				return null;
+			}
+
+			AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+			// reset the buffer because it would be deserialized again in SingleInputGate while getting next buffer.
+			// we can further improve to avoid double deserialization in the future.
+			buffer.setReaderIndex(0);
+			return event;
+		}

Review comment:
       This doesn't seem to fit in the `ChannelStatePersister` interface. Maybe move those methods somewhere else? As static methods to `InputChannel`? 
   
   Also, aren't they duplicating the same logic as somewhere on the output sides?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -63,9 +66,38 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		final boolean hasPriorityEvents = inputGate.getPriorityEventAvailableFuture().isDone();
+		if (hasPriorityEvents) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}

Review comment:
       maybe add a `checkState`, that we are not loosing some unexpected data?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -177,32 +187,53 @@ private boolean addBuffer(BufferConsumer bufferConsumer) {
 			buffers.add(bufferConsumer);
 			return false;
 		}
-		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-			"checkpoints");
 
 		buffers.addPriorityElement(bufferConsumer);
 		final int numPriorityElements = buffers.getNumPriorityElements();
 
-		// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-		final Iterator<BufferConsumer> iterator = buffers.iterator();
-		Iterators.advance(iterator, numPriorityElements);
-		while (iterator.hasNext()) {
-			BufferConsumer buffer = iterator.next();
-
-			if (buffer.isBuffer()) {
-				try (BufferConsumer bc = buffer.copy()) {
-					inflightBufferSnapshot.add(bc.build());
+		CheckpointBarrier barrier = parseCheckpointBarrier(bufferConsumer);
+		if (barrier != null) {
+			checkState(
+				barrier.getCheckpointOptions().isUnalignedCheckpoint(),
+				"Only unaligned checkpoints should be priority events");
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, numPriorityElements);
+			List<Buffer> inflightBuffers = new ArrayList<>();
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
+						inflightBuffers.add(bc.build());
+					}
 				}
 			}
+			if (!inflightBuffers.isEmpty()) {
+				channelStateWriter.addOutputData(
+					barrier.getId(),
+					subpartitionInfo,
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					inflightBuffers.toArray(new Buffer[0]));
+			}
 		}

Review comment:
       nit: move to a separate `processCheckpointBarrier` method?
   
   or maybe also one step further:
   ```
   	private boolean addBuffer(BufferConsumer bufferConsumer) {
   		assert Thread.holdsLock(buffers);	
   		if (bufferConsumer.getDataType().hasPriority()) {
   			processPriorityBuffer(bufferConsumer);
   		}
   		buffers.add(bufferConsumer);
   		return false;
   	}
   ```
   
   to highlight that on the hot path, (without priority) we are skipping this whole code?




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837",
       "triggerID" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   * cb585f45dd49a291daefda4b4eb847262b75856e UNKNOWN
   * 7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837) 
   * 3333fd4ca67be7345172128322ff1cf17951808f 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 407169bc192b5d05bfba2c264849f1a6e7b81415 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402) 
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a6a2d36afcce1758e885f0751359ccf035ae2378 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -171,10 +183,36 @@ public InputStatus processInput() throws Exception {
 		}
 
 		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		updatePriorityAvailability();
 		checkFinished(inputStatus, readingInputIndex);
 		return inputSelectionHandler.updateStatus(inputStatus, readingInputIndex);
 	}
 
+	private void updatePriorityAvailability() {
+		if (lastPriorityInputIndex != InputSelection.NONE_AVAILABLE) {
+			final CompletableFuture<?> priorityEventAvailableFuture =
+				inputProcessors[lastPriorityInputIndex].taskInput.getPriorityEventAvailableFuture();
+			// no more priority events for the input
+			if (!priorityEventAvailableFuture.isDone()) {
+				prioritySelectionHandler.setUnavailableInput(lastPriorityInputIndex);
+				if (!prioritySelectionHandler.isAnyInputAvailable()) {
+					priorityAvailability.resetUnavailable();
+				}
+				priorityEventAvailableFuture.thenRun(onPriorityEvent(lastPriorityInputIndex));
+			}
+		}
+	}
+
+	private Runnable onPriorityEvent(int index) {
+		return () -> {
+			// set the priority flag in a mail before notifying StreamTask of availability
+			mainMailboxExecutor.execute(() -> {
+				prioritySelectionHandler.setAvailableInput(index);
+				priorityAvailability.getUnavailableToResetAvailable().complete(null);
+			}, "priority event {}", index);

Review comment:
       Commit removed; on `CheckpointedInputGate`, I'm adding the gate.toString().




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * b3a1520089c241fc74837902b6440d84a9636c14 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333) 
   * 16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349) 
   * 11cb1939f8a98340acab9b795c6f1894808fb606 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349) 
   * 11cb1939f8a98340acab9b795c6f1894808fb606 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * b3a1520089c241fc74837902b6440d84a9636c14 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333) 
   * 16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349) 
   * 11cb1939f8a98340acab9b795c6f1894808fb606 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -63,9 +66,38 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		final boolean hasPriorityEvents = inputGate.getPriorityEventAvailableFuture().isDone();
+		if (hasPriorityEvents) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}
+		}
+
+		// re-enqueue mail to process priority events
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	private void waitForPriorityEvents(InputGate inputGate, MailboxExecutor mailboxExecutor) {
+		final CompletableFuture<?> priorityEventAvailableFuture = inputGate.getPriorityEventAvailableFuture();
+		priorityEventAvailableFuture.thenRun(() -> {
+			mailboxExecutor.execute(this::processPriorityEvents, "process priority even @ gate %s", inputGate);
+		});

Review comment:
       1. Nope, this assumption does not hold. That's why the first thing that `processPriorityEvents` does is to check if the future is still completed. If the task polled the only priority event in the meantime, the future has been reset. During the execution of `processPriorityEvents` in the task thread, the task cannot concurrently pull the priority event, so this is safe.
   2.+3. The basic idea of not involving `StreamTaskNetworkInput#emitNext` or using `pollNext()` is to not make non-blocking output more complicated. Currently, `emitNext` or `pollNext` are only called when an output buffer is available. In the meantime only mails are processed. Hence, I used a mail to perform `processPriorityEvents`.
   Note that the assumption here is that no priority event ever need to be handled in `emitNext` (which currently only handles `EndOfPartitionEvent`)




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 19c4f0d1d1710b829946a78cb58aff768baab684 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 11cb1939f8a98340acab9b795c6f1894808fb606 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837",
       "triggerID" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6844",
       "triggerID" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "36c366400126cb68f0a8c8cf14b5c5e98ef2b58d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6848",
       "triggerID" : "36c366400126cb68f0a8c8cf14b5c5e98ef2b58d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * cb585f45dd49a291daefda4b4eb847262b75856e UNKNOWN
   * 3333fd4ca67be7345172128322ff1cf17951808f Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6844) 
   * 36c366400126cb68f0a8c8cf14b5c5e98ef2b58d Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6848) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778) 
   * 74094b3c0206de0bad49bdb7526534a8c43a5934 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740) 
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 1a86b34a3cd22d0ae34d16e9fa96036650dd6323 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765) 
   * 3c429de7c86d271a8980a34759026e18d3cf24b8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775) 
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -74,34 +106,34 @@ public CheckpointedInputGate(
 	}
 
 	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {

Review comment:
       Yes, it's changing semantics (as I had written in commit message). I have not found a good reason why it's not always exited and it makes things easier especially since this method can now be used to process priority events.
   Btw I think it also changes semantics for all event that are not handled at all, but I'm not sure which events survive at this point (Superstep?).




----------------------------------------------------------------
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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -621,61 +626,84 @@ public boolean isFinished() {
 		return Optional.of(transformToBufferOrEvent(
 			inputWithData.data.buffer(),
 			inputWithData.moreAvailable,
-			inputWithData.input));
+			inputWithData.input,
+			inputWithData.morePriorityEvents));
 	}
 
 	private Optional<InputWithData<InputChannel, BufferAndAvailability>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<InputChannel> inputChannel = getChannel(blocking);
-			if (!inputChannel.isPresent()) {
+			Optional<InputChannel> inputChannelOpt = getChannel(blocking);
+			if (!inputChannelOpt.isPresent()) {
 				return Optional.empty();
 			}
 
 			// Do not query inputChannel under the lock, to avoid potential deadlocks coming from
 			// notifications.
-			Optional<BufferAndAvailability> result = inputChannel.get().getNextBuffer();
+			final InputChannel inputChannel = inputChannelOpt.get();
+			Optional<BufferAndAvailability> bufferAndAvailabilityOpt = inputChannel.getNextBuffer();
 
 			synchronized (inputChannelsWithData) {
-				if (result.isPresent() && result.get().moreAvailable()) {
+				if (!bufferAndAvailabilityOpt.isPresent()) {
+					if (inputChannelsWithData.isEmpty()) {
+						availabilityHelper.resetUnavailable();
+					}
+					continue;

Review comment:
       maybe if the `result` variable rename and adding `continue` branch had happened in an independent "refactor" commit, It would have saved me a couple of minutes while reading this code while I was trying to understand the change :( 
   
   maybe not, as I can see how the changes are a bit interconnected.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a22657562a42dda270f89c176e5c260ee73c6698 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740) 
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 1a86b34a3cd22d0ae34d16e9fa96036650dd6323 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765) 
   * 3c429de7c86d271a8980a34759026e18d3cf24b8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775) 
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a6a2d36afcce1758e885f0751359ccf035ae2378 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446) 
   * fa3e495b6ba201769d908adea420e4944ddd7643 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a1fb1c3114305ed5fce898dfcc03abed4d6963d4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539) 
   * 834c28da93d4b24e562e017979b483e288db02a2 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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






----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Added a ticket and referenced it properly https://issues.apache.org/jira/browse/FLINK-19338 .

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       I added an implNote in the javadoc.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       I'm moving it to the commit that starts using it.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
##########
@@ -22,13 +22,10 @@
 import org.apache.flink.annotation.VisibleForTesting;

Review comment:
       I admit that it's an awkward cut. 
   
   However, it's only spilled in one place as the `BufferReceivedListener` methods are effectively not called in the previous commits anymore. I will make a later pass to see that all tests pass though.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -314,6 +315,21 @@ protected StreamTask(
 		}
 
 		this.channelIOExecutor = Executors.newSingleThreadExecutor(new ExecutorThreadFactory("channel-state-unspilling"));
+
+		injectChannelStateWriterIntoChannels();
+	}
+
+	private void injectChannelStateWriterIntoChannels() {
+		final Environment env = getEnvironment();
+		final ChannelStateWriter channelStateWriter = subtaskCheckpointCoordinator.getChannelStateWriter();
+		for (final InputGate gate : env.getAllInputGates()) {
+			gate.setChannelStateWriter(channelStateWriter);
+		}
+		for (ResultPartitionWriter writer : env.getAllWriters()) {
+			if (writer instanceof ChannelStateHolder) {
+				((ChannelStateHolder) writer).setChannelStateWriter(channelStateWriter);
+			}
+		}

Review comment:
       Happy for any other suggestion. I think the cast itself is in line with recent changes done by Stephan.
   
   The proper solution would be to inject in ctor but that will not happen until we merge `runtime` and `streaming`.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
##########
@@ -232,19 +232,16 @@ public int getInputIndex() {
 			ChannelStateWriter channelStateWriter,
 			long checkpointId) throws IOException {
 		for (int channelIndex = 0; channelIndex < recordDeserializers.length; channelIndex++) {
-			final InputChannel channel = checkpointedInputGate.getChannel(channelIndex);
-
-			// Assumption for retrieving buffers = one concurrent checkpoint
 			RecordDeserializer<?> deserializer = recordDeserializers[channelIndex];
 			if (deserializer != null) {
+				final InputChannel channel = checkpointedInputGate.getChannel(channelIndex);
+
 				channelStateWriter.addInputData(
 					checkpointId,
 					channel.getChannelInfo(),
 					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
 					deserializer.getUnconsumedBuffer());
 			}
-
-			checkpointedInputGate.spillInflightBuffers(checkpointId, channelIndex, channelStateWriter);

Review comment:
       We discussed offline, but let me summarize the main point. When spilling in main thread, buffers would be spilled rather late as you pointed out initially. 
   One solution is to trigger the channel spilling in task thread as soon as possible and then resume spilling from netty until barrier arrives. However, that results in a complex threading model with lots of race condition as we found out in the version in master.
   Another solution is to spill in task thread and use any poll to discover new buffers and spill them. It's slightly slower and also requires lots of internal knowledge at the `Unaligner` about the channels to work well (mark all spilled channels). It's probably also suboptimal as new buffers are usually enqueued right after the head is polled, so one buffer is enqueued in the channel but not persisted until the new head is polled.
   The proposed solution is to spill in netty thread entirely. That's the fastest possible solution with an comparably easy threading model. Downside is the added complexity on channel-side, but the general idea is that upstream and downstream side of a channel is now self-contained.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       Moved this method to the commit that spills immediately. We need it in that method to retrieve the checkpoint id to spill correctly.
   
   Deserialization is only necessary for priority events, which are very rare and rather cheap (30 bytes). I'd argue that adding a new call chain just to optimize it is not warranted.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       Moved the removal to the commit that spills immediately.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -173,50 +181,51 @@ private boolean addBuffer(BufferConsumer bufferConsumer) {
 			buffers.add(bufferConsumer);
 			return false;
 		}
-		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-			"checkpoints");
 
 		final int pos = buffers.getNumPriorityElements();
 		buffers.addPriorityElement(bufferConsumer);
 
-		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
-		if (unalignedCheckpoint) {
+		CheckpointBarrier barrier = parseCheckpointBarrier(bufferConsumer);
+		if (barrier != null) {
+			checkState(
+				barrier.getCheckpointOptions().isUnalignedCheckpoint(),
+				"Only unaligned checkpoints should be priority events");
 			final Iterator<BufferConsumer> iterator = buffers.iterator();
 			Iterators.advance(iterator, pos + 1);
+			List<Buffer> inflightBuffers = new ArrayList<>();
 			while (iterator.hasNext()) {
 				BufferConsumer buffer = iterator.next();
 
 				if (buffer.isBuffer()) {
 					try (BufferConsumer bc = buffer.copy()) {
-						inflightBufferSnapshot.add(bc.build());
+						inflightBuffers.add(bc.build());
 					}
 				}
 			}
+			channelStateWriter.addOutputData(
+				barrier.getId(),
+				subpartitionInfo,
+				ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+				inflightBuffers.toArray(new Buffer[0]));
 		}
 		return pos == 0;
 	}
 
-	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
-		boolean unalignedCheckpoint;
+	@Nullable
+	private CheckpointBarrier parseCheckpointBarrier(BufferConsumer bufferConsumer) {

Review comment:
       Hm I have a hard time coming up with a better code structure. I could add checkpoint parsing and the `if` in `addBuffer` already in the first commit `[FLINK-19026][network] Adding PrioritizedDeque and use it in PipelinedSubpartition.`. Then this diff would be only about persisting itself.
   
   But I was convinced that you would be confused why we need to parse the barrier at that commit.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       Removed thanks to alternative fix of FLINK-12510 (see previous commit now).

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerTestBase.java
##########
@@ -369,9 +369,7 @@ public void testMissingCancellationBarriers() throws Exception {
 		inputGate = createBarrierBuffer(2, sequence, validator);

Review comment:
       > A side-effect of this commit is that all events are handed over from CheckpointedInputGate to StreamTaskNetworkInput and break up the poll loop. However, since events are rare, it should have no visible impact on the throughput.
   
   The changes to the tests are now handling the additionally emitted events. Imho tests are easier to read now (no magically disappearing buffers in the sequence).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       I expanded the comment (now javadoc) to clearly state the contract.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -171,10 +183,36 @@ public InputStatus processInput() throws Exception {
 		}
 
 		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		updatePriorityAvailability();
 		checkFinished(inputStatus, readingInputIndex);
 		return inputSelectionHandler.updateStatus(inputStatus, readingInputIndex);
 	}
 
+	private void updatePriorityAvailability() {
+		if (lastPriorityInputIndex != InputSelection.NONE_AVAILABLE) {
+			final CompletableFuture<?> priorityEventAvailableFuture =
+				inputProcessors[lastPriorityInputIndex].taskInput.getPriorityEventAvailableFuture();
+			// no more priority events for the input
+			if (!priorityEventAvailableFuture.isDone()) {
+				prioritySelectionHandler.setUnavailableInput(lastPriorityInputIndex);
+				if (!prioritySelectionHandler.isAnyInputAvailable()) {
+					priorityAvailability.resetUnavailable();
+				}
+				priorityEventAvailableFuture.thenRun(onPriorityEvent(lastPriorityInputIndex));
+			}
+		}
+	}
+
+	private Runnable onPriorityEvent(int index) {
+		return () -> {
+			// set the priority flag in a mail before notifying StreamTask of availability
+			mainMailboxExecutor.execute(() -> {
+				prioritySelectionHandler.setAvailableInput(index);
+				priorityAvailability.getUnavailableToResetAvailable().complete(null);

Review comment:
       Moved it even further up towards `CheckpointedInputGate`. At this point, we need to make sure that a priority event is really at the top (hence the optimistic lock protocol for notification).




----------------------------------------------------------------
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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837",
       "triggerID" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6844",
       "triggerID" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "36c366400126cb68f0a8c8cf14b5c5e98ef2b58d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6848",
       "triggerID" : "36c366400126cb68f0a8c8cf14b5c5e98ef2b58d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * cb585f45dd49a291daefda4b4eb847262b75856e UNKNOWN
   * 7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837) 
   * 3333fd4ca67be7345172128322ff1cf17951808f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6844) 
   * 36c366400126cb68f0a8c8cf14b5c5e98ef2b58d Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6848) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a22657562a42dda270f89c176e5c260ee73c6698 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693) 
   * f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f0bb8a255816919a6578a6994499757075ab371b 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * ef2c7ecea91a30e4d721efa759ca1b15728a24cb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Added a ticket and referenced it properly https://issues.apache.org/jira/browse/FLINK-19338 .

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       I added an implNote in the javadoc.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       I'm moving it to the commit that starts using it.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
##########
@@ -22,13 +22,10 @@
 import org.apache.flink.annotation.VisibleForTesting;

Review comment:
       I admit that it's an awkward cut. 
   
   However, it's only spilled in one place as the `BufferReceivedListener` methods are effectively not called in the previous commits anymore. I will make a later pass to see that all tests pass though.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -314,6 +315,21 @@ protected StreamTask(
 		}
 
 		this.channelIOExecutor = Executors.newSingleThreadExecutor(new ExecutorThreadFactory("channel-state-unspilling"));
+
+		injectChannelStateWriterIntoChannels();
+	}
+
+	private void injectChannelStateWriterIntoChannels() {
+		final Environment env = getEnvironment();
+		final ChannelStateWriter channelStateWriter = subtaskCheckpointCoordinator.getChannelStateWriter();
+		for (final InputGate gate : env.getAllInputGates()) {
+			gate.setChannelStateWriter(channelStateWriter);
+		}
+		for (ResultPartitionWriter writer : env.getAllWriters()) {
+			if (writer instanceof ChannelStateHolder) {
+				((ChannelStateHolder) writer).setChannelStateWriter(channelStateWriter);
+			}
+		}

Review comment:
       Happy for any other suggestion. I think the cast itself is in line with recent changes done by Stephan.
   
   The proper solution would be to inject in ctor but that will not happen until we merge `runtime` and `streaming`.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
##########
@@ -232,19 +232,16 @@ public int getInputIndex() {
 			ChannelStateWriter channelStateWriter,
 			long checkpointId) throws IOException {
 		for (int channelIndex = 0; channelIndex < recordDeserializers.length; channelIndex++) {
-			final InputChannel channel = checkpointedInputGate.getChannel(channelIndex);
-
-			// Assumption for retrieving buffers = one concurrent checkpoint
 			RecordDeserializer<?> deserializer = recordDeserializers[channelIndex];
 			if (deserializer != null) {
+				final InputChannel channel = checkpointedInputGate.getChannel(channelIndex);
+
 				channelStateWriter.addInputData(
 					checkpointId,
 					channel.getChannelInfo(),
 					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
 					deserializer.getUnconsumedBuffer());
 			}
-
-			checkpointedInputGate.spillInflightBuffers(checkpointId, channelIndex, channelStateWriter);

Review comment:
       We discussed offline, but let me summarize the main point. When spilling in main thread, buffers would be spilled rather late as you pointed out initially. 
   One solution is to trigger the channel spilling in task thread as soon as possible and then resume spilling from netty until barrier arrives. However, that results in a complex threading model with lots of race condition as we found out in the version in master.
   Another solution is to spill in task thread and use any poll to discover new buffers and spill them. It's slightly slower and also requires lots of internal knowledge at the `Unaligner` about the channels to work well (mark all spilled channels). It's probably also suboptimal as new buffers are usually enqueued right after the head is polled, so one buffer is enqueued in the channel but not persisted until the new head is polled.
   The proposed solution is to spill in netty thread entirely. That's the fastest possible solution with an comparably easy threading model. Downside is the added complexity on channel-side, but the general idea is that upstream and downstream side of a channel is now self-contained.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       Moved this method to the commit that spills immediately. We need it in that method to retrieve the checkpoint id to spill correctly.
   
   Deserialization is only necessary for priority events, which are very rare and rather cheap (30 bytes). I'd argue that adding a new call chain just to optimize it is not warranted.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       Moved the removal to the commit that spills immediately.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -173,50 +181,51 @@ private boolean addBuffer(BufferConsumer bufferConsumer) {
 			buffers.add(bufferConsumer);
 			return false;
 		}
-		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-			"checkpoints");
 
 		final int pos = buffers.getNumPriorityElements();
 		buffers.addPriorityElement(bufferConsumer);
 
-		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
-		if (unalignedCheckpoint) {
+		CheckpointBarrier barrier = parseCheckpointBarrier(bufferConsumer);
+		if (barrier != null) {
+			checkState(
+				barrier.getCheckpointOptions().isUnalignedCheckpoint(),
+				"Only unaligned checkpoints should be priority events");
 			final Iterator<BufferConsumer> iterator = buffers.iterator();
 			Iterators.advance(iterator, pos + 1);
+			List<Buffer> inflightBuffers = new ArrayList<>();
 			while (iterator.hasNext()) {
 				BufferConsumer buffer = iterator.next();
 
 				if (buffer.isBuffer()) {
 					try (BufferConsumer bc = buffer.copy()) {
-						inflightBufferSnapshot.add(bc.build());
+						inflightBuffers.add(bc.build());
 					}
 				}
 			}
+			channelStateWriter.addOutputData(
+				barrier.getId(),
+				subpartitionInfo,
+				ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+				inflightBuffers.toArray(new Buffer[0]));
 		}
 		return pos == 0;
 	}
 
-	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
-		boolean unalignedCheckpoint;
+	@Nullable
+	private CheckpointBarrier parseCheckpointBarrier(BufferConsumer bufferConsumer) {

Review comment:
       Hm I have a hard time coming up with a better code structure. I could add checkpoint parsing and the `if` in `addBuffer` already in the first commit `[FLINK-19026][network] Adding PrioritizedDeque and use it in PipelinedSubpartition.`. Then this diff would be only about persisting itself.
   
   But I was convinced that you would be confused why we need to parse the barrier at that commit.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       Removed thanks to alternative fix of FLINK-12510 (see previous commit now).

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerTestBase.java
##########
@@ -369,9 +369,7 @@ public void testMissingCancellationBarriers() throws Exception {
 		inputGate = createBarrierBuffer(2, sequence, validator);

Review comment:
       > A side-effect of this commit is that all events are handed over from CheckpointedInputGate to StreamTaskNetworkInput and break up the poll loop. However, since events are rare, it should have no visible impact on the throughput.
   
   The changes to the tests are now handling the additionally emitted events. Imho tests are easier to read now (no magically disappearing buffers in the sequence).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       I expanded the comment (now javadoc) to clearly state the contract.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       Extracted into `GateNotificationHelper`, please check if it's actually helping to reduce complexity.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       Added a few thoughts. Let me know if it makes things clearer.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -147,6 +148,14 @@ else if (configuredInput instanceof SourceInputConfig) {
 		return anyInputAvailable;
 	}
 
+	@Override
+	public CompletableFuture<?> getPriorityEventAvailableFuture() {
+		return CompletableFuture.anyOf(
+			Arrays.stream(inputProcessors)
+				.map(inputProcessor -> inputProcessor.taskInput.getPriorityEventAvailableFuture())
+				.toArray(CompletableFuture[]::new));

Review comment:
       Commit is removed.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -173,50 +181,51 @@ private boolean addBuffer(BufferConsumer bufferConsumer) {
 			buffers.add(bufferConsumer);
 			return false;
 		}
-		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-			"checkpoints");
 
 		final int pos = buffers.getNumPriorityElements();
 		buffers.addPriorityElement(bufferConsumer);
 
-		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
-		if (unalignedCheckpoint) {
+		CheckpointBarrier barrier = parseCheckpointBarrier(bufferConsumer);
+		if (barrier != null) {
+			checkState(
+				barrier.getCheckpointOptions().isUnalignedCheckpoint(),
+				"Only unaligned checkpoints should be priority events");
 			final Iterator<BufferConsumer> iterator = buffers.iterator();
 			Iterators.advance(iterator, pos + 1);
+			List<Buffer> inflightBuffers = new ArrayList<>();
 			while (iterator.hasNext()) {
 				BufferConsumer buffer = iterator.next();
 
 				if (buffer.isBuffer()) {
 					try (BufferConsumer bc = buffer.copy()) {
-						inflightBufferSnapshot.add(bc.build());
+						inflightBuffers.add(bc.build());
 					}
 				}
 			}
+			channelStateWriter.addOutputData(
+				barrier.getId(),
+				subpartitionInfo,
+				ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+				inflightBuffers.toArray(new Buffer[0]));

Review comment:
       Not relevant in the final version where the channel spills by itself (no return value on this method). I can make it clearer in the commit message if you want.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
##########
@@ -210,15 +221,25 @@ public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelSt
 		}
 
 		Buffer buffer = next.buffer();
-		CheckpointBarrier notifyReceivedBarrier = parseCheckpointBarrierOrNull(buffer);
-		if (notifyReceivedBarrier != null) {
-			receivedCheckpointId = notifyReceivedBarrier.getId();
-		} else if (receivedCheckpointId < lastRequestedCheckpointId && buffer.isBuffer()) {
-			inputGate.getBufferReceivedListener().notifyBufferReceived(buffer.retainBuffer(), channelInfo);
-		}
 
 		numBytesIn.inc(buffer.getSize());
 		numBuffersIn.inc();
+		if (buffer.isBuffer()) {
+			for (final long barrierId : pendingCheckpointBarriers) {
+				channelStateWriter.addInputData(
+					barrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));

Review comment:
       `LocalInputChannel` only spills when it awaits barrier. So it spills the buffer on first sight and it cannot be better on downsteam level. We could of course also move spilling lingering buffers to the upstream. It might also be an improvement for later.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
##########
@@ -210,15 +221,25 @@ public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelSt
 		}
 
 		Buffer buffer = next.buffer();
-		CheckpointBarrier notifyReceivedBarrier = parseCheckpointBarrierOrNull(buffer);
-		if (notifyReceivedBarrier != null) {
-			receivedCheckpointId = notifyReceivedBarrier.getId();
-		} else if (receivedCheckpointId < lastRequestedCheckpointId && buffer.isBuffer()) {
-			inputGate.getBufferReceivedListener().notifyBufferReceived(buffer.retainBuffer(), channelInfo);
-		}
 
 		numBytesIn.inc(buffer.getSize());
 		numBuffersIn.inc();
+		if (buffer.isBuffer()) {
+			for (final long barrierId : pendingCheckpointBarriers) {
+				channelStateWriter.addInputData(
+					barrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));

Review comment:
       `LocalInputChannel` only spills when it awaits barrier. So it spills the buffer on first sight and it cannot be better on downsteam level. We could of course also move spilling lingering buffers to the upstream. It might also be an improvement for later, but it adds quite a bit of complexity as barriers also need to be propagated upstream.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -89,17 +96,15 @@
 	/** The number of available buffers that have not been announced to the producer yet. */
 	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
 
-	/**
-	 * The latest already triggered checkpoint id which would be updated during
-	 * {@link #spillInflightBuffers(long, ChannelStateWriter)}.
-	 */
-	@GuardedBy("receivedBuffers")
-	private long lastRequestedCheckpointId = -1;
+	private final BufferManager bufferManager;
 
-	/** The current received checkpoint id from the network. */
-	private long receivedCheckpointId = -1;
+	/** Stores #overtaken buffers when a checkpoint barrier is received before task thread started checkpoint. */
+	@GuardedBy("receivedBuffers")
+	private Map<Long, Integer> numBuffersOvertaken = new HashMap<>();
 
-	private final BufferManager bufferManager;
+	/** All started checkpoints where a barrier has not been received yet. */
+	@GuardedBy("receivedBuffers")
+	private Deque<Long> pendingCheckpointBarriers = new ArrayDeque<>(2);

Review comment:
       Changed it to support only one concurrent checkpoint. I also extracted the whole logic into one helper class that can be used by both `LocalInputChannel` and `RemoteInputChannel` (with syncrhonization).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -454,42 +431,106 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 				}
 
 				wasEmpty = receivedBuffers.isEmpty();
-				receivedBuffers.add(buffer);
 
-				if (listener != null && buffer.isBuffer() && receivedCheckpointId < lastRequestedCheckpointId) {
-					notifyReceivedBuffer = buffer.retainBuffer();
+				AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+				if (priorityEvent != null) {
+					receivedBuffers.addPriorityElement(buffer);
+					final int pos = receivedBuffers.getNumPriorityElements();
+					if (priorityEvent instanceof CheckpointBarrier) {
+						final long barrierId = ((CheckpointBarrier) priorityEvent).getId();
+						// don't spill future buffers for this checkpoint
+						if (!pendingCheckpointBarriers.remove(barrierId)) {
+							// checkpoint was not yet started by task thread,
+							// so remember the numbers of buffers to spill for the time when it will be started
+							numBuffersOvertaken.put(barrierId, receivedBuffers.size() - pos);
+						}
+					}
+					firstPriorityEvent = pos == 1;
 				} else {
-					notifyReceivedBuffer = null;
+					receivedBuffers.add(buffer);
+					if (buffer.isBuffer()) {
+						for (final long checkpointId : pendingCheckpointBarriers) {
+							channelStateWriter.addInputData(
+								checkpointId,
+								channelInfo,
+								sequenceNumber,
+								CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+						}
+					}

Review comment:
       Much smaller now thanks to the helper class.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -454,42 +431,106 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 				}
 
 				wasEmpty = receivedBuffers.isEmpty();
-				receivedBuffers.add(buffer);
 
-				if (listener != null && buffer.isBuffer() && receivedCheckpointId < lastRequestedCheckpointId) {
-					notifyReceivedBuffer = buffer.retainBuffer();
+				AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+				if (priorityEvent != null) {
+					receivedBuffers.addPriorityElement(buffer);
+					final int pos = receivedBuffers.getNumPriorityElements();
+					if (priorityEvent instanceof CheckpointBarrier) {
+						final long barrierId = ((CheckpointBarrier) priorityEvent).getId();
+						// don't spill future buffers for this checkpoint
+						if (!pendingCheckpointBarriers.remove(barrierId)) {
+							// checkpoint was not yet started by task thread,
+							// so remember the numbers of buffers to spill for the time when it will be started
+							numBuffersOvertaken.put(barrierId, receivedBuffers.size() - pos);
+						}
+					}
+					firstPriorityEvent = pos == 1;
 				} else {
-					notifyReceivedBuffer = null;
+					receivedBuffers.add(buffer);
+					if (buffer.isBuffer()) {
+						for (final long checkpointId : pendingCheckpointBarriers) {
+							channelStateWriter.addInputData(
+								checkpointId,
+								channelInfo,
+								sequenceNumber,
+								CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+						}
+					}
 				}
-				notifyReceivedBarrier = listener != null ? parseCheckpointBarrierOrNull(buffer) : null;
 			}
 			recycleBuffer = false;
 
 			++expectedSequenceNumber;
 
+			if (firstPriorityEvent) {
+				notifyPriorityEvent();
+			}
 			if (wasEmpty) {
 				notifyChannelNonEmpty();
 			}
 
 			if (backlog >= 0) {
 				onSenderBacklog(backlog);
 			}
-
-			if (notifyReceivedBarrier != null) {
-				receivedCheckpointId = notifyReceivedBarrier.getId();
-				if (notifyReceivedBarrier.isCheckpoint()) {
-					listener.notifyBarrierReceived(notifyReceivedBarrier, channelInfo);
-				}
-			} else if (notifyReceivedBuffer != null) {
-				listener.notifyBufferReceived(notifyReceivedBuffer, channelInfo);
-			}
 		} finally {
 			if (recycleBuffer) {
 				buffer.recycleBuffer();
 			}
 		}
 	}
 
+	/**
+	 * Spills all queued buffers on checkpoint start. If barrier has already been received (and reordered), spill only
+	 * the overtaken buffers.
+	 */
+	public void checkpointStarted(CheckpointBarrier barrier) {
+		checkState(channelStateWriter != null, "Channel state writer not injected");
+		synchronized (receivedBuffers) {
+			final Integer numBuffers = numBuffersOvertaken.get(barrier.getId());
+			if (numBuffers != null) {
+				// already received barrier before the task thread picked up the barrier of this or another channel
+				spillBuffers(barrier.getId(), numBuffers);
+			} else {
+				// barrier not yet received, spill all current and future buffers
+				spillBuffers(barrier.getId(), receivedBuffers.getNumUnprioritizedElements());
+				pendingCheckpointBarriers.add(barrier.getId());
+			}
+		}
+	}
+
+	public void checkpointStopped(long checkpointId) {
+		synchronized (receivedBuffers) {
+			numBuffersOvertaken.remove(checkpointId);
+			pendingCheckpointBarriers.remove(checkpointId);
+		}
+	}
+
+	private void spillBuffers(long checkpointId, int numBuffers) {

Review comment:
       I added a helper class `ChannelStatePersister` and used `persist` everywhere to have a clear separation of the two names. `Persister` is more on the logical level and `Writer` on the physical implementation.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -171,10 +183,36 @@ public InputStatus processInput() throws Exception {
 		}
 
 		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		updatePriorityAvailability();
 		checkFinished(inputStatus, readingInputIndex);
 		return inputSelectionHandler.updateStatus(inputStatus, readingInputIndex);
 	}
 
+	private void updatePriorityAvailability() {
+		if (lastPriorityInputIndex != InputSelection.NONE_AVAILABLE) {
+			final CompletableFuture<?> priorityEventAvailableFuture =
+				inputProcessors[lastPriorityInputIndex].taskInput.getPriorityEventAvailableFuture();
+			// no more priority events for the input
+			if (!priorityEventAvailableFuture.isDone()) {
+				prioritySelectionHandler.setUnavailableInput(lastPriorityInputIndex);
+				if (!prioritySelectionHandler.isAnyInputAvailable()) {
+					priorityAvailability.resetUnavailable();
+				}
+				priorityEventAvailableFuture.thenRun(onPriorityEvent(lastPriorityInputIndex));
+			}
+		}
+	}
+
+	private Runnable onPriorityEvent(int index) {
+		return () -> {
+			// set the priority flag in a mail before notifying StreamTask of availability
+			mainMailboxExecutor.execute(() -> {
+				prioritySelectionHandler.setAvailableInput(index);
+				priorityAvailability.getUnavailableToResetAvailable().complete(null);
+			}, "priority event {}", index);

Review comment:
       Commit removed; on `CheckpointedInputGate`, I'm adding the gate.toString().

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
##########
@@ -171,10 +183,36 @@ public InputStatus processInput() throws Exception {
 		}
 
 		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		updatePriorityAvailability();
 		checkFinished(inputStatus, readingInputIndex);
 		return inputSelectionHandler.updateStatus(inputStatus, readingInputIndex);
 	}
 
+	private void updatePriorityAvailability() {
+		if (lastPriorityInputIndex != InputSelection.NONE_AVAILABLE) {
+			final CompletableFuture<?> priorityEventAvailableFuture =
+				inputProcessors[lastPriorityInputIndex].taskInput.getPriorityEventAvailableFuture();
+			// no more priority events for the input
+			if (!priorityEventAvailableFuture.isDone()) {
+				prioritySelectionHandler.setUnavailableInput(lastPriorityInputIndex);
+				if (!prioritySelectionHandler.isAnyInputAvailable()) {
+					priorityAvailability.resetUnavailable();
+				}
+				priorityEventAvailableFuture.thenRun(onPriorityEvent(lastPriorityInputIndex));
+			}
+		}
+	}
+
+	private Runnable onPriorityEvent(int index) {
+		return () -> {
+			// set the priority flag in a mail before notifying StreamTask of availability
+			mainMailboxExecutor.execute(() -> {
+				prioritySelectionHandler.setAvailableInput(index);
+				priorityAvailability.getUnavailableToResetAvailable().complete(null);

Review comment:
       Moved it even further up towards `CheckpointedInputGate`. At this point, we need to make sure that a priority event is really at the top (hence the optimistic lock protocol for notification).

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
##########
@@ -113,7 +113,11 @@ public void processBarrier(CheckpointBarrier barrier, InputChannelInfo channelIn
 
 			if (++numBarriersReceived == numOpenChannels) {
 				allBarriersReceivedFuture.complete(null);
-				resetPendingCheckpoint(barrierId);
+				for (final InputGate gate : inputGates) {
+					for (int index = 0, numChannels = gate.getNumberOfInputChannels(); index < numChannels; index++) {
+						gate.getChannel(index).checkpointStopped(currentCheckpointId);
+					}
+				}

Review comment:
       Sorry that was just a test commit to see if the stuck e2e failed because of this change. I removed it. The original change is covered by a few unit tests already.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -361,4 +360,89 @@ public String toString() {
 				'}';
 		}
 	}
+
+	/**
+	 * Helper class for persisting channel state via {@link ChannelStateWriter}.
+	 */
+	@NotThreadSafe
+	protected final class ChannelStatePersister {
+		private static final long CHECKPOINT_COMPLETED = -1;
+
+		private static final long BARRIER_RECEIVED = -2;
+
+		/** All started checkpoints where a barrier has not been received yet. */
+		private long pendingCheckpointBarrierId = CHECKPOINT_COMPLETED;
+
+		/** Writer must be initialized before usage. {@link #startPersisting(long, List)} enforces this invariant. */
+		@Nullable
+		private final ChannelStateWriter channelStateWriter;
+
+		public ChannelStatePersister(@Nullable ChannelStateWriter channelStateWriter) {
+			this.channelStateWriter = channelStateWriter;
+		}
+
+		protected void startPersisting(long barrierId, List<Buffer> knownBuffers) {
+			checkState(isInitialized(), "Channel state writer not injected");
+
+			if (pendingCheckpointBarrierId != BARRIER_RECEIVED) {
+				pendingCheckpointBarrierId = barrierId;
+			}
+			if (knownBuffers.size() > 0) {
+				channelStateWriter.addInputData(
+					barrierId,
+					channelInfo,
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.fromList(knownBuffers, Buffer::recycleBuffer));
+			}
+		}
+
+		protected boolean isInitialized() {
+			return channelStateWriter != null;
+		}
+
+		protected void stopPersisting() {
+			pendingCheckpointBarrierId = CHECKPOINT_COMPLETED;
+		}
+
+		protected void maybePersist(Buffer buffer) {
+			if (pendingCheckpointBarrierId >= 0 && buffer.isBuffer()) {
+				channelStateWriter.addInputData(
+					pendingCheckpointBarrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+			}
+		}
+
+		protected boolean checkForBarrier(Buffer buffer) throws IOException {
+			final AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+			if (priorityEvent instanceof CheckpointBarrier) {
+				pendingCheckpointBarrierId = BARRIER_RECEIVED;
+				return true;
+			}
+			return false;
+		}
+
+		/**
+		 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
+		 * returns null in all other cases.
+		 */
+		@Nullable
+		protected AbstractEvent parsePriorityEvent(Buffer buffer) throws IOException {
+			if (buffer.isBuffer() || !buffer.getDataType().hasPriority()) {
+				return null;
+			}
+
+			AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+			// reset the buffer because it would be deserialized again in SingleInputGate while getting next buffer.
+			// we can further improve to avoid double deserialization in the future.
+			buffer.setReaderIndex(0);
+			return event;
+		}

Review comment:
       It's only used in this class and thus I inlined it. As is, it really looks like an interface of the Persister.
   
   There is similar code for `BufferConsumer` on output side, but the implementation is too different to align (copy buffer vs. reader index reset, different `EventSerializer.fromBuffer` overloads).

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java
##########
@@ -120,67 +112,10 @@ public void testNoDataProcessedAfterCheckpointBarrier() throws Exception {
 		assertEquals(0, output.getNumberOfEmittedRecords());
 	}
 
-	@Test
-	public void testSnapshotWithTwoInputGates() throws Exception {

Review comment:
       I added this test for FLINK-18139 - input gate index issues in `Unaligner#hasInflightData`, which is called by `StreamTaskNetworkInput`, hence the test here.
   
   However, in this commit, `hasInflightData` is removed. Spilling is fully encapsulated in `InputChannel` + `CheckpointBarrierUnaligner`/`CheckpointedInputGate` and this interaction is already covered in tests at `CheckpointBarrierUnalignerTest`. 

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
##########
@@ -812,242 +788,13 @@ public void testQueuedBuffers() throws Exception {
 		}
 	}
 
-	@Test
-	public void testBufferReceivedListener() throws Exception {

Review comment:
       Good catch, my intent was to delete `testBufferReceivedListener` and `testPartitionNotFoundExceptionWhileGetNextBuffer` but not the test in between them.
   
   `testBufferReceivedListener` tests `BufferReceivedListener` which this commits renders useless (and is later removed).
   
   `testPartitionNotFoundExceptionWhileGetNextBuffer` tests concurrent spilling of lingering buffers and receiving of such lingering buffers. Both now happens in the same thread, so the test does not make any sense.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -62,9 +65,37 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		if (inputGate.getPriorityEventAvailableFuture().isDone()) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}
+		}
+

Review comment:
       Good idea, I solved it in the following way:
   
   ```
   		// check if the priority event is still not processed (could have been pulled before mail was being executed)
   		boolean hasPriorityEvent = inputGate.getPriorityEventAvailableFuture().isDone();
   		while (hasPriorityEvent) {
   			// process as many priority events as possible
   			final Optional<BufferOrEvent> bufferOrEventOpt = pollNext();
   			bufferOrEventOpt.ifPresent(bufferOrEvent ->
   				checkState(bufferOrEvent.hasPriority(), "Should only poll priority events"));
   			hasPriorityEvent = bufferOrEventOpt.map(BufferOrEvent::morePriorityEvents).orElse(false);
   		}
   ```
   
   `checkState(!inputGate.getPriorityEventAvailableFuture().isDone())` might be failing if netty receives a new priority event and triggers this available future while the task thread polled the last priority event. This case should happen quite often when the first barrier arrives (at that time the only priority event, morePriorityEvents = false) and triggers the whole checkpointing process. The second barrier would then complete the `getPriorityEventAvailableFuture` causing a more or less immediate re-execution of this 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] AHeise commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       Extracted into `GateNotificationHelper`, please check if it's actually helping to reduce complexity.




----------------------------------------------------------------
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] zentol commented on pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   CI shows a timeout in {{DataSinkTaskTest.testDataSinkTask}}.


----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 19c4f0d1d1710b829946a78cb58aff768baab684 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365) 
   * 40c76a4ce74a5bbc800cf9833b96bb0156cacc41 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a1fb1c3114305ed5fce898dfcc03abed4d6963d4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   


----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -361,4 +360,89 @@ public String toString() {
 				'}';
 		}
 	}
+
+	/**
+	 * Helper class for persisting channel state via {@link ChannelStateWriter}.
+	 */
+	@NotThreadSafe
+	protected final class ChannelStatePersister {
+		private static final long CHECKPOINT_COMPLETED = -1;
+
+		private static final long BARRIER_RECEIVED = -2;
+
+		/** All started checkpoints where a barrier has not been received yet. */
+		private long pendingCheckpointBarrierId = CHECKPOINT_COMPLETED;
+
+		/** Writer must be initialized before usage. {@link #startPersisting(long, List)} enforces this invariant. */
+		@Nullable
+		private final ChannelStateWriter channelStateWriter;
+
+		public ChannelStatePersister(@Nullable ChannelStateWriter channelStateWriter) {
+			this.channelStateWriter = channelStateWriter;
+		}
+
+		protected void startPersisting(long barrierId, List<Buffer> knownBuffers) {
+			checkState(isInitialized(), "Channel state writer not injected");
+
+			if (pendingCheckpointBarrierId != BARRIER_RECEIVED) {
+				pendingCheckpointBarrierId = barrierId;
+			}
+			if (knownBuffers.size() > 0) {
+				channelStateWriter.addInputData(
+					barrierId,
+					channelInfo,
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.fromList(knownBuffers, Buffer::recycleBuffer));
+			}
+		}
+
+		protected boolean isInitialized() {
+			return channelStateWriter != null;
+		}
+
+		protected void stopPersisting() {
+			pendingCheckpointBarrierId = CHECKPOINT_COMPLETED;
+		}
+
+		protected void maybePersist(Buffer buffer) {
+			if (pendingCheckpointBarrierId >= 0 && buffer.isBuffer()) {
+				channelStateWriter.addInputData(
+					pendingCheckpointBarrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+			}
+		}
+
+		protected boolean checkForBarrier(Buffer buffer) throws IOException {
+			final AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+			if (priorityEvent instanceof CheckpointBarrier) {
+				pendingCheckpointBarrierId = BARRIER_RECEIVED;
+				return true;
+			}
+			return false;
+		}
+
+		/**
+		 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
+		 * returns null in all other cases.
+		 */
+		@Nullable
+		protected AbstractEvent parsePriorityEvent(Buffer buffer) throws IOException {
+			if (buffer.isBuffer() || !buffer.getDataType().hasPriority()) {
+				return null;
+			}
+
+			AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+			// reset the buffer because it would be deserialized again in SingleInputGate while getting next buffer.
+			// we can further improve to avoid double deserialization in the future.
+			buffer.setReaderIndex(0);
+			return event;
+		}

Review comment:
       It's only used in this class and thus I inlined it. As is, it really looks like an interface of the Persister.
   
   There is similar code for `BufferConsumer` on output side, but the implementation is too different to align (copy buffer vs. reader index reset, different `EventSerializer.fromBuffer` overloads).




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f0bb8a255816919a6578a6994499757075ab371b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f0bb8a255816919a6578a6994499757075ab371b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819) 
   * 67942d0ea538985ec25cecf7cefdc1bec0c1b53b 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -74,34 +106,34 @@ public CheckpointedInputGate(
 	}
 
 	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {
-			Optional<BufferOrEvent> next = inputGate.pollNext();
+	public Optional<BufferOrEvent> pollNext() throws IOException, InterruptedException {
+		Optional<BufferOrEvent> next = inputGate.pollNext();
 
-			if (!next.isPresent()) {
-				return handleEmptyBuffer();
-			}
+		if (!next.isPresent()) {
+			return handleEmptyBuffer();
+		}
 
-			BufferOrEvent bufferOrEvent = next.get();
-			checkState(!barrierHandler.isBlocked(bufferOrEvent.getChannelInfo()));
+		BufferOrEvent bufferOrEvent = next.get();
+		checkState(!barrierHandler.isBlocked(bufferOrEvent.getChannelInfo()));
 
-			if (bufferOrEvent.isBuffer()) {
-				return next;
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) {
-				CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent();
-				barrierHandler.processBarrier(checkpointBarrier, bufferOrEvent.getChannelInfo());
-				return next;
-			}
-			else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
-				barrierHandler.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent());
-			}
-			else {
-				if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) {
-					barrierHandler.processEndOfPartition();
-				}
-				return next;
-			}
+		if (bufferOrEvent.isEvent()) {
+			handleEvent(bufferOrEvent);
+		} else {
+			barrierHandler.processBuffer(bufferOrEvent.getBuffer(), bufferOrEvent.getChannelInfo());

Review comment:
       Yes, it's used in the next commit to persist in-flight data (replaces `notifyBufferReceived`).




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * fa3e495b6ba201769d908adea420e4944ddd7643 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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






----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   * cb585f45dd49a291daefda4b4eb847262b75856e UNKNOWN
   * 7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 4af565f8d262b2ed2ebfb7614e0f24494eb22186 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -770,34 +808,50 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {

Review comment:
       Well this change is less about expressing priority events and more about making sure that channels with priority events are always polled first. It's some kind of potential double notification, where the priority notification overrides the normal data available notification.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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






----------------------------------------------------------------
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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * b3a1520089c241fc74837902b6440d84a9636c14 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333) 
   * 16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * b3a1520089c241fc74837902b6440d84a9636c14 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333) 
   * 16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0 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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fce9f56a12b7c28a827c85669ce4bb0a8d31a48b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 755c3410f798bf7b3fffef91b7f2349b021613a9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -133,14 +136,14 @@ public boolean isAvailable() {
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	@Nullable
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!
-		if (numCreditsAvailable > 0) {
-			return bufferAndBacklog.isDataAvailable();
-		}
-		else {
-			return bufferAndBacklog.isEventAvailable();
+		final Buffer.DataType nextDataType = bufferAndBacklog.getNextDataType();
+		if (numCreditsAvailable > 0 || (nextDataType != null && nextDataType.isEvent())) {
+			return nextDataType;
 		}
+		return null;

Review comment:
       An enum type NONE would work for me and might make the code a bit clearer. However, be aware that this is mostly a copy&replace; I don't think it would simplify any code path.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837",
       "triggerID" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6844",
       "triggerID" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "36c366400126cb68f0a8c8cf14b5c5e98ef2b58d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "36c366400126cb68f0a8c8cf14b5c5e98ef2b58d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   * cb585f45dd49a291daefda4b4eb847262b75856e UNKNOWN
   * 7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837) 
   * 3333fd4ca67be7345172128322ff1cf17951808f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6844) 
   * 36c366400126cb68f0a8c8cf14b5c5e98ef2b58d 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -74,34 +106,34 @@ public CheckpointedInputGate(
 	}
 
 	@Override
-	public Optional<BufferOrEvent> pollNext() throws Exception {
-		while (true) {

Review comment:
       Yes, it's changing semantics (as written in commit message). I have not found a good reason why it's not always exited and it makes things easier especially since this method can now be used to process priority events.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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






----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * 834c28da93d4b24e562e017979b483e288db02a2 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -173,50 +181,51 @@ private boolean addBuffer(BufferConsumer bufferConsumer) {
 			buffers.add(bufferConsumer);
 			return false;
 		}
-		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-			"checkpoints");
 
 		final int pos = buffers.getNumPriorityElements();
 		buffers.addPriorityElement(bufferConsumer);
 
-		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
-		if (unalignedCheckpoint) {
+		CheckpointBarrier barrier = parseCheckpointBarrier(bufferConsumer);
+		if (barrier != null) {
+			checkState(
+				barrier.getCheckpointOptions().isUnalignedCheckpoint(),
+				"Only unaligned checkpoints should be priority events");
 			final Iterator<BufferConsumer> iterator = buffers.iterator();
 			Iterators.advance(iterator, pos + 1);
+			List<Buffer> inflightBuffers = new ArrayList<>();
 			while (iterator.hasNext()) {
 				BufferConsumer buffer = iterator.next();
 
 				if (buffer.isBuffer()) {
 					try (BufferConsumer bc = buffer.copy()) {
-						inflightBufferSnapshot.add(bc.build());
+						inflightBuffers.add(bc.build());
 					}
 				}
 			}
+			channelStateWriter.addOutputData(
+				barrier.getId(),
+				subpartitionInfo,
+				ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+				inflightBuffers.toArray(new Buffer[0]));

Review comment:
       Not relevant in the final version where the channel spills by itself (no return value on this method). I can make it clearer in the commit message if you want.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 755c3410f798bf7b3fffef91b7f2349b021613a9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301) 
   * 4af565f8d262b2ed2ebfb7614e0f24494eb22186 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778) 
   * 74094b3c0206de0bad49bdb7526534a8c43a5934 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805) 
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   
   <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] pnowojski commented on a change in pull request #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();
+		buffers.addPriorityElement(bufferConsumer);
+
+		boolean unalignedCheckpoint = isUnalignedCheckpoint(bufferConsumer);
+		if (unalignedCheckpoint) {
+			final Iterator<BufferConsumer> iterator = buffers.iterator();
+			Iterators.advance(iterator, pos + 1);
+			while (iterator.hasNext()) {
+				BufferConsumer buffer = iterator.next();
+
+				if (buffer.isBuffer()) {
+					try (BufferConsumer bc = buffer.copy()) {
 						inflightBufferSnapshot.add(bc.build());
 					}
 				}
 			}
+		}
+	}
 
-			buffers.addFirst(bufferConsumer);
-		} else {
-			buffers.add(bufferConsumer);
+	private boolean isUnalignedCheckpoint(BufferConsumer bufferConsumer) {
+		boolean unalignedCheckpoint;
+		try (BufferConsumer bc = bufferConsumer.copy()) {
+			Buffer buffer = bc.build();
+			try {
+				final AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+				unalignedCheckpoint = event instanceof CheckpointBarrier;
+			} catch (IOException e) {
+				throw new IllegalStateException("Should always be able to deserialize in-memory event", e);
+			} finally {
+				buffer.recycleBuffer();
+			}
 		}
+		return unalignedCheckpoint;

Review comment:
       this method is only `isCheckpointBarrier` and it seems to not care if it's aligned or not, right?
   
   Besides, do we really need to deserialise the event? Previously we were snapshotting in-flight data every time we were inserting buffer as a head. I think it was just as not elegant, but simpler.
   
   I guess this is currently a dead code, but would change if we ever want to have priority cancelation markers? If that's a sole motivation, I would revisit this problem in the future. Who knows if we will need this with checkpoint abort RPC. And if we will do, there is also another option:
   
   Inserting priority UC barrier, could go through a separate method , that would return overtaken in-flight data:
   ```
   Collection<Buffer> insertAsHeadAndGetInFlightData(checkpointBarrier)
   ```
   which would also eliminate the currently existing assumption/hack that `requestInflightBufferSnapshot` has to be always called immediately after inserting as a head. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.
-			for (BufferConsumer buffer : buffers) {
-				try (BufferConsumer bc = buffer.copy()) {
-					if (bc.isBuffer()) {
+		final int pos = buffers.getNumPriorityElements();

Review comment:
       nit: `pos` -> `numberOfPriorityEvents`? It would make the following line:
   ```
   Iterators.advance(iterator, pos + 1);
   ```
   more readable.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {
 		// BEWARE: this must be in sync with #isAvailable()!

Review comment:
       It's hard to say now, what does it mean it should be kept in sync, as they return very different things. On the other hand, this comment is valuable, so would be nice to keep it?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
##########
@@ -109,14 +113,14 @@ private void setupInputChannels() {
 					bufferBuilder.finish();
 
 					// Call getCurrentBuffer to ensure size is set
-					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(bufferConsumer.build(), nextType, 0));
 				} else if (input != null && input.isEvent()) {
 					AbstractEvent event = input.getEvent();
 					if (event instanceof EndOfPartitionEvent) {
 						inputChannels[channelIndex].setReleased();
 					}
 
-					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), moreAvailable, 0));
+					return Optional.of(new BufferAndAvailability(EventSerializer.toBuffer(event, false), nextType,	0));

Review comment:
       nit: whitespace?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       could you elaborate a bit more in the commit message, what has been simplified, why and what are the benefits?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;

Review comment:
       optional nit:
   extract `toNotify` and `toNotifyPriority` pair to some simple inner class `
   
   ```
   public static class DataNotification() {
     @Nullable
     CompletableFuture<?> toNotifyPriority = null;
     @Nullable
     CompletableFuture<?> toNotify = null;
   
     // two setters
     setXYZ(...);
   
     void complete() {
    		if (toNotifyPriority != null) {
   			toNotifyPriority.complete(null);
   		}
   		if (toNotify != null) {
   			toNotify.complete(null);
   		}
     }
   }
   ```
   and re-use in `UnionInputGate` as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java
##########
@@ -167,23 +171,46 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean inser
 
 	private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) {
 		assert Thread.holdsLock(buffers);
-		if (insertAsHead) {
-			checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
-				"checkpoints");
+		if (!insertAsHead) {
+			buffers.add(bufferConsumer);
+			return;
+		}
+		checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " +
+			"checkpoints");
 
-			// Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later.

Review comment:
       nit: maybe it's worth keeping this comment?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PrioritizedDeque.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * A deque-like data structure that supports prioritization of elements, such they will be polled before any
+ * non-priority elements.
+ *
+ * <p>{@implNote The current implementation deliberately does not implement the respective interface to minimize the maintenance
+ * effort. Furthermore, it's optimized for handling non-priority elements, such that all operations for adding priority
+ * elements are much slower than the non-priority counter-parts.}
+ *
+ * <p>Note that all element tests are performed by identity.
+ *
+ * @param <T> the element type.
+ */
+@Internal
+public final class PrioritizedDeque<T> implements Iterable<T> {
+	private final Deque<T> deque = new ArrayDeque<>();
+	private int numPriorityElements;
+
+	/**
+	 * Adds a priority element to this deque, such that it will be polled after all existing priority elements but
+	 * before any non-priority element.
+	 *
+	 * @param element the element to add
+	 */
+	public void addPriorityElement(T element) {
+		// priority elements are rather rare and short-lived, so most of there are none
+		if (numPriorityElements == 0) {
+			deque.addFirst(element);
+		} else if (numPriorityElements == deque.size()) {
+			// no non-priority elements
+			deque.add(element);
+		} else {
+			// remove all priority elements
+			final ArrayDeque<T> priorPriority = new ArrayDeque<>(numPriorityElements);
+			for (int index = 0; index < numPriorityElements; index++) {
+				priorPriority.addFirst(deque.poll());
+			}
+			deque.addFirst(element);
+			// readd them before the newly added element
+			for (final T priorityEvent : priorPriority) {
+				deque.addFirst(priorityEvent);
+			}
+		}
+		numPriorityElements++;
+	}
+
+	/**
+	 * Adds a non-priority element to this deque, which will be polled last.
+	 *
+	 * @param element the element to add
+	 */
+	public void add(T element) {
+		deque.add(element);
+	}
+
+	/**
+	 * Convenience method for adding an element with optional priority and prior removal.
+	 *
+	 * @param element the element to add
+	 * @param priority flag indicating if it's a priority or non-priority element
+	 * @param alreadyContained flag that hints that the element is already in this deque, potentially as non-priority element.
+	 */
+	public void add(T element, boolean priority, boolean alreadyContained) {

Review comment:
       is this method being used? I think at least not in this commit

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -779,34 +820,56 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) {
 			}));
 	}
 
-	private void queueChannel(InputChannel channel) {
-		int availableChannels;
+	private void queueChannel(InputChannel channel, boolean priority) {
 
+		CompletableFuture<?> toNotifyPriority = null;
 		CompletableFuture<?> toNotify = null;
 
 		synchronized (inputChannelsWithData) {
-			if (enqueuedInputChannelsWithData.get(channel.getChannelIndex())) {
+			// do not enqueue if the channel is currently polled because priority event could have been polled already
+			// let #waitAndGetNextData re-enqueue the channel correctly instead
+			if (priority && selectedChannel == channel) {
 				return;
 			}
-			availableChannels = inputChannelsWithData.size();
 
-			inputChannelsWithData.add(channel);
-			enqueuedInputChannelsWithData.set(channel.getChannelIndex());
+			if (!queueChannelUnsafe(channel, priority)) {
+				return;
+			}
 
-			if (availableChannels == 0) {
+			if (priority && inputChannelsWithData.getNumPriorityElements() == 1) {
+				toNotifyPriority = priorityAvailabilityHelper.getUnavailableToResetAvailable();
+			}
+			if (inputChannelsWithData.size() == 1) {
 				inputChannelsWithData.notifyAll();
 				toNotify = availabilityHelper.getUnavailableToResetAvailable();
 			}
 		}
 
+		if (toNotifyPriority != null) {
+			toNotifyPriority.complete(null);
+		}
 		if (toNotify != null) {
 			toNotify.complete(null);
 		}
 	}
 
+	private boolean queueChannelUnsafe(InputChannel channel, boolean priority) {

Review comment:
       add java doc what does it return?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       Is this a fix for an existing bug on the master branch? Or are you changing the behaviour/contract of this `unregisterSourceReader` call in some later commit? 
   
   Would it be worthwhile/easy adding a test?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null
 	 */
-	private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
+	private Buffer.DataType getNextDataType(BufferAndBacklog bufferAndBacklog) {

Review comment:
       nit: you've broken a comment reference (L125):
   ```
   #isAvailable(BufferAndBacklog)
   ```
   replace it with javadoc's `{@link}`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -290,6 +294,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
 	/**
 	 * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or
 	 * returns null in all other cases.
+	 * @return

Review comment:
       good to know that it returns something :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
##########
@@ -124,23 +127,22 @@ public boolean isAvailable() {
 	}
 
 	/**
-	 * Check whether this reader is available or not (internal use, in sync with
-	 * {@link #isAvailable()}, but slightly faster).
+	 * Returns the {@link org.apache.flink.runtime.io.network.buffer.Buffer.DataType} of the next buffer in line.
 	 *
-	 * <p>Returns true only if the next buffer is an event or the reader has both available
+	 * <p>Returns the next data type only if the next buffer is an event or the reader has both available
 	 * credits and buffers.
 	 *
 	 * @param bufferAndBacklog
 	 * 		current buffer and backlog including information about the next buffer
+	 * @return the next data type if the next buffer can be pulled immediately or null

Review comment:
       `or {@link DataType.NONE}`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -89,6 +92,14 @@
 	 */
 	private final int[] inputGateChannelIndexOffsets;
 
+	/**
+	 * The channel from which is currently polled, which allows interleaving of
+	 * {@link #queueInputGate(IndexedInputGate, boolean)} and {@link #pollNext()} (FLINK-12510 (Deadlock when reading from InputGates)).
+	 */
+	@GuardedBy("inputGatesWithData")
+	@Nullable
+	private IndexedInputGate currentInputGate;
+

Review comment:
       I haven't fully understood this part yet. Maybe let's sync later off-line?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java
##########
@@ -186,34 +203,47 @@ public boolean isFinished() {
 	private Optional<InputWithData<IndexedInputGate, BufferOrEvent>> waitAndGetNextData(boolean blocking)
 			throws IOException, InterruptedException {
 		while (true) {
-			Optional<IndexedInputGate> inputGate = getInputGate(blocking);
-			if (!inputGate.isPresent()) {
+			Optional<IndexedInputGate> inputGateOpt = getInputGate(blocking);
+			if (!inputGateOpt.isPresent()) {
 				return Optional.empty();
 			}
+			final IndexedInputGate inputGate = inputGateOpt.get();
 
 			// In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data.
 			// Do not poll the gate under inputGatesWithData lock, since this can trigger notifications
 			// that could deadlock because of wrong locks taking order.
-			Optional<BufferOrEvent> bufferOrEvent = inputGate.get().pollNext();
+			Optional<BufferOrEvent> nextOpt = inputGate.pollNext();
+			if (!nextOpt.isPresent()) {
+				inputGate.getAvailableFuture().thenRun(() -> queueInputGate(inputGate, false));
+				continue;
+			}
+			final BufferOrEvent bufferOrEvent = nextOpt.get();

Review comment:
       nit: extract everything below to:
   ```
   return Optional.of(processBufferOrEvent(nextOpt.get()));
   ```
   ?




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778) 
   * 74094b3c0206de0bad49bdb7526534a8c43a5934 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805) 
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java
##########
@@ -62,9 +65,37 @@
 	 */
 	public CheckpointedInputGate(
 			InputGate inputGate,
-			CheckpointBarrierHandler barrierHandler) {
+			CheckpointBarrierHandler barrierHandler,
+			MailboxExecutor mailboxExecutor) {
 		this.inputGate = inputGate;
 		this.barrierHandler = barrierHandler;
+		this.mailboxExecutor = mailboxExecutor;
+
+		waitForPriorityEvents(inputGate, mailboxExecutor);
+	}
+
+	/**
+	 * Eagerly pulls and processes all priority events. Must be called from task thread.
+	 *
+	 * <p>Basic assumption is that no priority event needs to be handled by the {@link StreamTaskNetworkInput}.
+	 */
+	private void processPriorityEvents() throws IOException, InterruptedException {
+		// check if the priority event is still not processed (could have been pulled before mail was being executed)
+		if (inputGate.getPriorityEventAvailableFuture().isDone()) {
+			// process as many priority events as possible
+			while (pollNext().map(BufferOrEvent::morePriorityEvents).orElse(false)) {
+			}
+		}
+

Review comment:
       Good idea, I solved it in the following way:
   
   ```
   		// check if the priority event is still not processed (could have been pulled before mail was being executed)
   		boolean hasPriorityEvent = inputGate.getPriorityEventAvailableFuture().isDone();
   		while (hasPriorityEvent) {
   			// process as many priority events as possible
   			final Optional<BufferOrEvent> bufferOrEventOpt = pollNext();
   			bufferOrEventOpt.ifPresent(bufferOrEvent ->
   				checkState(bufferOrEvent.hasPriority(), "Should only poll priority events"));
   			hasPriorityEvent = bufferOrEventOpt.map(BufferOrEvent::morePriorityEvents).orElse(false);
   		}
   ```
   
   `checkState(!inputGate.getPriorityEventAvailableFuture().isDone())` might be failing if netty receives a new priority event and triggers this available future while the task thread polled the last priority event. This case should happen quite often when the first barrier arrives (at that time the only priority event, morePriorityEvents = false) and triggers the whole checkpointing process. The second barrier would then complete the `getPriorityEventAvailableFuture` causing a more or less immediate re-execution of this 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 74094b3c0206de0bad49bdb7526534a8c43a5934 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805) 
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java
##########
@@ -120,67 +112,10 @@ public void testNoDataProcessedAfterCheckpointBarrier() throws Exception {
 		assertEquals(0, output.getNumberOfEmittedRecords());
 	}
 
-	@Test
-	public void testSnapshotWithTwoInputGates() throws Exception {

Review comment:
       I added this test for FLINK-18139 - input gate index issues in `Unaligner#hasInflightData`, which is called by `StreamTaskNetworkInput`, hence the test here.
   
   However, in this commit, `hasInflightData` is removed. Spilling is fully encapsulated in `InputChannel` + `CheckpointBarrierUnaligner`/`CheckpointedInputGate` and this interaction is already covered in tests at `CheckpointBarrierUnalignerTest`. 




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724) 
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
##########
@@ -159,10 +158,12 @@ public InputStatus emitNext(DataOutput<T> output) throws Exception {
 			if (bufferOrEvent.isPresent()) {
 				// return to the mailbox after receiving a checkpoint barrier to avoid processing of
 				// data after the barrier before checkpoint is performed for unaligned checkpoint mode
-				if (bufferOrEvent.get().isEvent() && bufferOrEvent.get().getEvent() instanceof CheckpointBarrier) {
+				if (bufferOrEvent.get().isBuffer()) {
+					processBuffer(bufferOrEvent.get());
+				} else {
+					processEvent(bufferOrEvent.get());

Review comment:
       Sorry I haven't pulled that change out, since it's more than a refactoring; it's the core change that modifies the semantics to always return on events. 
   Except for that change, everything else is more or less directly related to passing the priority future, so I'm not sure what to pull out.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
##########
@@ -210,15 +221,25 @@ public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelSt
 		}
 
 		Buffer buffer = next.buffer();
-		CheckpointBarrier notifyReceivedBarrier = parseCheckpointBarrierOrNull(buffer);
-		if (notifyReceivedBarrier != null) {
-			receivedCheckpointId = notifyReceivedBarrier.getId();
-		} else if (receivedCheckpointId < lastRequestedCheckpointId && buffer.isBuffer()) {
-			inputGate.getBufferReceivedListener().notifyBufferReceived(buffer.retainBuffer(), channelInfo);
-		}
 
 		numBytesIn.inc(buffer.getSize());
 		numBuffersIn.inc();
+		if (buffer.isBuffer()) {
+			for (final long barrierId : pendingCheckpointBarriers) {
+				channelStateWriter.addInputData(
+					barrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));

Review comment:
       `LocalInputChannel` only spills when it awaits barrier. So it spills the buffer on first sight and it cannot be better on downsteam level. We could of course also move spilling lingering buffers to the upstream. It might also be an improvement for later, but it adds quite a bit of complexity as barriers also need to be propagated upstream.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierUnaligner.java
##########
@@ -92,306 +90,173 @@
 		super(toNotifyOnCheckpoint);
 
 		this.taskName = taskName;
-		hasInflightBuffers = Arrays.stream(inputGates)
+		this.inputGates = inputGates;
+		storeNewBuffers = Arrays.stream(inputGates)
 			.flatMap(gate -> gate.getChannelInfos().stream())
 			.collect(Collectors.toMap(Function.identity(), info -> false));
-		threadSafeUnaligner = new ThreadSafeUnaligner(checkNotNull(checkpointCoordinator), this, inputGates);
+		numOpenChannels = storeNewBuffers.size();
+		this.checkpointCoordinator = checkpointCoordinator;
 	}
 
-	/**
-	 * We still need to trigger checkpoint via {@link ThreadSafeUnaligner#notifyBarrierReceived(CheckpointBarrier, InputChannelInfo)}
-	 * while reading the first barrier from one channel, because this might happen
-	 * earlier than the previous async trigger via mailbox by netty thread.
-	 *
-	 * <p>Note this is also suitable for the trigger case of local input channel.
-	 */
 	@Override
-	public void processBarrier(CheckpointBarrier receivedBarrier, InputChannelInfo channelInfo) throws IOException {
-		long barrierId = receivedBarrier.getId();
-		if (currentConsumedCheckpointId > barrierId || (currentConsumedCheckpointId == barrierId && !isCheckpointPending())) {
+	public void processBarrier(CheckpointBarrier barrier, InputChannelInfo channelInfo) throws IOException {
+		long barrierId = barrier.getId();
+		if (currentCheckpointId > barrierId || (currentCheckpointId == barrierId && !isCheckpointPending())) {
 			// ignore old and cancelled barriers
 			return;
 		}
-		if (currentConsumedCheckpointId < barrierId) {
-			currentConsumedCheckpointId = barrierId;
-			numBarrierConsumed = 0;
-			hasInflightBuffers.entrySet().forEach(hasInflightBuffer -> hasInflightBuffer.setValue(true));
+		if (currentCheckpointId < barrierId) {
+			handleNewCheckpoint(barrier);
+			notifyCheckpoint(barrier, 0);
 		}
-		if (currentConsumedCheckpointId == barrierId) {
-			hasInflightBuffers.put(channelInfo, false);
-			numBarrierConsumed++;
+		if (currentCheckpointId == barrierId) {
+			if (storeNewBuffers.put(channelInfo, false)) {
+				LOG.debug("{}: Received barrier from channel {} @ {}.", taskName, channelInfo, barrierId);
+
+				inputGates[channelInfo.getGateIdx()].getChannel(channelInfo.getInputChannelIdx())
+					.spillInflightBuffers(barrierId, checkpointCoordinator.getChannelStateWriter());
+
+				if (++numBarriersReceived == numOpenChannels) {
+					allBarriersReceivedFuture.complete(null);
+				}
+			}
 		}
-		threadSafeUnaligner.notifyBarrierReceived(receivedBarrier, channelInfo);
 	}
 
 	@Override
 	public void abortPendingCheckpoint(long checkpointId, CheckpointException exception) throws IOException {
-		threadSafeUnaligner.tryAbortPendingCheckpoint(checkpointId, exception);
+		tryAbortPendingCheckpoint(checkpointId, exception);
 
-		if (checkpointId > currentConsumedCheckpointId) {
-			resetPendingCheckpoint(checkpointId);
+		if (checkpointId > currentCheckpointId) {
+			resetPendingCheckpoint();
 		}
 	}
 
 	@Override
 	public void processCancellationBarrier(CancelCheckpointMarker cancelBarrier) throws IOException {
 		final long cancelledId = cancelBarrier.getCheckpointId();
-		boolean shouldAbort = threadSafeUnaligner.setCancelledCheckpointId(cancelledId);
+		boolean shouldAbort = setCancelledCheckpointId(cancelledId);
 		if (shouldAbort) {
 			notifyAbort(
 				cancelledId,
 				new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_ON_CANCELLATION_BARRIER));
 		}
 
-		if (cancelledId >= currentConsumedCheckpointId) {
-			resetPendingCheckpoint(cancelledId);
-			currentConsumedCheckpointId = cancelledId;
+		if (cancelledId >= currentCheckpointId) {
+			resetPendingCheckpoint();
+			currentCheckpointId = cancelledId;
 		}
 	}
 
 	@Override
 	public void processEndOfPartition() throws IOException {
-		threadSafeUnaligner.onChannelClosed();
-		resetPendingCheckpoint(-1L);
+		numOpenChannels--;
+
+		resetPendingCheckpoint();
+		notifyAbort(
+			currentCheckpointId,
+			new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED_INPUT_END_OF_STREAM));
 	}
 
-	private void resetPendingCheckpoint(long checkpointId) {
-		if (isCheckpointPending()) {
-			LOG.warn("{}: Received barrier or EndOfPartition(-1) {} before completing current checkpoint {}. " +
-					"Skipping current checkpoint.",
-				taskName,
-				checkpointId,
-				currentConsumedCheckpointId);
+	private void resetPendingCheckpoint() {
+		LOG.warn("{}: Received barrier or EndOfPartition(-1) before completing current checkpoint {}. " +
+				"Skipping current checkpoint.",
+			taskName,
+			currentCheckpointId);
 
-			hasInflightBuffers.entrySet().forEach(hasInflightBuffer -> hasInflightBuffer.setValue(false));
-			numBarrierConsumed = 0;
-		}
+		storeNewBuffers.entrySet().forEach(storeNewBuffer -> storeNewBuffer.setValue(false));
+		numBarriersReceived = 0;
 	}
 
 	@Override
 	public long getLatestCheckpointId() {
-		return currentConsumedCheckpointId;
+		return currentCheckpointId;
 	}
 
 	@Override
 	public String toString() {
-		return String.format("%s: last checkpoint: %d", taskName, currentConsumedCheckpointId);
+		return String.format("%s: last checkpoint: %d", taskName, currentCheckpointId);
 	}
 
 	@Override
 	public void close() throws IOException {
 		super.close();
-		threadSafeUnaligner.close();
-	}
-
-	@Override
-	public boolean hasInflightData(long checkpointId, InputChannelInfo channelInfo) {
-		if (checkpointId < currentConsumedCheckpointId) {
-			return false;
-		}
-		if (checkpointId > currentConsumedCheckpointId) {
-			return true;
-		}
-		return hasInflightBuffers.get(channelInfo);
-	}
-
-	@Override
-	public CompletableFuture<Void> getAllBarriersReceivedFuture(long checkpointId) {
-		return threadSafeUnaligner.getAllBarriersReceivedFuture(checkpointId);
-	}
-
-	@Override
-	public Optional<BufferReceivedListener> getBufferReceivedListener() {
-		return Optional.of(threadSafeUnaligner);
+		allBarriersReceivedFuture.cancel(false);
 	}
 
 	@Override
 	protected boolean isCheckpointPending() {
-		return numBarrierConsumed > 0;
-	}
-
-	@VisibleForTesting
-	int getNumOpenChannels() {
-		return threadSafeUnaligner.getNumOpenChannels();
-	}
-
-	@VisibleForTesting
-	ThreadSafeUnaligner getThreadSafeUnaligner() {
-		return threadSafeUnaligner;
+		return numBarriersReceived > 0;
 	}
 
-	private void notifyCheckpoint(CheckpointBarrier barrier) throws IOException {
-		// ignore the previous triggered checkpoint by netty thread if it was already canceled or aborted before.
-		if (barrier.getId() >= threadSafeUnaligner.getCurrentCheckpointId()) {
-			super.notifyCheckpoint(barrier, 0);
+	@Override
+	public void processBuffer(Buffer buffer, InputChannelInfo channelInfo) {
+		if (storeNewBuffers.get(channelInfo)) {
+			checkpointCoordinator.getChannelStateWriter().addInputData(
+				currentCheckpointId,
+				channelInfo,
+				ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+				ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));

Review comment:
       Let me sketch the most general data flow:
   * We have couple of buffers in our input channel.
   * Checkpoint is triggered by another channel. *
   * All buffers that are now pulled by `CheckpointedInputGate` from the first channel get persisted by above function.
   * Then the barrier comes in. *
   * It overtakes all buffers and is now at the head. *
   * `CheckpointedInputGate` gets priority notification and polls the barrier.
   * Upon dispatching, it calls `Unaligner`, which spills additionally all overtaken buffers.
   * Further buffers are not persisted.
   
   All steps marked with * are performed in a different thread (other task thread / netty).
   
   For me this is equivalent to the current behavior, but maybe I missed something.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * a22657562a42dda270f89c176e5c260ee73c6698 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693) 
   * f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837",
       "triggerID" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6844",
       "triggerID" : "3333fd4ca67be7345172128322ff1cf17951808f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "36c366400126cb68f0a8c8cf14b5c5e98ef2b58d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6848",
       "triggerID" : "36c366400126cb68f0a8c8cf14b5c5e98ef2b58d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   * cb585f45dd49a291daefda4b4eb847262b75856e UNKNOWN
   * 7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837) 
   * 3333fd4ca67be7345172128322ff1cf17951808f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6844) 
   * 36c366400126cb68f0a8c8cf14b5c5e98ef2b58d Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6848) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f1bfd2b98e8665739bd07ae828d2ca73e2e927fc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 1a86b34a3cd22d0ae34d16e9fa96036650dd6323 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765) 
   * 3c429de7c86d271a8980a34759026e18d3cf24b8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775) 
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java
##########
@@ -34,11 +30,8 @@
 	void notifyDataAvailable();
 
 	/**
-	 * Allows the listener to react to a priority event before it is added to the outgoing buffer queue.
-	 *
-	 * @return true if the event has been fully processed and should not be added to the buffer queue.
+	 * Called when the first priority event is added to the head of the buffer queue.
 	 */
-	default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException {
-		return false;
+	default void notifyPriorityEvent() {

Review comment:
       Added a few thoughts. Let me know if it makes things clearer.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6805",
       "triggerID" : "74094b3c0206de0bad49bdb7526534a8c43a5934",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806",
       "triggerID" : "ef87978fc0d3da0134ccf57eb144aeb456a5d345",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb585f45dd49a291daefda4b4eb847262b75856e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837",
       "triggerID" : "7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * ef87978fc0d3da0134ccf57eb144aeb456a5d345 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6806) 
   * cb585f45dd49a291daefda4b4eb847262b75856e UNKNOWN
   * 7ed17f2f8d8b67b72b0d453a006e7d26bfa54eaa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6837) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -89,17 +96,15 @@
 	/** The number of available buffers that have not been announced to the producer yet. */
 	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
 
-	/**
-	 * The latest already triggered checkpoint id which would be updated during
-	 * {@link #spillInflightBuffers(long, ChannelStateWriter)}.
-	 */
-	@GuardedBy("receivedBuffers")
-	private long lastRequestedCheckpointId = -1;
+	private final BufferManager bufferManager;
 
-	/** The current received checkpoint id from the network. */
-	private long receivedCheckpointId = -1;
+	/** Stores #overtaken buffers when a checkpoint barrier is received before task thread started checkpoint. */
+	@GuardedBy("receivedBuffers")
+	private Map<Long, Integer> numBuffersOvertaken = new HashMap<>();
 
-	private final BufferManager bufferManager;
+	/** All started checkpoints where a barrier has not been received yet. */
+	@GuardedBy("receivedBuffers")
+	private Deque<Long> pendingCheckpointBarriers = new ArrayDeque<>(2);

Review comment:
       Changed it to support only one concurrent checkpoint. I also extracted the whole logic into one helper class that can be used by both `LocalInputChannel` and `RemoteInputChannel` (with syncrhonization).




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740) 
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 1a86b34a3cd22d0ae34d16e9fa96036650dd6323 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765) 
   * 3c429de7c86d271a8980a34759026e18d3cf24b8 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6740",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ae5ba61e30645172fd0dc7e6574a060e437fd3ba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6765",
       "triggerID" : "1a86b34a3cd22d0ae34d16e9fa96036650dd6323",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775",
       "triggerID" : "3c429de7c86d271a8980a34759026e18d3cf24b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9f9b09068f221ca39956bd6a94a986e3ac594b94",
       "triggerType" : "PUSH"
     }, {
       "hash" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778",
       "triggerID" : "355fecafd9215ef93471360f84be51e0520ce1bc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * ae5ba61e30645172fd0dc7e6574a060e437fd3ba UNKNOWN
   * 3c429de7c86d271a8980a34759026e18d3cf24b8 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6775) 
   * 9f9b09068f221ca39956bd6a94a986e3ac594b94 UNKNOWN
   * 355fecafd9215ef93471360f84be51e0520ce1bc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6778) 
   
   <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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -454,42 +431,106 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 				}
 
 				wasEmpty = receivedBuffers.isEmpty();
-				receivedBuffers.add(buffer);
 
-				if (listener != null && buffer.isBuffer() && receivedCheckpointId < lastRequestedCheckpointId) {
-					notifyReceivedBuffer = buffer.retainBuffer();
+				AbstractEvent priorityEvent = parsePriorityEvent(buffer);
+				if (priorityEvent != null) {
+					receivedBuffers.addPriorityElement(buffer);
+					final int pos = receivedBuffers.getNumPriorityElements();
+					if (priorityEvent instanceof CheckpointBarrier) {
+						final long barrierId = ((CheckpointBarrier) priorityEvent).getId();
+						// don't spill future buffers for this checkpoint
+						if (!pendingCheckpointBarriers.remove(barrierId)) {
+							// checkpoint was not yet started by task thread,
+							// so remember the numbers of buffers to spill for the time when it will be started
+							numBuffersOvertaken.put(barrierId, receivedBuffers.size() - pos);
+						}
+					}
+					firstPriorityEvent = pos == 1;
 				} else {
-					notifyReceivedBuffer = null;
+					receivedBuffers.add(buffer);
+					if (buffer.isBuffer()) {
+						for (final long checkpointId : pendingCheckpointBarriers) {
+							channelStateWriter.addInputData(
+								checkpointId,
+								channelInfo,
+								sequenceNumber,
+								CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));
+						}
+					}
 				}
-				notifyReceivedBarrier = listener != null ? parseCheckpointBarrierOrNull(buffer) : null;
 			}
 			recycleBuffer = false;
 
 			++expectedSequenceNumber;
 
+			if (firstPriorityEvent) {
+				notifyPriorityEvent();
+			}
 			if (wasEmpty) {
 				notifyChannelNonEmpty();
 			}
 
 			if (backlog >= 0) {
 				onSenderBacklog(backlog);
 			}
-
-			if (notifyReceivedBarrier != null) {
-				receivedCheckpointId = notifyReceivedBarrier.getId();
-				if (notifyReceivedBarrier.isCheckpoint()) {
-					listener.notifyBarrierReceived(notifyReceivedBarrier, channelInfo);
-				}
-			} else if (notifyReceivedBuffer != null) {
-				listener.notifyBufferReceived(notifyReceivedBuffer, channelInfo);
-			}
 		} finally {
 			if (recycleBuffer) {
 				buffer.recycleBuffer();
 			}
 		}
 	}
 
+	/**
+	 * Spills all queued buffers on checkpoint start. If barrier has already been received (and reordered), spill only
+	 * the overtaken buffers.
+	 */
+	public void checkpointStarted(CheckpointBarrier barrier) {
+		checkState(channelStateWriter != null, "Channel state writer not injected");
+		synchronized (receivedBuffers) {
+			final Integer numBuffers = numBuffersOvertaken.get(barrier.getId());
+			if (numBuffers != null) {
+				// already received barrier before the task thread picked up the barrier of this or another channel
+				spillBuffers(barrier.getId(), numBuffers);
+			} else {
+				// barrier not yet received, spill all current and future buffers
+				spillBuffers(barrier.getId(), receivedBuffers.getNumUnprioritizedElements());
+				pendingCheckpointBarriers.add(barrier.getId());
+			}
+		}
+	}
+
+	public void checkpointStopped(long checkpointId) {
+		synchronized (receivedBuffers) {
+			numBuffersOvertaken.remove(checkpointId);
+			pendingCheckpointBarriers.remove(checkpointId);
+		}
+	}
+
+	private void spillBuffers(long checkpointId, int numBuffers) {

Review comment:
       I added a helper class `ChannelStatePersister` and used `persist` everywhere to have a clear separation of the two names. `Persister` is more on the logical level and `Writer` on the physical 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -261,8 +260,7 @@ void registerSourceReader(ReaderInfo readerInfo) {
 	 * @param subtaskId the subtask id of the source reader.
 	 */
 	void unregisterSourceReader(int subtaskId) {
-		Preconditions.checkNotNull(registeredReaders.remove(subtaskId), String.format(
-				"Failed to unregister source reader of id %s because it is not registered.", subtaskId));
+		registeredReaders.remove(subtaskId);

Review comment:
       It's a bug that I discovered in my UC ITCase and confirmed with Becket and Stephan. I added a small (trivial) test case.




----------------------------------------------------------------
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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f0bb8a255816919a6578a6994499757075ab371b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5819",
       "triggerID" : "f0bb8a255816919a6578a6994499757075ab371b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5833",
       "triggerID" : "67942d0ea538985ec25cecf7cefdc1bec0c1b53b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6225",
       "triggerID" : "273cc6bf445b4a44daca0e98d49aa0745a36a534",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6229",
       "triggerID" : "f1bfd2b98e8665739bd07ae828d2ca73e2e927fc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6290",
       "triggerID" : "fce9f56a12b7c28a827c85669ce4bb0a8d31a48b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d141f10ef060063162dc073b1cd66729f5f75a3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6297",
       "triggerID" : "ef2c7ecea91a30e4d721efa759ca1b15728a24cb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6301",
       "triggerID" : "755c3410f798bf7b3fffef91b7f2349b021613a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6311",
       "triggerID" : "4af565f8d262b2ed2ebfb7614e0f24494eb22186",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6333",
       "triggerID" : "b3a1520089c241fc74837902b6440d84a9636c14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6349",
       "triggerID" : "16c32eb1e2d49fa3c84cd4a82380fd72d5dcf5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6351",
       "triggerID" : "11cb1939f8a98340acab9b795c6f1894808fb606",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6365",
       "triggerID" : "19c4f0d1d1710b829946a78cb58aff768baab684",
       "triggerType" : "PUSH"
     }, {
       "hash" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6376",
       "triggerID" : "40c76a4ce74a5bbc800cf9833b96bb0156cacc41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6402",
       "triggerID" : "407169bc192b5d05bfba2c264849f1a6e7b81415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "89b0550c7b223f304b67a6f64dfc4252509eb76e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6446",
       "triggerID" : "a6a2d36afcce1758e885f0751359ccf035ae2378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6459",
       "triggerID" : "fa3e495b6ba201769d908adea420e4944ddd7643",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6539",
       "triggerID" : "a1fb1c3114305ed5fce898dfcc03abed4d6963d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "834c28da93d4b24e562e017979b483e288db02a2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6570",
       "triggerID" : "834c28da93d4b24e562e017979b483e288db02a2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6693",
       "triggerID" : "a22657562a42dda270f89c176e5c260ee73c6698",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724",
       "triggerID" : "f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d141f10ef060063162dc073b1cd66729f5f75a3b UNKNOWN
   * 89b0550c7b223f304b67a6f64dfc4252509eb76e UNKNOWN
   * f23e2a14cbbbf4c7ced3d66eea8073eb1d8c4acc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6724) 
   * 8f18616dbfce97d0fd6e1f442f7a3db892a6cf1e 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 #13228: [FLINK-19026][network] Improve threading model of CheckpointBarrierUnaligner

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java
##########
@@ -210,15 +221,25 @@ public void spillInflightBuffers(long checkpointId, ChannelStateWriter channelSt
 		}
 
 		Buffer buffer = next.buffer();
-		CheckpointBarrier notifyReceivedBarrier = parseCheckpointBarrierOrNull(buffer);
-		if (notifyReceivedBarrier != null) {
-			receivedCheckpointId = notifyReceivedBarrier.getId();
-		} else if (receivedCheckpointId < lastRequestedCheckpointId && buffer.isBuffer()) {
-			inputGate.getBufferReceivedListener().notifyBufferReceived(buffer.retainBuffer(), channelInfo);
-		}
 
 		numBytesIn.inc(buffer.getSize());
 		numBuffersIn.inc();
+		if (buffer.isBuffer()) {
+			for (final long barrierId : pendingCheckpointBarriers) {
+				channelStateWriter.addInputData(
+					barrierId,
+					getChannelInfo(),
+					ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN,
+					CloseableIterator.ofElement(buffer.retainBuffer(), Buffer::recycleBuffer));

Review comment:
       `LocalInputChannel` only spills when it awaits barrier. So it spills the buffer on first sight and it cannot be better on downsteam level. We could of course also move spilling lingering buffers to the upstream. It might also be an improvement for later.




----------------------------------------------------------------
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