|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.runtime.asyncprocessing; |
| 20 | + |
| 21 | +import org.slf4j.Logger; |
| 22 | +import org.slf4j.LoggerFactory; |
| 23 | + |
| 24 | +import javax.annotation.Nullable; |
| 25 | + |
| 26 | +import java.util.LinkedList; |
| 27 | + |
| 28 | +/** |
| 29 | + * Epoch manager segments inputs into distinct epochs, marked by the arrival of non-records(e.g. |
| 30 | + * watermark, record attributes). Records are assigned to a unique epoch based on their arrival, |
| 31 | + * records within an epoch are allowed to be parallelized, while the non-record of an epoch can only |
| 32 | + * be executed when all records in this epoch have finished. |
| 33 | + * |
| 34 | + * <p>For more details please refer to FLIP-425. |
| 35 | + */ |
| 36 | +public class EpochManager { |
| 37 | + private static final Logger LOG = LoggerFactory.getLogger(EpochManager.class); |
| 38 | + |
| 39 | + /** |
| 40 | + * This enum defines whether parallel execution between epochs is allowed. We should keep this |
| 41 | + * internal and away from API module for now, until we could see the concrete need for {@link |
| 42 | + * #PARALLEL_BETWEEN_EPOCH} from average users. |
| 43 | + */ |
| 44 | + public enum ParallelMode { |
| 45 | + /** |
| 46 | + * Subsequent epochs must wait until the previous epoch is completed before they can start. |
| 47 | + */ |
| 48 | + SERIAL_BETWEEN_EPOCH, |
| 49 | + /** |
| 50 | + * Subsequent epochs can begin execution even if the previous epoch has not yet completed. |
| 51 | + * Usually performs better than {@link #SERIAL_BETWEEN_EPOCH}. |
| 52 | + */ |
| 53 | + PARALLEL_BETWEEN_EPOCH |
| 54 | + } |
| 55 | + |
| 56 | + /** |
| 57 | + * The reference to the {@link AsyncExecutionController}, used for {@link |
| 58 | + * ParallelMode#SERIAL_BETWEEN_EPOCH}. Can be null when testing. |
| 59 | + */ |
| 60 | + final AsyncExecutionController<?> asyncExecutionController; |
| 61 | + |
| 62 | + /** The number of epochs that have arrived. */ |
| 63 | + long epochNum; |
| 64 | + |
| 65 | + /** The output queue to hold ongoing epochs. */ |
| 66 | + LinkedList<Epoch> outputQueue; |
| 67 | + |
| 68 | + /** Current active epoch, only one active epoch at the same time. */ |
| 69 | + Epoch activeEpoch; |
| 70 | + |
| 71 | + public EpochManager(AsyncExecutionController<?> aec) { |
| 72 | + this.epochNum = 0; |
| 73 | + this.outputQueue = new LinkedList<>(); |
| 74 | + this.asyncExecutionController = aec; |
| 75 | + // init an empty epoch, the epoch action will be updated when non-record is received. |
| 76 | + this.activeEpoch = new Epoch(epochNum++); |
| 77 | + } |
| 78 | + |
| 79 | + /** |
| 80 | + * Add a record to the current epoch and return the current open epoch, the epoch will be |
| 81 | + * associated with the {@link RecordContext} of this record. Must be invoked within task thread. |
| 82 | + * |
| 83 | + * @return the current open epoch. |
| 84 | + */ |
| 85 | + public Epoch onRecord() { |
| 86 | + activeEpoch.ongoingRecordCount++; |
| 87 | + return activeEpoch; |
| 88 | + } |
| 89 | + |
| 90 | + /** |
| 91 | + * Add a non-record to the current epoch, close current epoch and open a new epoch. Must be |
| 92 | + * invoked within task thread. |
| 93 | + * |
| 94 | + * @param action the action associated with this non-record. |
| 95 | + * @param parallelMode the parallel mode for this epoch. |
| 96 | + */ |
| 97 | + public void onNonRecord(Runnable action, ParallelMode parallelMode) { |
| 98 | + LOG.trace( |
| 99 | + "on NonRecord, old epoch: {}, outputQueue size: {}", |
| 100 | + activeEpoch, |
| 101 | + outputQueue.size()); |
| 102 | + switchActiveEpoch(action); |
| 103 | + if (parallelMode == ParallelMode.SERIAL_BETWEEN_EPOCH) { |
| 104 | + asyncExecutionController.drainInflightRecords(0); |
| 105 | + } |
| 106 | + } |
| 107 | + |
| 108 | + /** |
| 109 | + * Complete one record in the specific epoch. Must be invoked within task thread. |
| 110 | + * |
| 111 | + * @param epoch the specific epoch |
| 112 | + */ |
| 113 | + public void completeOneRecord(Epoch epoch) { |
| 114 | + if (--epoch.ongoingRecordCount == 0) { |
| 115 | + tryFinishInQueue(); |
| 116 | + } |
| 117 | + } |
| 118 | + |
| 119 | + private void tryFinishInQueue() { |
| 120 | + // If one epoch has been closed before and all records in |
| 121 | + // this epoch have finished, the epoch will be removed from the output queue. |
| 122 | + while (!outputQueue.isEmpty() && outputQueue.peek().tryFinish()) { |
| 123 | + LOG.trace( |
| 124 | + "Finish epoch: {}, outputQueue size: {}", |
| 125 | + outputQueue.peek(), |
| 126 | + outputQueue.size()); |
| 127 | + outputQueue.pop(); |
| 128 | + } |
| 129 | + } |
| 130 | + |
| 131 | + private void switchActiveEpoch(Runnable action) { |
| 132 | + activeEpoch.close(action); |
| 133 | + outputQueue.offer(activeEpoch); |
| 134 | + this.activeEpoch = new Epoch(epochNum++); |
| 135 | + tryFinishInQueue(); |
| 136 | + } |
| 137 | + |
| 138 | + /** The status of an epoch, see Fig.6 in FLIP-425 for details. */ |
| 139 | + enum EpochStatus { |
| 140 | + /** |
| 141 | + * The subsequent non-record input has not arrived. So arriving records will be collected |
| 142 | + * into current epoch. |
| 143 | + */ |
| 144 | + OPEN, |
| 145 | + /** |
| 146 | + * The records belong to this epoch is settled since the following non-record input has |
| 147 | + * arrived, the newly arriving records would be collected into the next epoch. |
| 148 | + */ |
| 149 | + CLOSED, |
| 150 | + /** |
| 151 | + * One epoch can only be finished when it meets the following three conditions. 1. The |
| 152 | + * records of this epoch have finished execution. 2. The epoch is closed. 3. The epoch is in |
| 153 | + * the front of outputQueue. |
| 154 | + */ |
| 155 | + FINISHED |
| 156 | + } |
| 157 | + |
| 158 | + /** |
| 159 | + * All inputs are segment into distinct epochs, marked by the arrival of non-record inputs. |
| 160 | + * Records are assigned to a unique epoch based on their arrival. |
| 161 | + */ |
| 162 | + public static class Epoch { |
| 163 | + /** The id of this epoch for easy debugging. */ |
| 164 | + long id; |
| 165 | + /** The number of records that are still ongoing in this epoch. */ |
| 166 | + int ongoingRecordCount; |
| 167 | + |
| 168 | + /** The action associated with non-record of this epoch(e.g. advance watermark). */ |
| 169 | + @Nullable Runnable action; |
| 170 | + |
| 171 | + EpochStatus status; |
| 172 | + |
| 173 | + public Epoch(long id) { |
| 174 | + this.id = id; |
| 175 | + this.ongoingRecordCount = 0; |
| 176 | + this.status = EpochStatus.OPEN; |
| 177 | + this.action = null; |
| 178 | + } |
| 179 | + |
| 180 | + /** |
| 181 | + * Try to finish this epoch. |
| 182 | + * |
| 183 | + * @return whether this epoch has been finished. |
| 184 | + */ |
| 185 | + boolean tryFinish() { |
| 186 | + if (this.status == EpochStatus.FINISHED) { |
| 187 | + return true; |
| 188 | + } |
| 189 | + if (ongoingRecordCount == 0 && this.status == EpochStatus.CLOSED) { |
| 190 | + this.status = EpochStatus.FINISHED; |
| 191 | + if (action != null) { |
| 192 | + action.run(); |
| 193 | + } |
| 194 | + return true; |
| 195 | + } |
| 196 | + return false; |
| 197 | + } |
| 198 | + |
| 199 | + /** Close this epoch. */ |
| 200 | + void close(Runnable action) { |
| 201 | + this.action = action; |
| 202 | + this.status = EpochStatus.CLOSED; |
| 203 | + } |
| 204 | + |
| 205 | + public String toString() { |
| 206 | + return String.format( |
| 207 | + "Epoch{id=%d, ongoingRecord=%d, status=%s}", id, ongoingRecordCount, status); |
| 208 | + } |
| 209 | + } |
| 210 | +} |
0 commit comments