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

[GitHub] [flink] pnowojski opened a new pull request #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

pnowojski opened a new pull request #13741:
URL: https://github.com/apache/flink/pull/13741


   This PR adds `CheckpointBarrierAnnouncement` messages 
   
   ## 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] AHeise commented on a change in pull request #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -437,19 +440,20 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 
 				wasEmpty = receivedBuffers.isEmpty();
 
-				if (buffer.getDataType().hasPriority()) {
-					receivedBuffers.addPriorityElement(new SequenceBuffer(buffer, sequenceNumber));
-					if (channelStatePersister.checkForBarrier(buffer)) {
-						// 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 = receivedBuffers.getNumUnprioritizedElements();
-					}
-					firstPriorityEvent = receivedBuffers.getNumPriorityElements() == 1;
+				SequenceBuffer sequenceBuffer = new SequenceBuffer(buffer, sequenceNumber);
+				DataType dataType = buffer.getDataType();
+				if (dataType.hasPriority()) {
+					checkPriorityXorAnnouncement(buffer);
+					firstPriorityEvent = addPriorityBuffer(sequenceBuffer);
 				} else {
-					receivedBuffers.add(new SequenceBuffer(buffer, sequenceNumber));
+					receivedBuffers.add(sequenceBuffer);
 					channelStatePersister.maybePersist(buffer);
-				}
 
+					if (dataType.requiresAnnouncement()) {
+						checkPriorityXorAnnouncement(buffer);
+						firstPriorityEvent = addPriorityBuffer(announce(sequenceBuffer));
+					}

Review comment:
       Missed the `announce` part.




----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 33b05f3143953a3bcb7a526326034cfd3bc6da5b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 152532018780b987ea2447abd20097375457655f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113) 
   * 152532018780b987ea2447abd20097375457655f 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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 827c2f19269d5ef5ff2ff6fb214b30dd12955b01 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java
##########
@@ -47,29 +50,56 @@
 
 	private final boolean isUnalignedCheckpoint;
 
+	private final long alignmentTimeout;
+
+	public static CheckpointOptions create(
+			CheckpointType checkpointType,
+			CheckpointStorageLocationReference locationReference,
+			boolean isExactlyOnceMode,
+			boolean unalignedCheckpointsEnabled,
+			long alignmentTimeout) {
+		boolean canBeUnaligned = checkpointType == CheckpointType.CHECKPOINT && unalignedCheckpointsEnabled;
+		return new CheckpointOptions(
+				checkpointType,
+				locationReference,
+				isExactlyOnceMode,
+				canBeUnaligned && alignmentTimeout == 0,
+				canBeUnaligned ? alignmentTimeout : NO_ALIGNMENT_TIME_OUT);
+	}
+
 	@VisibleForTesting
 	public CheckpointOptions(
 			CheckpointType checkpointType,
 			CheckpointStorageLocationReference targetLocation) {
-		this(checkpointType, targetLocation, true, false);
+		this(checkpointType, targetLocation, true, false, NO_ALIGNMENT_TIME_OUT);
 	}
 
 	public CheckpointOptions(
 			CheckpointType checkpointType,
 			CheckpointStorageLocationReference targetLocation,
 			boolean isExactlyOnceMode,
-			boolean isUnalignedCheckpoint) {
+			boolean isUnalignedCheckpoint,
+			long alignmentTimeout) {
 
 		this.checkpointType = checkNotNull(checkpointType);
 		this.targetLocation = checkNotNull(targetLocation);
 		this.isExactlyOnceMode = isExactlyOnceMode;
 		this.isUnalignedCheckpoint = isUnalignedCheckpoint;
+		this.alignmentTimeout = alignmentTimeout;
 	}
 
 	public boolean needsAlignment() {
 		return isExactlyOnceMode() && (getCheckpointType().isSavepoint() || !isUnalignedCheckpoint());
 	}
 
+	public long getAlignmentTimeout() {
+		return alignmentTimeout;
+	}
+
+	public boolean isTimeoutable() {
+		return alignmentTimeout > 0 && alignmentTimeout != NO_ALIGNMENT_TIME_OUT;

Review comment:
       Initially I had this, but `Long.MAX_VALUE` works better with `getAlignmentTimeout()`, as `Long.MAX_VALUE` is for all of the practical purposes `NO_ALIGNMENT_TIME_OUT` :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierAnnouncement.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.api;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.event.RuntimeEvent;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * {@link CheckpointBarrierAnnouncement} is announcing presence or receiving of a {@link CheckpointBarrier}.
+ * That {@link #announcedBarrier} is identified by it's sequence number.
+ */
+public class CheckpointBarrierAnnouncement extends RuntimeEvent {

Review comment:
       At the same time, I'm not sure if we will ever need announcement of other events and it would either:
   - require `checkState` very early on, and still complicate the code a little bit
   - add `checkState` later and complicate the code even more




----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java
##########
@@ -47,29 +50,56 @@
 
 	private final boolean isUnalignedCheckpoint;
 
+	private final long alignmentTimeout;

Review comment:
       No, `alignmentTimeout` is a global timeout, measured against the `CheckpointBarrier#getTimestamp`, so since the creation of the `CheckpointBarrier` on the `CheckpointCoordinator`. If `CheckpointCoordinator` has timed out on an upstream task, it will be timed out for any downstream operator as well.




----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AlternatingController.java
##########
@@ -35,7 +35,6 @@
 	private final AlignedController alignedController;
 	private final UnalignedController unalignedController;
 	private  CheckpointBarrierBehaviourController activeController;
-	private long lastSeenBarrierId;

Review comment:
       Typo is commit msg.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AlternatingController.java
##########
@@ -35,7 +35,6 @@
 	private final AlignedController alignedController;
 	private final UnalignedController unalignedController;
 	private  CheckpointBarrierBehaviourController activeController;

Review comment:
       Can you fix the double-spacing here as well?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java
##########
@@ -47,29 +50,56 @@
 
 	private final boolean isUnalignedCheckpoint;
 
+	private final long alignmentTimeout;

Review comment:
       Could you motivate (in commit msg) why this is a property of the checkpoint vs. a config of the task?
   
   I don't mind this solution but I want to understand the reason. Intuitively, I'd put it into the task config, which would ultimately allow a fine-grain checkpoint configuration for each operator.
   
   Do you plan to adjust the `alignmentTimeout` when the barrier travels downstream?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java
##########
@@ -143,4 +143,20 @@
 					TextElement.code(CheckpointingMode.EXACTLY_ONCE.toString()),
 					TextElement.code(MAX_CONCURRENT_CHECKPOINTS.key()))
 				.build());
+
+	public static final ConfigOption<Duration> ALIGNMENT_TIMEOUT =
+		ConfigOptions.key("execution.checkpointing.alignment-timeout")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription(Description.builder()
+				.text("Only relevant if %s is enabled.", TextElement.code(ENABLE_UNALIGNED.key()))
+				.linebreak()
+				.linebreak()
+				.text("If timeout has value equal to 0, checkpoints will always start unaligned.")

Review comment:
       Quite technical compared to the other descriptions (=too precise). 
   > "If timeout is (set to) 0, checkpoints will always start unaligned."

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java
##########
@@ -47,29 +50,56 @@
 
 	private final boolean isUnalignedCheckpoint;
 
+	private final long alignmentTimeout;
+
+	public static CheckpointOptions create(
+			CheckpointType checkpointType,
+			CheckpointStorageLocationReference locationReference,
+			boolean isExactlyOnceMode,
+			boolean unalignedCheckpointsEnabled,
+			long alignmentTimeout) {
+		boolean canBeUnaligned = checkpointType == CheckpointType.CHECKPOINT && unalignedCheckpointsEnabled;
+		return new CheckpointOptions(
+				checkpointType,
+				locationReference,
+				isExactlyOnceMode,
+				canBeUnaligned && alignmentTimeout == 0,
+				canBeUnaligned ? alignmentTimeout : NO_ALIGNMENT_TIME_OUT);
+	}
+
 	@VisibleForTesting
 	public CheckpointOptions(
 			CheckpointType checkpointType,
 			CheckpointStorageLocationReference targetLocation) {
-		this(checkpointType, targetLocation, true, false);
+		this(checkpointType, targetLocation, true, false, NO_ALIGNMENT_TIME_OUT);
 	}
 
 	public CheckpointOptions(
 			CheckpointType checkpointType,
 			CheckpointStorageLocationReference targetLocation,
 			boolean isExactlyOnceMode,
-			boolean isUnalignedCheckpoint) {
+			boolean isUnalignedCheckpoint,
+			long alignmentTimeout) {
 
 		this.checkpointType = checkNotNull(checkpointType);
 		this.targetLocation = checkNotNull(targetLocation);
 		this.isExactlyOnceMode = isExactlyOnceMode;
 		this.isUnalignedCheckpoint = isUnalignedCheckpoint;
+		this.alignmentTimeout = alignmentTimeout;
 	}
 
 	public boolean needsAlignment() {
 		return isExactlyOnceMode() && (getCheckpointType().isSavepoint() || !isUnalignedCheckpoint());
 	}
 
+	public long getAlignmentTimeout() {
+		return alignmentTimeout;
+	}
+
+	public boolean isTimeoutable() {
+		return alignmentTimeout > 0 && alignmentTimeout != NO_ALIGNMENT_TIME_OUT;

Review comment:
       Have `NO_ALIGNMENT_TIME_OUT = -1`?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java
##########
@@ -143,4 +143,20 @@
 					TextElement.code(CheckpointingMode.EXACTLY_ONCE.toString()),
 					TextElement.code(MAX_CONCURRENT_CHECKPOINTS.key()))
 				.build());
+
+	public static final ConfigOption<Duration> ALIGNMENT_TIMEOUT =
+		ConfigOptions.key("execution.checkpointing.alignment-timeout")
+			.durationType()
+			.defaultValue(Duration.ofSeconds(30))
+			.withDescription(Description.builder()
+				.text("Only relevant if %s is enabled.", TextElement.code(ENABLE_UNALIGNED.key()))
+				.linebreak()
+				.linebreak()
+				.text("If timeout has value equal to 0, checkpoints will always start unaligned.")
+				.linebreak()
+				.linebreak()
+				.text("If time has value greater then 0, checkpoints will start aligned. " +

Review comment:
       > "If timeout is (set to) non-zero/positive value, checkpoints will..."

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -773,11 +776,12 @@ private void snapshotTaskState(
 		Execution[] executions,
 		boolean advanceToEndOfTime) {
 
-		final CheckpointOptions checkpointOptions = new CheckpointOptions(
-			props.getCheckpointType(),
-			checkpointStorageLocation.getLocationReference(),
-			isExactlyOnceMode,
-			props.getCheckpointType() == CheckpointType.CHECKPOINT && unalignedCheckpointsEnabled);
+		final CheckpointOptions checkpointOptions = CheckpointOptions.create(
+				props.getCheckpointType(),

Review comment:
       nit: double indent. Also in various other places in this commit. Ignore if intended.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -437,19 +440,20 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 
 				wasEmpty = receivedBuffers.isEmpty();
 
-				if (buffer.getDataType().hasPriority()) {
-					receivedBuffers.addPriorityElement(new SequenceBuffer(buffer, sequenceNumber));
-					if (channelStatePersister.checkForBarrier(buffer)) {
-						// 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 = receivedBuffers.getNumUnprioritizedElements();
-					}
-					firstPriorityEvent = receivedBuffers.getNumPriorityElements() == 1;
+				SequenceBuffer sequenceBuffer = new SequenceBuffer(buffer, sequenceNumber);
+				DataType dataType = buffer.getDataType();
+				if (dataType.hasPriority()) {
+					checkPriorityXorAnnouncement(buffer);
+					firstPriorityEvent = addPriorityBuffer(sequenceBuffer);
 				} else {
-					receivedBuffers.add(new SequenceBuffer(buffer, sequenceNumber));
+					receivedBuffers.add(sequenceBuffer);
 					channelStatePersister.maybePersist(buffer);
-				}
 
+					if (dataType.requiresAnnouncement()) {
+						checkPriorityXorAnnouncement(buffer);
+						firstPriorityEvent = addPriorityBuffer(announce(sequenceBuffer));
+					}

Review comment:
       ```
   if (dataType.hasPriority() || dataType.requiresAnnouncement()) {
     firstPriorityEvent = addPriorityBuffer(announce(sequenceBuffer));
   }
   if (!dataType.hasPriority()) {
     receivedBuffers.add(sequenceBuffer);
     channelStatePersister.maybePersist(buffer);
   }
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierAnnouncement.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.api;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.event.RuntimeEvent;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * {@link CheckpointBarrierAnnouncement} is announcing presence or receiving of a {@link CheckpointBarrier}.
+ * That {@link #announcedBarrier} is identified by it's sequence number.
+ */
+public class CheckpointBarrierAnnouncement extends RuntimeEvent {

Review comment:
       I was hoping that we could make the announcement mechanism a bit independent of checkpoints. I don't see how we can generalize DataType, but I could imagine having a generic `AnnouncementEvent`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -471,6 +475,40 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 		}
 	}
 
+	private void checkPriorityXorAnnouncement(Buffer buffer) {

Review comment:
       Instead of checking it during deserialization on all buffers, why not simply check it in `DataType` enum once?
   In particular, I don't see the need to ever announce priority events.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/SingleCheckpointBarrierHandler.java
##########
@@ -147,6 +148,13 @@ public void processBarrier(CheckpointBarrier barrier, InputChannelInfo channelIn
 		}
 	}
 
+	@Override
+	public void processBarrierAnnouncement(
+			CheckpointBarrierAnnouncement barrierAnnouncement,
+			InputChannelInfo channelInfo) throws IOException {
+		// Ignore for now.

Review comment:
       Add TODO instead?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
##########
@@ -292,21 +313,37 @@ public boolean isEvent() {
 			return isEvent;
 		}
 
+		public boolean isBlockingUpstream() {

Review comment:
       Why did you move this method?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
##########
@@ -247,41 +247,62 @@
 		/**
 		 * {@link #NONE} indicates that there is no buffer.
 		 */
-		NONE(false, false, false, false),
+		NONE(false, false, false, false, false),
 
 		/**
 		 * {@link #DATA_BUFFER} indicates that this buffer represents a non-event data buffer.
 		 */
-		DATA_BUFFER(true, false, false, false),
+		DATA_BUFFER(true, false, false, false, false),
 
 		/**
 		 * {@link #EVENT_BUFFER} indicates that this buffer represents serialized data of an event.
 		 * Note that this type can be further divided into more fine-grained event types
 		 * like {@link #ALIGNED_CHECKPOINT_BARRIER} and etc.
 		 */
-		EVENT_BUFFER(false, true, false, false),
+		EVENT_BUFFER(false, true, false, false, false),
 
 		/**
 		 * Same as EVENT_BUFFER, but the event has been prioritized (e.g. it skipped buffers).
 		 */
-		PRIORITIZED_EVENT_BUFFER(false, true, false, true),
+		PRIORITIZED_EVENT_BUFFER(false, true, false, true, false),
 
 		/**
 		 * {@link #ALIGNED_CHECKPOINT_BARRIER} indicates that this buffer represents a
 		 * serialized checkpoint barrier of aligned exactly-once checkpoint mode.
 		 */
-		ALIGNED_CHECKPOINT_BARRIER(false, true, true, false);
+		ALIGNED_CHECKPOINT_BARRIER(false, true, true, false, false),
+
+		/**
+		 * {@link #TIMEOUTABLE_ALIGNED_CHECKPOINT_BARRIER} indicates that this buffer represents a
+		 * serialized checkpoint barrier of aligned exactly-once checkpoint mode, that can be time-out'ed
+		 * to an unaligned checkpoint barrier.
+		 */
+		TIMEOUTABLE_ALIGNED_CHECKPOINT_BARRIER(false, true, true, false, true);

Review comment:
       As a set before, I'm not a huge fan of this very specific `DataType`s. (Why don't we have an `DataType` just for `EndOfPartitionEvent`s as well?) I also think that `ALIGNED_CHECKPOINT_BARRIER` is on a different level than all other types.
   
   One option that I could see is to just have `ANNOUNCED_EVENT_BUFFER(false, true, true, false, true)`.




----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 152532018780b987ea2447abd20097375457655f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133) 
   * eef96ff91e5f02a5f777d17cce91e860816ca15b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a68242a90fd180fcd0b4b4c60b5e1e49136c00f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082) 
   
   <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 merged pull request #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   


----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 152532018780b987ea2447abd20097375457655f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133) 
   * eef96ff91e5f02a5f777d17cce91e860816ca15b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226) 
   * 827c2f19269d5ef5ff2ff6fb214b30dd12955b01 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 827c2f19269d5ef5ff2ff6fb214b30dd12955b01 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227) 
   * 33b05f3143953a3bcb7a526326034cfd3bc6da5b 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 pull request #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   > I don't see the benefit to add alignment timeout to CheckpointOptions. I'd rather have it on task level pass to Aligner.
   
   if not `alignmentTimeout`, I would still need to add some flag like `isTimeoutable` or `requiresAnnouncement` to the `CheckpointOptions`, to distinguish in the `DataType#getDataType` between:
   - aligned non-timeoutable checkpoint/savepoint
   - aligned timeoutable checkpoint
   - unaligned checkpoint
   
   if we already need this bit of information, why not use it pass the timeout itself?
   
   


----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428",
       "triggerID" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8512",
       "triggerID" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42483b09d6194fdab365ee9696522c9c75688391",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8527",
       "triggerID" : "42483b09d6194fdab365ee9696522c9c75688391",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ab71efa489d12df19be9eeb6cc814ee60206598c Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8512) 
   * 42483b09d6194fdab365ee9696522c9c75688391 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8527) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierAnnouncement.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.api;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.event.RuntimeEvent;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * {@link CheckpointBarrierAnnouncement} is announcing presence or receiving of a {@link CheckpointBarrier}.
+ * That {@link #announcedBarrier} is identified by it's sequence number.
+ */
+public class CheckpointBarrierAnnouncement extends RuntimeEvent {

Review comment:
       Okay let's go with specialized and generalize when a second use case pops up.




----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -437,19 +440,20 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOEx
 
 				wasEmpty = receivedBuffers.isEmpty();
 
-				if (buffer.getDataType().hasPriority()) {
-					receivedBuffers.addPriorityElement(new SequenceBuffer(buffer, sequenceNumber));
-					if (channelStatePersister.checkForBarrier(buffer)) {
-						// 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 = receivedBuffers.getNumUnprioritizedElements();
-					}
-					firstPriorityEvent = receivedBuffers.getNumPriorityElements() == 1;
+				SequenceBuffer sequenceBuffer = new SequenceBuffer(buffer, sequenceNumber);
+				DataType dataType = buffer.getDataType();
+				if (dataType.hasPriority()) {
+					checkPriorityXorAnnouncement(buffer);
+					firstPriorityEvent = addPriorityBuffer(sequenceBuffer);
 				} else {
-					receivedBuffers.add(new SequenceBuffer(buffer, sequenceNumber));
+					receivedBuffers.add(sequenceBuffer);
 					channelStatePersister.maybePersist(buffer);
-				}
 
+					if (dataType.requiresAnnouncement()) {
+						checkPriorityXorAnnouncement(buffer);
+						firstPriorityEvent = addPriorityBuffer(announce(sequenceBuffer));
+					}

Review comment:
       This is incorrect, it would have to be:
   ```
   				if (dataType.requiresAnnouncement()) {
   					firstPriorityEvent = addPriorityBuffer(announce(sequenceBuffer));
   				}
   				if (dataType.hasPriority()) {
   					firstPriorityEvent = addPriorityBuffer(sequenceBuffer);
   				}
   				else {
   					receivedBuffers.add(sequenceBuffer);
   					channelStatePersister.maybePersist(buffer);
   				}
   ```
   but that would suggest priority event can be announced, hence I moved 
   ```
   				if (dataType.requiresAnnouncement()) {
   					firstPriorityEvent = addPriorityBuffer(announce(sequenceBuffer));
   				}
   ```
   into the non priority branch.




----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 35aaad405a63bb56ff1be4aa45028b22058c0858 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 152532018780b987ea2447abd20097375457655f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133) 
   * eef96ff91e5f02a5f777d17cce91e860816ca15b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226) 
   * 827c2f19269d5ef5ff2ff6fb214b30dd12955b01 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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428",
       "triggerID" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8512",
       "triggerID" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd9a7059d4b4fa7c0760f2c82731436fe1d1312a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428) 
   * ab71efa489d12df19be9eeb6cc814ee60206598c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8512) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a68242a90fd180fcd0b4b4c60b5e1e49136c00f 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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   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 8a68242a90fd180fcd0b4b4c60b5e1e49136c00f (Thu Oct 22 06:55:23 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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 33b05f3143953a3bcb7a526326034cfd3bc6da5b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237) 
   * 35aaad405a63bb56ff1be4aa45028b22058c0858 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428",
       "triggerID" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd9a7059d4b4fa7c0760f2c82731436fe1d1312a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428) 
   * ab71efa489d12df19be9eeb6cc814ee60206598c 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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 152532018780b987ea2447abd20097375457655f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133) 
   * eef96ff91e5f02a5f777d17cce91e860816ca15b 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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 827c2f19269d5ef5ff2ff6fb214b30dd12955b01 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227) 
   * 33b05f3143953a3bcb7a526326034cfd3bc6da5b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 35aaad405a63bb56ff1be4aa45028b22058c0858 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243) 
   * fd9a7059d4b4fa7c0760f2c82731436fe1d1312a 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 pull request #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   Azure green, merging.


----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * eef96ff91e5f02a5f777d17cce91e860816ca15b Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226) 
   * 827c2f19269d5ef5ff2ff6fb214b30dd12955b01 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428",
       "triggerID" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 35aaad405a63bb56ff1be4aa45028b22058c0858 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243) 
   * fd9a7059d4b4fa7c0760f2c82731436fe1d1312a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428",
       "triggerID" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd9a7059d4b4fa7c0760f2c82731436fe1d1312a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8a68242a90fd180fcd0b4b4c60b5e1e49136c00f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082) 
   * 1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b 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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428",
       "triggerID" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8512",
       "triggerID" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42483b09d6194fdab365ee9696522c9c75688391",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "42483b09d6194fdab365ee9696522c9c75688391",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ab71efa489d12df19be9eeb6cc814ee60206598c Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8512) 
   * 42483b09d6194fdab365ee9696522c9c75688391 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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113) 
   * 152532018780b987ea2447abd20097375457655f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133) 
   
   <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 pull request #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   @AHeise, I have replaced `CheckpointBarrierAnnouncement` with `EventAnnouncement` as you were suggesting. Regarding the `DataType.TIMEOUTABLE_ALIGNED_CHECKPOINT_BARRIER` being inappropriate, it has the same pre-existing problem as  `ALIGNED_CHECKPOINT_BARRIER`, since both require `isBlockingUpstream` flag. 


----------------------------------------------------------------
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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8243",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8428",
       "triggerID" : "fd9a7059d4b4fa7c0760f2c82731436fe1d1312a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8512",
       "triggerID" : "ab71efa489d12df19be9eeb6cc814ee60206598c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "42483b09d6194fdab365ee9696522c9c75688391",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8527",
       "triggerID" : "42483b09d6194fdab365ee9696522c9c75688391",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 42483b09d6194fdab365ee9696522c9c75688391 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8527) 
   
   <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 #13741: [FLINK-19680][checkpointing] Announce timeoutable CheckpointBarriers

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8082",
       "triggerID" : "8a68242a90fd180fcd0b4b4c60b5e1e49136c00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8113",
       "triggerID" : "1d5e07a51aa4c2ad129dae99e1e34c02f217ec4b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "152532018780b987ea2447abd20097375457655f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8133",
       "triggerID" : "152532018780b987ea2447abd20097375457655f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8226",
       "triggerID" : "eef96ff91e5f02a5f777d17cce91e860816ca15b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8227",
       "triggerID" : "827c2f19269d5ef5ff2ff6fb214b30dd12955b01",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237",
       "triggerID" : "33b05f3143953a3bcb7a526326034cfd3bc6da5b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "35aaad405a63bb56ff1be4aa45028b22058c0858",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 33b05f3143953a3bcb7a526326034cfd3bc6da5b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8237) 
   * 35aaad405a63bb56ff1be4aa45028b22058c0858 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