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/05/15 08:35:04 UTC

[GitHub] [flink-statefun] tzulitai commented on a change in pull request #105: [FLINK-17533] Add support for multiple concurrent checkpoints

tzulitai commented on a change in pull request #105:
URL: https://github.com/apache/flink-statefun/pull/105#discussion_r425636013



##########
File path: statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/feedback/FeedbackUnionOperator.java
##########
@@ -84,14 +90,30 @@ public void processFeedback(T element) {
     if (closedOrDisposed) {
       return;
     }
-    if (isBarrierMessage.test(element)) {
-      feedbackLogger.commit();
+    OptionalLong maybeCheckpoint = isBarrierMessage.apply(element);
+    if (maybeCheckpoint.isPresent()) {
+      commitCheckpoint(maybeCheckpoint.getAsLong());
     } else {
       sendDownstream(element);
-      feedbackLogger.append(element);
+      appendToCheckpoint(element);
+    }
+  }
+
+  private void appendToCheckpoint(T element) {
+    for (UnboundedFeedbackLogger<T> logger : uncompletedCheckpoints.values()) {
+      logger.append(element);
     }
   }
 
+  private void commitCheckpoint(final long checkpointId) {
+    UnboundedFeedbackLogger<T> logger = uncompletedCheckpoints.remove(checkpointId);

Review comment:
       Maybe more of a question from my side:
   Is this method guaranteed to be called for a given checkpoint id, even if the checkpoint already failed?
   I'm wondering if that's a case where this `uncompletedCheckpoints` map grows indefinitely in size.

##########
File path: statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/feedback/Checkpoints.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.statefun.flink.core.feedback;
+
+import java.io.OutputStream;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import org.apache.flink.statefun.flink.core.logger.UnboundedFeedbackLogger;
+import org.apache.flink.statefun.flink.core.logger.UnboundedFeedbackLoggerFactory;
+import org.apache.flink.util.IOUtils;
+
+final class Checkpoints<T> implements AutoCloseable {

Review comment:
       Ha 😄You read my mind ahead of time @igalshilman.
   This again addresses my previous nitpick on the new logic introduced in `FeedbackUnionOperator`.
   
   What I think is only missing and nice to have is then a unit test for this `Checkpoints` class 👍 

##########
File path: statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/feedback/FeedbackUnionOperator.java
##########
@@ -105,13 +104,11 @@ private void appendToCheckpoint(T element) {
     }
   }
 
-  private void commitCheckpoint(final long checkpointId) {
-    UnboundedFeedbackLogger<T> logger = uncompletedCheckpoints.remove(checkpointId);
-    checkState(
-        logger != null,
-        "checkpoint barrier for a checkpoint id %d is missing a logger.",
-        checkpointId);
-    logger.commit();
+  private void commitCheckpointUntil(final long checkpointId) {

Review comment:
       Ah :) This answers my previous question already 👍 

##########
File path: statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/feedback/FeedbackUnionOperator.java
##########
@@ -43,20 +48,21 @@
 
   // -- configuration
   private final FeedbackKey<T> feedbackKey;
-  private final SerializablePredicate<T> isBarrierMessage;
+  private final SerializableFunction<T, OptionalLong> isBarrierMessage;
   private final SerializableFunction<T, ?> keySelector;
   private final long totalMemoryUsedForFeedbackCheckpointing;
   private final TypeSerializer<T> elementSerializer;
 
   // -- runtime
-  private transient UnboundedFeedbackLogger<T> feedbackLogger;
+  private transient UnboundedFeedbackLoggerFactory<T> feedbackLoggerFactory;
+  private transient Map<Long, UnboundedFeedbackLogger<T>> uncompletedCheckpoints;
   private transient boolean closedOrDisposed;
   private transient MailboxExecutor mailboxExecutor;
   private transient StreamRecord<T> reusable;
 
   FeedbackUnionOperator(

Review comment:
       Perhaps a bit of a nitpick, but with the added functionality to this class, we're now slightly lacking in unit test coverage.
   
   Before having just the `UnboundedFeedbackLoggerTest` is sufficient as it already essentially encapsulates the core logic.
   
   Maybe moving some of the new methods, such as `commitCheckpoint(maybeCheckpoint.getAsLong());` / `appendToCheckpoints(element);` and the `uncompletedCheckpoints` map, essentially the logic of bookkeeping multiple loggers, to a separate `UnboundedFeedbackLoggers` class, and have a unit test for that is enough.

##########
File path: statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/feedback/FeedbackUnionOperator.java
##########
@@ -84,14 +90,30 @@ public void processFeedback(T element) {
     if (closedOrDisposed) {
       return;
     }
-    if (isBarrierMessage.test(element)) {
-      feedbackLogger.commit();
+    OptionalLong maybeCheckpoint = isBarrierMessage.apply(element);
+    if (maybeCheckpoint.isPresent()) {
+      commitCheckpoint(maybeCheckpoint.getAsLong());
     } else {
       sendDownstream(element);
-      feedbackLogger.append(element);
+      appendToCheckpoint(element);

Review comment:
       nit: maybe rename to -
   ```suggestion
         appendToCheckpoints(element);
   ```




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