pnowojski commented on code in PR #20233:
URL: https://github.com/apache/flink/pull/20233#discussion_r923305444


##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java:
##########
@@ -208,19 +238,42 @@ public void abort(long checkpointId, Throwable cause, 
boolean cleanup) {
         enqueue(
                 ChannelStateWriteRequest.abort(checkpointId, cause),
                 false); // abort enqueued but not started
-        if (cleanup) {
-            results.remove(checkpointId);
+        synchronized (lock) {
+            if (checkpointId >= ongoingCheckpointId) {
+                abortedCheckpointIds.add(checkpointId);
+            }
+            if (cleanup && checkpointId == ongoingCheckpointId) {
+                result = null;
+            }
         }
     }
 
     @Override
     public ChannelStateWriteResult getAndRemoveWriteResult(long checkpointId) {
         LOG.debug("{} requested write result, checkpoint {}", taskName, 
checkpointId);
-        ChannelStateWriteResult result = results.remove(checkpointId);
-        Preconditions.checkArgument(
-                result != null,
-                taskName + " channel state write result not found for 
checkpoint " + checkpointId);
-        return result;
+        ChannelStateWriteResult returnResult;
+        synchronized (lock) {
+            Preconditions.checkState(checkpointId == ongoingCheckpointId);
+            returnResult = result;
+            result = null;
+        }
+        if (returnResult == null) {
+            Preconditions.checkArgument(
+                    isUnavailableCheckpoint(checkpointId),
+                    taskName
+                            + " channel state write result not found for 
checkpoint "
+                            + checkpointId);
+            returnResult = new ChannelStateWriteResult();
+            returnResult.fail(new CancellationException("The checkpoint is 
aborted."));
+        }
+        return returnResult;
+    }
+
+    private boolean isUnavailableCheckpoint(long checkpointId) {

Review Comment:
   `isUnavailableCheckpoint` -> `isCheckpointSubsumedOrAborted` or 
`isValidCheckpoint`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java:
##########
@@ -208,19 +238,42 @@ public void abort(long checkpointId, Throwable cause, 
boolean cleanup) {
         enqueue(
                 ChannelStateWriteRequest.abort(checkpointId, cause),
                 false); // abort enqueued but not started
-        if (cleanup) {
-            results.remove(checkpointId);
+        synchronized (lock) {
+            if (checkpointId >= ongoingCheckpointId) {
+                abortedCheckpointIds.add(checkpointId);
+            }
+            if (cleanup && checkpointId == ongoingCheckpointId) {
+                result = null;
+            }
         }
     }
 
     @Override
     public ChannelStateWriteResult getAndRemoveWriteResult(long checkpointId) {
         LOG.debug("{} requested write result, checkpoint {}", taskName, 
checkpointId);
-        ChannelStateWriteResult result = results.remove(checkpointId);
-        Preconditions.checkArgument(
-                result != null,
-                taskName + " channel state write result not found for 
checkpoint " + checkpointId);
-        return result;
+        ChannelStateWriteResult returnResult;
+        synchronized (lock) {
+            Preconditions.checkState(checkpointId == ongoingCheckpointId);
+            returnResult = result;
+            result = null;
+        }
+        if (returnResult == null) {
+            Preconditions.checkArgument(
+                    isUnavailableCheckpoint(checkpointId),
+                    taskName
+                            + " channel state write result not found for 
checkpoint "
+                            + checkpointId);
+            returnResult = new ChannelStateWriteResult();
+            returnResult.fail(new CancellationException("The checkpoint is 
aborted."));
+        }

Review Comment:
   Why can this be `null`? And if it happens, why are you creating a new empty 
`ChannelStateWriteResult` just to fail it?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java:
##########
@@ -60,89 +63,78 @@
 public class ChannelStateWriterImpl implements ChannelStateWriter {
 
     private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
-    private static final int DEFAULT_MAX_CHECKPOINTS =
-            1000; // includes max-concurrent-checkpoints + checkpoints to be 
aborted (scheduled via
-    // mailbox)
 
     private final String taskName;
     private final ChannelStateWriteRequestExecutor executor;
-    private final ConcurrentMap<Long, ChannelStateWriteResult> results;
-    private final int maxCheckpoints;
 
-    /**
-     * Creates a {@link ChannelStateWriterImpl} with {@link 
#DEFAULT_MAX_CHECKPOINTS} as {@link
-     * #maxCheckpoints}.
-     */
-    public ChannelStateWriterImpl(
-            String taskName, int subtaskIndex, CheckpointStorageWorkerView 
streamFactoryResolver) {
-        this(taskName, subtaskIndex, streamFactoryResolver, 
DEFAULT_MAX_CHECKPOINTS);
-    }
+    private final Object lock = new Object();
+
+    @GuardedBy("lock")
+    private long ongoingCheckpointId;
+
+    @GuardedBy("lock")
+    private final NavigableSet<Long> abortedCheckpointIds;
+
+    // The result of ongoingCheckpointId, the checkpoint that CheckpointId is 
less than
+    // ongoingCheckpointId should be aborted due to concurrent unaligned 
checkpoint is currently not
+    // supported.
+    @GuardedBy("lock")
+    private ChannelStateWriteResult result;
 
     /**
      * Creates a {@link ChannelStateWriterImpl} with {@link 
ChannelStateSerializerImpl default}
      * {@link ChannelStateSerializer}, and a {@link 
ChannelStateWriteRequestExecutorImpl}.
      *
      * @param taskName
      * @param streamFactoryResolver a factory to obtain output stream factory 
for a given checkpoint
-     * @param maxCheckpoints maximum number of checkpoints to be written 
currently or finished but
-     *     not taken yet.
      */
-    ChannelStateWriterImpl(
-            String taskName,
-            int subtaskIndex,
-            CheckpointStorageWorkerView streamFactoryResolver,
-            int maxCheckpoints) {
+    public ChannelStateWriterImpl(
+            String taskName, int subtaskIndex, CheckpointStorageWorkerView 
streamFactoryResolver) {
         this(
                 taskName,
-                new ConcurrentHashMap<>(maxCheckpoints),
                 new ChannelStateWriteRequestExecutorImpl(
                         taskName,
                         new ChannelStateWriteRequestDispatcherImpl(
                                 taskName,
                                 subtaskIndex,
                                 streamFactoryResolver,
-                                new ChannelStateSerializerImpl())),
-                maxCheckpoints);
+                                new ChannelStateSerializerImpl())));
     }
 
-    ChannelStateWriterImpl(
-            String taskName,
-            ConcurrentMap<Long, ChannelStateWriteResult> results,
-            ChannelStateWriteRequestExecutor executor,
-            int maxCheckpoints) {
+    ChannelStateWriterImpl(String taskName, ChannelStateWriteRequestExecutor 
executor) {
         this.taskName = taskName;
-        this.results = results;
-        this.maxCheckpoints = maxCheckpoints;
         this.executor = executor;
+        this.abortedCheckpointIds = new TreeSet<>();
+        this.ongoingCheckpointId = 0;
     }
 
     @Override
     public void start(long checkpointId, CheckpointOptions checkpointOptions) {
         LOG.debug("{} starting checkpoint {} ({})", taskName, checkpointId, 
checkpointOptions);
-        ChannelStateWriteResult result = new ChannelStateWriteResult();
-        ChannelStateWriteResult put =
-                results.computeIfAbsent(
+        synchronized (lock) {
+            if (isUnavailableCheckpoint(checkpointId)) {
+                LOG.debug(
+                        "The checkpoint {} of task {} has been aborted, so 
don't start.",
                         checkpointId,
-                        id -> {
-                            Preconditions.checkState(
-                                    results.size() < maxCheckpoints,
-                                    String.format(
-                                            "%s can't start %d, results.size() 
> maxCheckpoints: %d > %d",
-                                            taskName,
-                                            checkpointId,
-                                            results.size(),
-                                            maxCheckpoints));
-                            enqueue(
-                                    new CheckpointStartRequest(
-                                            checkpointId,
-                                            result,
-                                            
checkpointOptions.getTargetLocation()),
-                                    false);
-                            return result;
-                        });
-        Preconditions.checkArgument(
-                put == result,
-                taskName + " result future already present for checkpoint " + 
checkpointId);
+                        taskName);

Review Comment:
   Throw an exception instead?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java:
##########
@@ -60,89 +63,78 @@
 public class ChannelStateWriterImpl implements ChannelStateWriter {
 
     private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
-    private static final int DEFAULT_MAX_CHECKPOINTS =
-            1000; // includes max-concurrent-checkpoints + checkpoints to be 
aborted (scheduled via
-    // mailbox)

Review Comment:
   Why do you need this `NavigableSet<Long> abortedCheckpointIds;` and 
`ongoingCheckpointId`? Wouldn't `long nextExpectedCheckpointId` be enough on 
it's own?
   
   ```
   abort(long checkpointId) {
     nextExpectedCheckpointId = max(nextExpectedCheckpointId, checkpointId + 1);
     ...
   }
   
   start(long checkpointId) {
     checkState(checkpointId >= nextExpectedCheckpointId, "Unexpected next 
checkpoint. Concurrent checkpoints are not supported");
   
     nextExpectedCheckpointId = max(nextExpectedCheckpointId, checkpointId + 1);
     ...
   }
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImplTest.java:
##########
@@ -68,6 +69,15 @@ public void testAddEventBuffer() throws Exception {
         }
     }
 
+    @Test(expected = IllegalStateException.class)
+    public void testStartMultipleTimes() throws Exception {
+        runWithSyncWorker(
+                writer -> {
+                    callStart(writer);
+                    callStart(writer);
+                });

Review Comment:
   Aren't we missing a unit test for:
   ```
   1.
   
   writer.start(44);
   writer.start(42); //expected exception
   ```
   ? Can you also squash the commit adding unit tests with the previous commit 
that was implementing the change?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java:
##########
@@ -157,6 +149,14 @@ public void addInputData(
                 checkpointId,
                 info,
                 startSeqNum);
+        if (isUnavailableCheckpoint(checkpointId)) {
+            LOG.debug(
+                    "The checkpoint {} of task {} has been aborted, so don't 
addInputData.",
+                    checkpointId,
+                    taskName);
+            enqueue(closeBuffers(checkpointId, iterator), false);

Review Comment:
   If we are closing the buffers, why do we need to enqueue this as a request? 
Can not we just close the buffers directly here (and in other methods in this 
class)?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java:
##########
@@ -60,89 +63,78 @@
 public class ChannelStateWriterImpl implements ChannelStateWriter {
 
     private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
-    private static final int DEFAULT_MAX_CHECKPOINTS =
-            1000; // includes max-concurrent-checkpoints + checkpoints to be 
aborted (scheduled via
-    // mailbox)
 
     private final String taskName;
     private final ChannelStateWriteRequestExecutor executor;
-    private final ConcurrentMap<Long, ChannelStateWriteResult> results;
-    private final int maxCheckpoints;
 
-    /**
-     * Creates a {@link ChannelStateWriterImpl} with {@link 
#DEFAULT_MAX_CHECKPOINTS} as {@link
-     * #maxCheckpoints}.
-     */
-    public ChannelStateWriterImpl(
-            String taskName, int subtaskIndex, CheckpointStorageWorkerView 
streamFactoryResolver) {
-        this(taskName, subtaskIndex, streamFactoryResolver, 
DEFAULT_MAX_CHECKPOINTS);
-    }
+    private final Object lock = new Object();
+
+    @GuardedBy("lock")
+    private long ongoingCheckpointId;
+
+    @GuardedBy("lock")
+    private final NavigableSet<Long> abortedCheckpointIds;
+
+    // The result of ongoingCheckpointId, the checkpoint that CheckpointId is 
less than
+    // ongoingCheckpointId should be aborted due to concurrent unaligned 
checkpoint is currently not
+    // supported.
+    @GuardedBy("lock")
+    private ChannelStateWriteResult result;

Review Comment:
   Why do you need this lock? (This class is supposed to be owned and used only 
by the task thread)



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

To unsubscribe, e-mail: [email protected]

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

Reply via email to