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. + * + * <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); Review Comment: How about hold a reference of `Epoch activeEpoch` instead of get the last one from queue? ########## 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 { Review Comment: Should this be a property of `Epoch`? Or should the draining logic be in the `close()` of some specific epochs? ########## 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); Review Comment: My suggestion: use `outputQueue.pop();` here. And there should be a ``` while (outputQueue.peek().isFinished()) { outputQueue.pop(); } ``` Right? ########## 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; Review Comment: How about make this `Nullable`? And provide one action for the `Epoch` on `close()`. ########## 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 { Review Comment: I suggest provide an auto-increment id for epoch, which is useful for debugging. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
