fredia commented on code in PR #24748:
URL: https://github.com/apache/flink/pull/24748#discussion_r1596255057


##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/EpochManager.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.asyncprocessing;
+
+import javax.annotation.Nullable;
+
+import java.util.LinkedList;
+
+/**
+ * Epoch manager segments inputs into distinct epochs, marked by the arrival 
of non-records(e.g.
+ * watermark, record attributes). Records are assigned to a unique epoch based 
on their arrival,
+ * records within an epoch are allowed to be parallelized, while the 
non-record of an epoch can only
+ * be executed when all records in this epoch have finished.
+ *
+ * <p>For more details please refer to FLIP-425.
+ */
+public class EpochManager {
+
+    /**
+     * This enum defines whether parallel execution between epochs is allowed. 
We should keep this
+     * internal and away from API module for now, until we could see the 
concrete need for {@link
+     * #PARALLEL_BETWEEN_EPOCH} from average users.
+     */
+    public enum ParallelMode {
+        /**
+         * Subsequent epochs must wait until the previous epoch is completed 
before they can start.
+         */
+        SERIAL_BETWEEN_EPOCH,
+        /**
+         * Subsequent epochs can begin execution even if the previous epoch 
has not yet completed.
+         * Usually performs better than {@link #SERIAL_BETWEEN_EPOCH}.
+         */
+        PARALLEL_BETWEEN_EPOCH
+    }
+
+    /**
+     * The reference to the {@link AsyncExecutionController}, used for {@link
+     * ParallelMode#SERIAL_BETWEEN_EPOCH}. Can be null when testing.
+     */
+    final AsyncExecutionController asyncExecutionController;
+
+    /** The number of epochs that have arrived. */
+    long epochNum;
+
+    /** The output queue to hold ongoing epochs. */
+    LinkedList<Epoch> outputQueue;
+
+    /** Current active epoch, only one active epoch at the same time. */
+    Epoch activeEpoch;
+
+    public EpochManager(AsyncExecutionController aec) {
+        this.epochNum = 0;
+        this.outputQueue = new LinkedList<>();
+        // preset an empty epoch, the epoch action will be updated when 
non-record is received.
+        this.activeEpoch = new Epoch(epochNum++);
+        this.outputQueue.add(activeEpoch);
+        this.asyncExecutionController = aec;
+    }
+
+    /**
+     * Add a record to the current epoch and return the current open epoch, 
the epoch will be
+     * associated with the {@link RecordContext} of this record. Must be 
invoked within task thread.
+     *
+     * @return the current open epoch.
+     */
+    public Epoch onRecord() {
+        activeEpoch.ongoingRecordCount++;
+        return activeEpoch;
+    }
+
+    /**
+     * Add a non-record to the current epoch, close current epoch and open a 
new epoch. Must be
+     * invoked within task thread.
+     *
+     * @param action the action associated with this non-record.
+     * @param parallelMode the parallel mode for this epoch.
+     */
+    public void onNonRecord(Runnable action, ParallelMode parallelMode) {
+        if (parallelMode == ParallelMode.PARALLEL_BETWEEN_EPOCH) {
+            activeEpoch.close(action);
+            if (outputQueue.size() == 1) { // which means the first epoch
+                if (activeEpoch.tryFinish()) {
+                    outputQueue.pop();
+                }
+            }
+            activeEpoch = new Epoch(epochNum++);
+            outputQueue.add(activeEpoch);
+        } else {
+            assert outputQueue.size() == 1;
+            activeEpoch.close(action);
+            asyncExecutionController.drainInflightRecords(0);
+            if (activeEpoch.tryFinish() && outputQueue.size() > 0) {
+                outputQueue.pop();
+            }

Review Comment:
   I think the `while loop` is not necessary here, changing it to `while loop` 
won't have any bad effects either.
   
   Because this method is only called on non-record arrivals, this epoch is the 
last epoch in the queue, its status changes will not affect previous epochs. 
And the previous epochs status would be updated on `completeOneRecord()`.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to