curcur commented on a change in pull request #15897:
URL: https://github.com/apache/flink/pull/15897#discussion_r634345124



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/checkpointing/AbstractAlternatingAlignedBarrierHandlerState.java
##########
@@ -72,7 +71,11 @@ public final BarrierHandlerState barrierReceived(
 
     @Override
     public final BarrierHandlerState abort(long cancelledId) throws 
IOException {
+        return stopCheckpoint();
+    }
+
+    private BarrierHandlerState stopCheckpoint() throws IOException {

Review comment:
       this name (`stopCheckPoint`) is a bit confusing, it is more like 
resuming to a fresh consumption state instead of `stopCheckpoint`? This is also 
called right after `controller.triggerGlobalCheckpoint(checkpointBarrier);` 
sounds like we stop the checkpoint right after triggering it. Maybe 
`finishCheckpoint`?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/checkpointing/AlternatingWaitingForFirstBarrierUnaligned.java
##########
@@ -57,35 +57,37 @@ public BarrierHandlerState barrierReceived(
             InputChannelInfo channelInfo,
             CheckpointBarrier checkpointBarrier)
             throws CheckpointException, IOException {
+
         // we received an out of order aligned barrier, we should resume 
consumption for the
         // channel, as it is being blocked by the credit-based network
         if (!checkpointBarrier.getCheckpointOptions().isUnalignedCheckpoint()) 
{
-            inputs[channelInfo.getGateIdx()].resumeConsumption(channelInfo);
+            channelState.blockChannel(channelInfo);
         }
 
         CheckpointBarrier unalignedBarrier = checkpointBarrier.asUnaligned();
         controller.initInputsCheckpoint(unalignedBarrier);
-        for (CheckpointableInput input : inputs) {
+        for (CheckpointableInput input : channelState.getInputs()) {
             input.checkpointStarted(unalignedBarrier);
         }
         controller.triggerGlobalCheckpoint(unalignedBarrier);
         if (controller.allBarriersReceived()) {
-            for (CheckpointableInput input : inputs) {
+            for (CheckpointableInput input : channelState.getInputs()) {
                 input.checkpointStopped(unalignedBarrier.getId());
             }
-            if (alternating) {
-                return new AlternatingWaitingForFirstBarrier(inputs);
-            } else {
-                return this;
-            }
+            return stopCheckpoint();
         }
-        return new CollectingBarriersUnaligned(alternating, inputs);
+        return new AlternatingCollectingBarriersUnaligned(alternating, 
channelState);
     }
 
     @Override
-    public BarrierHandlerState abort(long cancelledId) {
+    public BarrierHandlerState abort(long cancelledId) throws IOException {
+        return stopCheckpoint();
+    }
+
+    private BarrierHandlerState stopCheckpoint() throws IOException {
+        channelState.unblockAllChannels();
         if (alternating) {
-            return new AlternatingWaitingForFirstBarrier(inputs);
+            return new 
AlternatingWaitingForFirstBarrier(channelState.emptyState());
         } else {
             return this;

Review comment:
       Should the channelstate of "this" be empty, same as above.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/checkpointing/AlternatingWaitingForFirstBarrierUnaligned.java
##########
@@ -57,35 +57,37 @@ public BarrierHandlerState barrierReceived(
             InputChannelInfo channelInfo,
             CheckpointBarrier checkpointBarrier)
             throws CheckpointException, IOException {
+
         // we received an out of order aligned barrier, we should resume 
consumption for the
         // channel, as it is being blocked by the credit-based network
         if (!checkpointBarrier.getCheckpointOptions().isUnalignedCheckpoint()) 
{
-            inputs[channelInfo.getGateIdx()].resumeConsumption(channelInfo);
+            channelState.blockChannel(channelInfo);

Review comment:
       I am a bit struggling thinking of cases where out-of-order barriers 
come?  Since transition happens when 
AlternatingWaitingForFirstBarrier#alignmentTimeout, and converting to prior 
events happen before transition.
   

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/checkpointing/AlternatingWaitingForFirstBarrierUnaligned.java
##########
@@ -57,35 +57,37 @@ public BarrierHandlerState barrierReceived(
             InputChannelInfo channelInfo,
             CheckpointBarrier checkpointBarrier)
             throws CheckpointException, IOException {
+
         // we received an out of order aligned barrier, we should resume 
consumption for the
         // channel, as it is being blocked by the credit-based network

Review comment:
       The comment needs to be updated accordingly.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/checkpointing/AlternatingCollectingBarriersUnaligned.java
##########
@@ -74,14 +74,15 @@ public BarrierHandlerState abort(long cancelledId) throws 
IOException {
         return stopCheckpoint(cancelledId);
     }
 
-    private BarrierHandlerState stopCheckpoint(long cancelledId) {
-        for (CheckpointableInput input : inputs) {
+    private BarrierHandlerState stopCheckpoint(long cancelledId) throws 
IOException {
+        for (CheckpointableInput input : channelState.getInputs()) {
             input.checkpointStopped(cancelledId);
         }
+        channelState.unblockAllChannels();
         if (alternating) {
-            return new AlternatingWaitingForFirstBarrier(inputs);
+            return new 
AlternatingWaitingForFirstBarrier(channelState.emptyState());
         } else {
-            return new WaitingForFirstBarrierUnaligned(false, inputs);
+            return new AlternatingWaitingForFirstBarrierUnaligned(false, 
channelState);

Review comment:
       Should this be `channelState.emptyState()` for unaligned checkpoint as 
well?
   I guess it means the same thing because 
AlternatingWaitingForFirstBarrierUnaligned(false,...) does not have 
announcements? But this requires the knowledge and implicit assumption of that.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/checkpointing/AlternatingCollectingBarriersUnaligned.java
##########
@@ -60,7 +60,7 @@ public BarrierHandlerState barrierReceived(
         // we received an out of order aligned barrier, we should resume 
consumption for the
         // channel, as it is being blocked by the credit-based network
         if (!checkpointBarrier.getCheckpointOptions().isUnalignedCheckpoint()) 
{
-            inputs[channelInfo.getGateIdx()].resumeConsumption(channelInfo);
+            channelState.blockChannel(channelInfo);

Review comment:
       The comment needs to be updated accordingly.

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/AlternatingCheckpointsTest.java
##########
@@ -507,13 +477,20 @@ public void 
testAllChannelsUnblockedAfterAlignmentTimeout() throws Exception {
 
         // we set timer on announcement and test channels do not produce 
announcements by themselves
         send(EventSerializer.toBuffer(new EventAnnouncement(checkpointBarrier, 
0), true), 0, gate);
-        send(checkpointBarrierBuffer, 0, gate);
         // emulate blocking channels on aligned barriers
         ((TestInputChannel) gate.getChannel(0)).setBlocked(true);

Review comment:
       Is this necessary? The channel is blocked after receiving the first 
aliged barrier in 
   
   AbstractAlternatingAlignedBarrierHandlerState#barrierReceived




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

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


Reply via email to