Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-16 Thread via GitHub


fredia merged PR #24748:
URL: https://github.com/apache/flink/pull/24748


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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-16 Thread via GitHub


fredia commented on PR #24748:
URL: https://github.com/apache/flink/pull/24748#issuecomment-2115075350

   Thanks all for the detailed review! 


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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-16 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/EpochManager.java:
##
@@ -0,0 +1,207 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.
+ *
+ * For more details please refer to FLIP-425.
+ */
+public class EpochManager {
+private static final Logger LOG = 
LoggerFactory.getLogger(EpochManager.class);
+
+/**
+ * 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 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<>();
+this.asyncExecutionController = aec;
+// init an empty epoch, the epoch action will be updated when 
non-record is received.
+initNewActiveEpoch();
+}
+
+/**
+ * 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.SERIAL_BETWEEN_EPOCH) {
+asyncExecutionController.drainInflightRecords(0);

Review Comment:
   > When doing drainInflightRecords, will the activeEpoch's ongoingRecordCount 
reached 0 and `completeOneRecord` called?
   
   yes, `completeOneRecord` would be called.
   I changed the logic to:  close `activeEpoch` first, then drain in-flight 
records. The open active epoch would not be updated when doing 
`drainInflightRecords`, and updating `closed` `epoch#ongoingRecordCount` is 
allowed.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-15 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/EpochManager.java:
##
@@ -0,0 +1,207 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.
+ *
+ * For more details please refer to FLIP-425.
+ */
+public class EpochManager {
+private static final Logger LOG = 
LoggerFactory.getLogger(EpochManager.class);
+
+/**
+ * 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 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<>();
+this.asyncExecutionController = aec;
+// init an empty epoch, the epoch action will be updated when 
non-record is received.
+initNewActiveEpoch();
+}
+
+/**
+ * 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.SERIAL_BETWEEN_EPOCH) {
+asyncExecutionController.drainInflightRecords(0);

Review Comment:
   When doing `drainInflightRecords`, will the `activeEpoch`'s 
`ongoingRecordCount` reached 0 and `completeOneRecord` called?
   
   I'd suggest not do drain here. Instead we'd better mark blocking in `AEC`, 
while the callback is to unblock the `AEC`



##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##
@@ -76,6 +77,9 @@ public class AsyncExecutionController implements 
StateRequestHandler {
  */
 private final MailboxExecutor mailboxExecutor;
 
+/** Exception handler to handle the exception thrown by asynchronous 
framework. */
+private AsyncFrameworkExceptionHandler exceptionHandler;

Review Comment:
   nit. make it `final`?



##

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-15 Thread via GitHub


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


##
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperatorV2.java:
##
@@ -188,6 +190,35 @@ public  InternalTimerService 
getInternalTimerService(
 (AsyncExecutionController) asyncExecutionController);
 }
 
+@Override
+public void processWatermark(Watermark mark) throws Exception {
+if (!isAsyncStateProcessingEnabled()) {
+super.processWatermark(mark);
+return;
+}
+asyncExecutionController.processNonRecord(() -> 
super.processWatermark(mark));

Review Comment:
   `AbstractStreamOperatorV2` does not implement `processWatermark1 `,  
`processWatermark2` and `processWatermark(Watermark, int)`, if those methods 
will be added in the future, I would override them.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-14 Thread via GitHub


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


##
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperatorV2.java:
##
@@ -188,6 +190,35 @@ public  InternalTimerService 
getInternalTimerService(
 (AsyncExecutionController) asyncExecutionController);
 }
 
+@Override
+public void processWatermark(Watermark mark) throws Exception {
+if (!isAsyncStateProcessingEnabled()) {
+super.processWatermark(mark);
+return;
+}
+asyncExecutionController.processNonRecord(() -> 
super.processWatermark(mark));

Review Comment:
   We may need to override `processWatermark1` and `processWatermark2` as well, 
or we can override `processWatermark(Watermark, int)`, like that for 
`processWatermarkStatus`. I also understand it that this PR is mainly 
responsible for introducing the epoch mechanism, and that we would have another 
jira ticket and PR to apply epoch to all events and all cases. So it is also OK 
for me if you would like to make the change in the next PR.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-13 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##
@@ -102,6 +104,12 @@ public class AsyncExecutionController implements 
StateRequestHandler {
 /** Max parallelism of the job. */
 private final int maxParallelism;
 
+/** The reference of epoch manager. */
+final EpochManager epochManager;
+
+/** The parallel mode of epoch execution. */
+ParallelMode epochParallelMode = ParallelMode.SERIAL_BETWEEN_EPOCH;

Review Comment:
   It seems that this parameter can be removed.



##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##
@@ -311,6 +324,22 @@ public void drainInflightRecords(int targetNum) {
 }
 }
 
+public void processNonRecord(Runnable action) {
+epochManager.onNonRecord(action, epochParallelMode);
+}
+
+/**
+ * Configure the parallel mode of epoch execution. We should keep this 
method internal for now,
+ * until we could see the concrete need for {@link 
ParallelMode#PARALLEL_BETWEEN_EPOCH} from
+ * average users.
+ *
+ * @param parallelMode the parallel mode to set.
+ */
+@Internal
+void configureEpochParallelMode(ParallelMode parallelMode) {

Review Comment:
   Same as 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.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-12 Thread via GitHub


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


##
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.
+ *
+ * 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 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:
   Got it. Given that we agreed to add 
`AsyncExecutionController.processNonRecord` in another comment and that 
`AsyncExecutionController` has `exceptionHandler`, how about keeping 
EpochManager using Runnable, and change 
`AsyncExecutionController.processNonRecord` to use ThrowingRunnable?



##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##
@@ -311,6 +324,22 @@ public void drainInflightRecords(int targetNum) {
 }
 }
 
+public void processNonRecord(Runnable action) {
+epochManager.onNonRecord(action, epochParallelMode);
+}
+
+/**
+ * Configure the parallel mode of epoch execution. We should keep this 
method internal for now,
+ * until we could see the concrete need for {@link 
ParallelMode#PARALLEL_BETWEEN_EPOCH} from
+ * average users.
+ *
+ * @param parallelMode the parallel mode to set.
+ */
+@Internal
+void configureEpochParallelMode(ParallelMode parallelMode) 

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-10 Thread via GitHub


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


##
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.
+ *
+ * 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 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:
   Because there is no exception in the signature of `disposeContext()` , if we 
change Runnable to ThrowingRunnable here, we need to catch the exception in 
`aec.disposeContext()` or  `epochManager.completeOneRecord()`.
   
   I think that catching exceptions in `StreamOperator` is more convenient



##
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:
   Thanks for the suggestion, I added `processNonRecord()` for aec, 
`EpochManager` is not visible for operators now. 



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

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
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.
+ *
+ * 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:
   I think  "out of order" is a bit too broad, so I changed it to the current 
name.
   As the 
[example](https://github.com/apache/flink/pull/24748#discussion_r1596141360 ) 
you mentioned, "out of order" can't describe the scenario that watermark1  
executes after record 0-3.
   
If there is a better way to describe "out of order", I am happy to change 
it.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
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:
   For `Parallel mode`, we don't guarantee that watermark 0 must have been 
finished before watermark 1.
   
   For `Serial mode`, there is only one epoch in the queue at the same time, 
which provides the above guarantee as you mentioned.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
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.
+ *
+ * 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 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:
   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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
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);
+} catch (Exception e) {
+handleAsyncStateException(
+String.format("Failed to process 
watermark %s.", mark), e);
+}
+},
+ParallelMode.SERIAL_BETWEEN_EPOCH);

Review Comment:
   My suggestion:
   1. `ParallelMode` could be configured within `EpochManager` or `AEC`, and 
could be overridden from outside.
   2. `AEC` should proxy the requests to `EpochManager`, the operator does not 
know the details.



##
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.
+ *
+ * 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 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.
+ *
+ 

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
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();
+if (outputQueue.size() == 1) { // which means the first epoch
+if (lastEpoch.tryFinish()) {
+outputQueue.remove(0);
+}
+}
+Epoch epoch = new Epoch(0, action);

Review Comment:
   fixed.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
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:
   Yes, it is, I checked `outputQueue.size() == 1` here.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##
@@ -130,11 +152,17 @@ public AsyncExecutionController(
 },
 "AEC-buffer-timeout"));
 
+if (parallelMode == ParallelMode.SERIAL_BETWEEN_EPOCH) {

Review Comment:
   Yes, thanks for the suggestion, I put `ParallelMode` as one parameter of 
`onNonRecord()`, then we can choose different modes for different epochs.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java:
##
@@ -220,6 +222,40 @@ public  InternalTimerService 
getInternalTimerService(
 (AsyncExecutionController) asyncExecutionController);
 }
 
+@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);
+} catch (Exception e) {
+// todo: error handling
+}
+});
+}
+
+@Override
+public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws 
Exception {

Review Comment:
   `InternalWatermark` also uses `AbstractStreamOperator#processWatermark`.
   For `LatencyMarker` `EndOfDataMarker` and `RecordAttributes`, I plan to do 
it in FLINK-35031.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


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

Review Comment:
   I removed `ParallelEpochManager` and `SerialEpochManager` as Zakelly's 
suggestion, the constructor was also removed.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
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:
   I noticed that `lastEpoch.close();` is invoked right before this block of 
code, so this epoch must have been closed.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
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:
   This block of code is for the case:
   all records in one epoch are completed before non-records arriving, 
`completeOneRecord()` can't finish this epoch cause this epoch is not closed, 
so we need to check it again here.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AbstractEpochManager.java:
##
@@ -0,0 +1,154 @@
+/*
+ * 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.Nonnull;
+
+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.
+ *
+ * For more details please refer to FLIP-425.
+ */
+public abstract class AbstractEpochManager {
+
+/**
+ * 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 output queue to hold ongoing epochs. */
+protected LinkedList outputQueue;
+
+public AbstractEpochManager() {
+this.outputQueue = new LinkedList<>();
+// preset an empty epoch, the epoch action will be updated when 
non-record is received.
+this.outputQueue.add(new Epoch(0, () -> {}));
+}
+
+/**
+ * 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() {
+Epoch lastEpoch = outputQueue.get(outputQueue.size() - 1);
+lastEpoch.ongoingRecordCount++;
+return lastEpoch;
+}
+
+/**
+ * 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.
+ */
+public abstract void onNonRecord(Runnable action);
+
+/**
+ * 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 this epoch has been closed before and all records in
+//  is epoch have finished, the epoch will be removed from the output 
queue.
+if (epoch.tryFinish() && outputQueue.size() > 0) {
+if (epoch == outputQueue.getFirst()) {
+outputQueue.remove(0);
+}
+}
+}
+
+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

Review Comment:
   Thanks for the suggestion, I will enrich the description here to mitigate 
confusion. I think Fig.6 in FLIP-425 is helpful for developers to understand 
these three `EpochStatus`.



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

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AbstractEpochManager.java:
##
@@ -0,0 +1,154 @@
+/*
+ * 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.Nonnull;
+
+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.
+ *
+ * For more details please refer to FLIP-425.
+ */
+public abstract class AbstractEpochManager {
+
+/**
+ * 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 output queue to hold ongoing epochs. */
+protected LinkedList outputQueue;
+
+public AbstractEpochManager() {
+this.outputQueue = new LinkedList<>();
+// preset an empty epoch, the epoch action will be updated when 
non-record is received.
+this.outputQueue.add(new Epoch(0, () -> {}));
+}
+
+/**
+ * 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() {
+Epoch lastEpoch = outputQueue.get(outputQueue.size() - 1);
+lastEpoch.ongoingRecordCount++;
+return lastEpoch;
+}
+
+/**
+ * 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.
+ */
+public abstract void onNonRecord(Runnable action);
+
+/**
+ * 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 this epoch has been closed before and all records in
+//  is epoch have finished, the epoch will be removed from the output 
queue.
+if (epoch.tryFinish() && outputQueue.size() > 0) {
+if (epoch == outputQueue.getFirst()) {
+outputQueue.remove(0);
+}
+}
+}
+
+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 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). */
+ 

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-09 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AbstractEpochManager.java:
##
@@ -0,0 +1,154 @@
+/*
+ * 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.Nonnull;
+
+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.
+ *
+ * For more details please refer to FLIP-425.
+ */
+public abstract class AbstractEpochManager {
+
+/**
+ * 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 output queue to hold ongoing epochs. */
+protected LinkedList outputQueue;
+
+public AbstractEpochManager() {
+this.outputQueue = new LinkedList<>();
+// preset an empty epoch, the epoch action will be updated when 
non-record is received.
+this.outputQueue.add(new Epoch(0, () -> {}));
+}
+
+/**
+ * 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() {
+Epoch lastEpoch = outputQueue.get(outputQueue.size() - 1);
+lastEpoch.ongoingRecordCount++;
+return lastEpoch;
+}
+
+/**
+ * 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.
+ */
+public abstract void onNonRecord(Runnable action);
+
+/**
+ * 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 this epoch has been closed before and all records in
+//  is epoch have finished, the epoch will be removed from the output 
queue.
+if (epoch.tryFinish() && outputQueue.size() > 0) {
+if (epoch == outputQueue.getFirst()) {
+outputQueue.remove(0);

Review Comment:
   right, I ignored the case that later epochs were completed first.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-08 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##
@@ -130,11 +152,17 @@ public AsyncExecutionController(
 },
 "AEC-buffer-timeout"));
 
+if (parallelMode == ParallelMode.SERIAL_BETWEEN_EPOCH) {

Review Comment:
   Whether two epochs can be executed in parallel depends on the semantics of 
the non-record in between, which means it is not a unified configuration that 
can be set on the whole operator.



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



Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-08 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AbstractEpochManager.java:
##
@@ -0,0 +1,154 @@
+/*
+ * 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.Nonnull;
+
+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.
+ *
+ * For more details please refer to FLIP-425.
+ */
+public abstract class AbstractEpochManager {
+
+/**
+ * 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 output queue to hold ongoing epochs. */
+protected LinkedList outputQueue;
+
+public AbstractEpochManager() {
+this.outputQueue = new LinkedList<>();
+// preset an empty epoch, the epoch action will be updated when 
non-record is received.
+this.outputQueue.add(new Epoch(0, () -> {}));
+}
+
+/**
+ * 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() {
+Epoch lastEpoch = outputQueue.get(outputQueue.size() - 1);
+lastEpoch.ongoingRecordCount++;
+return lastEpoch;
+}
+
+/**
+ * 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.
+ */
+public abstract void onNonRecord(Runnable action);
+
+/**
+ * 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 this epoch has been closed before and all records in
+//  is epoch have finished, the epoch will be removed from the output 
queue.

Review Comment:
   nit: is -> this



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

Re: [PR] [FLINK-35030][runtime] Introduce Epoch Manager under async execution [flink]

2024-05-07 Thread via GitHub


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


##
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();
+if (outputQueue.size() == 1) { // which means the first epoch
+if (lastEpoch.tryFinish()) {
+outputQueue.remove(0);
+}
+}
+Epoch epoch = new Epoch(0, action);

Review Comment:
   This action should not be passed into the new epoch.



##
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AbstractEpochManager.java:
##
@@ -0,0 +1,154 @@
+/*
+ * 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.Nonnull;
+
+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.
+ *
+ * For more details please refer to FLIP-425.
+ */
+public abstract class AbstractEpochManager {
+
+/**
+ * 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 output queue to hold ongoing epochs. */
+protected LinkedList outputQueue;
+
+public AbstractEpochManager() {
+this.outputQueue = new LinkedList<>();
+// preset an empty epoch, the epoch action will be updated when 
non-record is received.
+this.outputQueue.add(new Epoch(0, () -> {}));
+}
+
+/**
+ * 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() {
+Epoch lastEpoch = outputQueue.get(outputQueue.size() - 1);

Review Comment:
   How about hold a reference of `Epoch activeEpoch` instead of get the last 
one from queue?



##