yunfengzhou-hub commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1572300164
########## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperatorV2.java: ########## @@ -113,6 +120,26 @@ public final <T> ThrowingConsumer<StreamRecord<T>, Exception> getRecordProcessor + " since this part is handled by the Input."); } + @Override + @SuppressWarnings("unchecked") + public <K, N> InternalTimerService<N> getInternalTimerService( + String name, TypeSerializer<N> namespaceSerializer, Triggerable<K, N> triggerable) { Review Comment: A fallback plan when isAsyncStateProcessingEnabled=false might be needed here as well. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceAsyncImpl.java: ########## @@ -0,0 +1,146 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController; +import org.apache.flink.runtime.asyncprocessing.RecordContext; +import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTaskCancellationContext; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.function.BiConsumerWithException; + +import static org.apache.flink.runtime.asyncprocessing.KeyAccountingUnit.EMPTY_RECORD; + +/** + * An implementation of {@link InternalTimerService} that is used by {@link + * org.apache.flink.streaming.runtime.operators.asyncprocessing.AbstractAsyncStateStreamOperator}. + * The timer service will set {@link RecordContext} for the timers before invoking action to + * preserve the execution order between timer firing and records processing. + * + * @see <a + * href=https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model#FLIP425:AsynchronousExecutionModel-Timers>FLIP-425 + * timers section.</a> + * @param <K> Type of timer's key. + * @param <N> Type of the namespace to which timers are scoped. + */ +public class InternalTimerServiceAsyncImpl<K, N> extends InternalTimerServiceImpl<K, N> { + + private AsyncExecutionController<K> asyncExecutionController; + + InternalTimerServiceAsyncImpl( + TaskIOMetricGroup taskIOMetricGroup, + KeyGroupRange localKeyGroupRange, + KeyContext keyContext, + ProcessingTimeService processingTimeService, + KeyGroupedInternalPriorityQueue processingTimeTimersQueue, + KeyGroupedInternalPriorityQueue eventTimeTimersQueue, + StreamTaskCancellationContext cancellationContext, + AsyncExecutionController<K> asyncExecutionController) { + super( + taskIOMetricGroup, + localKeyGroupRange, + keyContext, + processingTimeService, + processingTimeTimersQueue, + eventTimeTimersQueue, + cancellationContext); + this.asyncExecutionController = asyncExecutionController; + this.processingTimeCallback = this::onProcessingTime; + } + + private void onProcessingTime(long time) throws Exception { + // null out the timer in case the Triggerable calls registerProcessingTimeTimer() + // inside the callback. + nextTimer = null; + + InternalTimer<K, N> timer; + + while ((timer = processingTimeTimersQueue.peek()) != null + && timer.getTimestamp() <= time + && !cancellationContext.isCancelled()) { + RecordContext<K> recordCtx = + asyncExecutionController.buildContext(EMPTY_RECORD, timer.getKey()); + recordCtx.retain(); + asyncExecutionController.setCurrentContext(recordCtx); + keyContext.setCurrentKey(timer.getKey()); + processingTimeTimersQueue.poll(); + final InternalTimer<K, N> timerToTrigger = timer; + asyncExecutionController.syncPointRequestWithCallback( + () -> triggerTarget.onProcessingTime(timerToTrigger)); + taskIOMetricGroup.getNumFiredTimers().inc(); + recordCtx.release(); + } + + if (timer != null && nextTimer == null) { + nextTimer = + processingTimeService.registerTimer( + timer.getTimestamp(), this::onProcessingTime); + } + } + + /** + * Advance one watermark, this will fire some event timers. + * + * @param time the time in watermark. + */ + @Override + public void advanceWatermark(long time) throws Exception { + currentWatermark = time; + + InternalTimer<K, N> timer; + + while ((timer = eventTimeTimersQueue.peek()) != null + && timer.getTimestamp() <= time + && !cancellationContext.isCancelled()) { + RecordContext<K> recordCtx = + asyncExecutionController.buildContext(EMPTY_RECORD, timer.getKey()); + recordCtx.retain(); + asyncExecutionController.setCurrentContext(recordCtx); + keyContext.setCurrentKey(timer.getKey()); + eventTimeTimersQueue.poll(); + final InternalTimer<K, N> timerToTrigger = timer; + asyncExecutionController.syncPointRequestWithCallback( + () -> triggerTarget.onEventTime(timerToTrigger)); + taskIOMetricGroup.getNumFiredTimers().inc(); + recordCtx.release(); + } + } + + protected void foreachTimer( Review Comment: How about throw UnsupportedException for now and add actual implementations later after the evaluation is complete? It might be better not to support certain functions than to support risky functions. -- 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