yunfengzhou-hub commented on code in PR #24748:
URL: https://github.com/apache/flink/pull/24748#discussion_r1596135042


##########
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();
+                }
+            }

Review Comment:
   Shall we use 
   ```java
           while (!outputQueue.isEmpty() && outputQueue.peek().tryFinish()) {
               outputQueue.pop();
           }
   ```
   here as well?



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

Review Comment:
   The naming here is slightly different from the nouns used in FLIP-425. How 
about naming them as "strict order" and "out of order"?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java:
##########
@@ -259,6 +262,44 @@ public Object getCurrentKey() {
         return currentProcessingContext.getKey();
     }
 
+    @Override
+    public void processWatermark(Watermark mark) throws Exception {
+        if (!isAsyncStateProcessingEnabled()) {
+            // If async state processing is disabled, fallback to the super 
class.
+            super.processWatermark(mark);
+            return;
+        }
+        asyncExecutionController
+                .getEpochManager()
+                .onNonRecord(
+                        () -> {
+                            try {
+                                if (timeServiceManager != null) {
+                                    timeServiceManager.advanceWatermark(mark);
+                                }
+                                output.emitWatermark(mark);

Review Comment:
   It might be simpler to use `super.processWatermark(mark)`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/ParallelEpochManager.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/** An implementation of {@link AbstractEpochManager} for parallel epoch 
execution. */
+public class ParallelEpochManager extends AbstractEpochManager {
+
+    public ParallelEpochManager() {
+        super();
+    }
+
+    @Override
+    public void onNonRecord(Runnable action) {
+        Epoch lastEpoch = outputQueue.get(outputQueue.size() - 1);
+        lastEpoch.action = action;
+        lastEpoch.close();

Review Comment:
   Thanks for adding the check. It just comes to me that the non-record might 
depend on not only all previous records, but also all previous non-record. Say 
we have an example data flow as follows
   - Epoch 0 records: record 0, record 1
   - Epoch 0 non-record: watermark 0
   - Epoch 1 records: record 2, record 3
   - Epoch 1 non-record: watermark 1
   
   In the current implementation, it is guaranteed that record 0~3 must have 
been finished before watermark 1 is processed. But actually we may also need to 
guarantee that watermark 0 must have been finished as well.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/asyncprocessing/EpochManagerTest.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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 org.apache.flink.runtime.asyncprocessing.EpochManager.Epoch;
+import org.apache.flink.runtime.asyncprocessing.EpochManager.EpochStatus;
+import org.apache.flink.runtime.asyncprocessing.EpochManager.ParallelMode;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+/** Test for Epoch Manager. */
+class EpochManagerTest {
+    @Test
+    void testBasic() {
+        EpochManager epochManager = new EpochManager(null);
+        Epoch epoch1 = epochManager.onRecord();
+        Epoch epoch2 = epochManager.onRecord();
+        assertThat(epoch1).isEqualTo(epoch2);
+        assertThat(epoch1.ongoingRecordCount).isEqualTo(2);
+        AtomicInteger output = new AtomicInteger(0);
+        epochManager.onNonRecord(
+                () -> {
+                    output.incrementAndGet();
+                },
+                ParallelMode.PARALLEL_BETWEEN_EPOCH);

Review Comment:
   Despite that `SERIAL_BETWEEN_EPOCH` have been covered in 
`AsyncExecutionControllerTest`, it might still be better to make 
`EpochManagerTest` cover this situation as well.



##########
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();
+            }
+            activeEpoch = new Epoch(epochNum++);
+            outputQueue.add(activeEpoch);
+        }
+    }
+
+    /**
+     * Complete one record in the specific epoch. Must be invoked within task 
thread.
+     *
+     * @param epoch the specific epoch
+     */
+    public void completeOneRecord(Epoch epoch) {
+        epoch.ongoingRecordCount--;
+        // If one epoch has been closed before and all records in
+        // this epoch have finished, the epoch will be removed from the output 
queue.
+        while (!outputQueue.isEmpty() && outputQueue.peek().tryFinish()) {
+            outputQueue.pop();
+        }
+    }
+
+    /** The status of an epoch, see Fig.6 in FLIP-425 for details. */
+    enum EpochStatus {
+        /**
+         * The subsequent non-record input has not arrived. So arriving 
records will be collected
+         * into current epoch.
+         */
+        Open,
+        /**
+         * The records belong to this epoch is settled since the following 
non-record input has
+         * arrived, the newly arriving records would be collected into the 
next epoch.
+         */
+        Closed,
+        /** The records of this epoch have finished execution after the epoch 
is closed. */
+        Finished
+    }
+
+    /**
+     * All inputs are segment into distinct epochs, marked by the arrival of 
non-record inputs.
+     * Records are assigned to a unique epoch based on their arrival.
+     */
+    public static class Epoch {
+        /** The id of this epoch for easy debugging. */
+        long id;
+        /** The number of records that are still ongoing in this epoch. */
+        int ongoingRecordCount;
+
+        /** The action associated with non-record of this epoch(e.g. advance 
watermark). */
+        @Nullable Runnable action;
+
+        EpochStatus status;
+
+        public Epoch(long id) {
+            this(id, null);
+        }
+
+        public Epoch(long id, Runnable action) {

Review Comment:
   Seems that this constructor can be removed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##########
@@ -311,6 +318,10 @@ public void drainInflightRecords(int targetNum) {
         }
     }
 
+    public EpochManager getEpochManager() {
+        return epochManager;

Review Comment:
   It seems that all usage of this method will be followed by `onNonRecord()`. 
Thus it might be more straightforward to expose a method like `onNonRecord` in 
this class. This would also be good for isolating implementation details.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/SerialEpochManager.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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;
+
+/** An implementation of {@link AbstractEpochManager} for serial epoch 
execution. */
+public class SerialEpochManager extends AbstractEpochManager {
+
+    AsyncExecutionController asyncExecutionController;
+
+    public SerialEpochManager(AsyncExecutionController 
asyncExecutionController) {
+        super();
+        this.asyncExecutionController = asyncExecutionController;
+    }
+
+    @Override
+    public void onNonRecord(Runnable action) {
+        assert outputQueue.size() == 1;
+        Epoch lastEpoch = outputQueue.get(outputQueue.size() - 1);
+        lastEpoch.action = action;
+        lastEpoch.close();
+        asyncExecutionController.drainInflightRecords(0);
+        if (lastEpoch.tryFinish() && outputQueue.size() > 0) {
+            outputQueue.remove(0);
+        }

Review Comment:
   Oh I understood the logic here. No problem for me now.



##########
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();
+            }
+            activeEpoch = new Epoch(epochNum++);
+            outputQueue.add(activeEpoch);
+        }
+    }
+
+    /**
+     * Complete one record in the specific epoch. Must be invoked within task 
thread.
+     *
+     * @param epoch the specific epoch
+     */
+    public void completeOneRecord(Epoch epoch) {
+        epoch.ongoingRecordCount--;
+        // If one epoch has been closed before and all records in
+        // this epoch have finished, the epoch will be removed from the output 
queue.
+        while (!outputQueue.isEmpty() && outputQueue.peek().tryFinish()) {
+            outputQueue.pop();
+        }
+    }
+
+    /** The status of an epoch, see Fig.6 in FLIP-425 for details. */
+    enum EpochStatus {
+        /**
+         * The subsequent non-record input has not arrived. So arriving 
records will be collected
+         * into current epoch.
+         */
+        Open,

Review Comment:
   It might be better to change all characters to uppercase.



##########
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();
+            }
+            activeEpoch = new Epoch(epochNum++);
+            outputQueue.add(activeEpoch);

Review Comment:
   nit: the common parts between these two cases can be extracted.



##########
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) {

Review Comment:
   How about changing Runnable to ThrowingRunnable? This can help reduce 
duplicated try-catch code blocks in invokers.



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