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



##########
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 what I commented 
above.




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