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. + * + * <p>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<Epoch> 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 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 should have been invoked in `completeOneRecord()` during `drainInFlightRecords`, thus could be removed. ########## 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: This constructor seems unnecessary. ########## 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. + * + * <p>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<Epoch> 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). */ + @Nonnull Runnable action; + + EpochStatus status; + + public Epoch(int recordCount, Runnable action) { + this.ongoingRecordCount = recordCount; + this.action = action; + this.status = EpochStatus.Open; + } + + /** + * Try to finish this epoch. + * + * @return whether this epoch has been finished. + */ + boolean tryFinish() { + if (ongoingRecordCount == 0 && this.status == EpochStatus.Closed) { + this.status = EpochStatus.Finished; + action.run(); + return true; + } + return false; Review Comment: If this method is invoked on an already finished epoch, the result should be true. ########## 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. + * + * <p>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<Epoch> 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). */ + @Nonnull Runnable action; + + EpochStatus status; + + public Epoch(int recordCount, Runnable action) { Review Comment: It seems that `recordCount` is always 0, thus this constructor parameter could be removed. ########## 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: The action should be invoked when ALL previous epochs have finished, instead of only the last one epoch. ########## 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. ########## 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. + * + * <p>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<Epoch> 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: In StreamOperators, `finish()` would be invoked prior to `close()`, while the design here is reversed. This might confuse other Flink developers. ########## 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); Review Comment: nit: get(0) is enough given that outputQueue.size() == 1. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java: ########## @@ -220,6 +222,40 @@ public <K, N> InternalTimerService<N> getInternalTimerService( (AsyncExecutionController<K>) 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: Apart from Watermark and WatermarkStatus, there are other events that could come through the data stream(See all subclasses of StreamElement and RuntimeEvent). This PR might need to handle all these events, and it needs some design how to treat all events in a unified way. -- 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