Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]
fredia commented on PR #24672: URL: https://github.com/apache/flink/pull/24672#issuecomment-2069427265 Thanks all for the detailed review, merged -- 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-35028][runtime] Timer firing under async execution model [flink]
fredia merged PR #24672: URL: https://github.com/apache/flink/pull/24672 -- 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-35028][runtime] Timer firing under async execution model [flink]
fredia commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1574062845 ## 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 https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model#FLIP425:AsynchronousExecutionModel-Timers>FLIP-425 + * timers section. + * @param Type of timer's key. + * @param Type of the namespace to which timers are scoped. + */ +public class InternalTimerServiceAsyncImpl extends InternalTimerServiceImpl { + +private AsyncExecutionController asyncExecutionController; + +InternalTimerServiceAsyncImpl( +TaskIOMetricGroup taskIOMetricGroup, +KeyGroupRange localKeyGroupRange, +KeyContext keyContext, +ProcessingTimeService processingTimeService, +KeyGroupedInternalPriorityQueue processingTimeTimersQueue, +KeyGroupedInternalPriorityQueue eventTimeTimersQueue, +StreamTaskCancellationContext cancellationContext, +AsyncExecutionController 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 timer; + +while ((timer = processingTimeTimersQueue.peek()) != null +&& timer.getTimestamp() <= time +&& !cancellationContext.isCancelled()) { +RecordContext recordCtx = +asyncExecutionController.buildContext(EMPTY_RECORD, timer.getKey()); +recordCtx.retain(); +asyncExecutionController.setCurrentContext(recordCtx); +keyContext.setCurrentKey(timer.getKey()); +processingTimeTimersQueue.poll(); +final InternalTimer 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 advance
Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]
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 ThrowingConsumer, Exception> getRecordProcessor + " since this part is handled by the Input."); } +@Override +@SuppressWarnings("unchecked") +public InternalTimerService getInternalTimerService( +String name, TypeSerializer namespaceSerializer, Triggerable 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 https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model#FLIP425:AsynchronousExecutionModel-Timers>FLIP-425 + * timers section. + * @param Type of timer's key. + * @param Type of the namespace to which timers are scoped. + */ +public class InternalTimerServiceAsyncImpl extends InternalTimerServiceImpl { + +private AsyncExecutionController asyncExecutionController; + +InternalTimerServiceAsyncImpl( +TaskIOMetricGroup taskIOMetricGroup, +KeyGroupRange localKeyGroupRange, +KeyContext keyContext, +ProcessingTimeService processingTimeService, +KeyGroupedInternalPriorityQueue processingTimeTimersQueue, +KeyGroupedInternalPriorityQueue eventTimeTimersQueue, +StreamTaskCancellationContext cancellationContext, +AsyncExecutionController 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 timer; + +while ((timer = processingTimeTimersQueue.peek()) != null +&& timer.getTimestamp() <= time +&& !cancellationContext.isCancelled()) { +RecordContext recordCtx = +asyncExecutionController.buildContext(EMPTY_RECORD, timer.getKey()); +recordCtx.retain(); +asyncExecutionController.setCurrentContext(recordCtx); +keyContext.setCurrentKey(timer.getKey()); +processingTimeTimersQueue.poll(); +final InternalTimer timerToTrigger = timer; +
Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]
fredia commented on PR #24672: URL: https://github.com/apache/flink/pull/24672#issuecomment-2066225062 @flinkbot run azure -- 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-35028][runtime] Timer firing under async execution model [flink]
Zakelly commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1571754617 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceAsyncImpl.java: ## @@ -0,0 +1,142 @@ +/* + * 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.annotation.Internal; +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 org.apache.flink.util.function.ThrowingRunnable; + +/** + * 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 https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model#FLIP425:AsynchronousExecutionModel-Timers>FLIP-425 + * timers section. + * @param Type of timer's key. + * @param Type of the namespace to which timers are scoped. + */ +@Internal +public class InternalTimerServiceAsyncImpl extends InternalTimerServiceImpl { + +private AsyncExecutionController asyncExecutionController; + +InternalTimerServiceAsyncImpl( +TaskIOMetricGroup taskIOMetricGroup, +KeyGroupRange localKeyGroupRange, +KeyContext keyContext, +ProcessingTimeService processingTimeService, +KeyGroupedInternalPriorityQueue> processingTimeTimersQueue, +KeyGroupedInternalPriorityQueue> eventTimeTimersQueue, +StreamTaskCancellationContext cancellationContext, +AsyncExecutionController asyncExecutionController) { +super( +taskIOMetricGroup, +localKeyGroupRange, +keyContext, +processingTimeService, +processingTimeTimersQueue, +eventTimeTimersQueue, +cancellationContext); +this.asyncExecutionController = asyncExecutionController; +} + +void onProcessingTime(long time) throws Exception { Review Comment: add `@Override` here? ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java: ## @@ -179,6 +180,55 @@ InternalTimerServiceImpl registerOrGetTimerService( return timerService; } +@Override +public InternalTimerService getAsyncInternalTimerService( +String name, +TypeSerializer keySerializer, +TypeSerializer namespaceSerializer, +Triggerable triggerable, +AsyncExecutionController asyncExecutionController) { +checkNotNull(keySerializer, "Timers can only be used on keyed operators."); + +// the following casting is to overcome type restrictions. +TimerSerializer timerSerializer = +new TimerSerializer<>(keySerializer, namespaceSerializer); + +InternalTimerServiceAsyncImpl timerService = +registerOrGetAsyncTimerService(name, timerSerializer, asyncExecutionController); + +timerService.startTimerService( +timerSerializer.getKeySerializer(), +timerSerializer.getNamespaceSerializer(), +triggerable); + +return timerService; +} + + InternalTimerServiceAsyncImpl registerOrGetAsyncTimerService( +String name, +TimerSerializer timerSerializer, +AsyncExecutionController asyncExecutionC
Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]
fredia commented on PR #24672: URL: https://github.com/apache/flink/pull/24672#issuecomment-2063204411 @Zakelly @yunfengzhou-hub Thanks for the detailed review, I updated the PR and addressed some comments, would you please take a look again? -- 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-35028][runtime] Timer firing under async execution model [flink]
fredia commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1570140913 ## 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 https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model#FLIP425:AsynchronousExecutionModel-Timers>FLIP-425 + * timers section. + * @param Type of timer's key. + * @param Type of the namespace to which timers are scoped. + */ +public class InternalTimerServiceAsyncImpl extends InternalTimerServiceImpl { + +private AsyncExecutionController asyncExecutionController; + +InternalTimerServiceAsyncImpl( +TaskIOMetricGroup taskIOMetricGroup, +KeyGroupRange localKeyGroupRange, +KeyContext keyContext, +ProcessingTimeService processingTimeService, +KeyGroupedInternalPriorityQueue processingTimeTimersQueue, +KeyGroupedInternalPriorityQueue eventTimeTimersQueue, +StreamTaskCancellationContext cancellationContext, +AsyncExecutionController 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 timer; + +while ((timer = processingTimeTimersQueue.peek()) != null +&& timer.getTimestamp() <= time +&& !cancellationContext.isCancelled()) { +RecordContext recordCtx = +asyncExecutionController.buildContext(EMPTY_RECORD, timer.getKey()); +recordCtx.retain(); +asyncExecutionController.setCurrentContext(recordCtx); +keyContext.setCurrentKey(timer.getKey()); +processingTimeTimersQueue.poll(); +final InternalTimer 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 advance
Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]
fredia commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1570100169 ## flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/KeyAccountingUnit.java: ## @@ -32,6 +32,8 @@ */ public class KeyAccountingUnit { +public static final Object EMPTY_RECORD = "EMPTY_RECORD"; Review Comment: The implementation of `KeyAccountingUnit ` restricts the value(record) from being null. I changed the `EMPTY_RECORD` to `new Object()`. -- 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-35028][runtime] Timer firing under async execution model [flink]
fredia commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1570075307 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java: ## @@ -117,6 +120,7 @@ public class InternalTimerServiceImpl implements InternalTimerService { startIdx = Math.min(keyGroupIdx, startIdx); } this.localKeyGroupRangeStartIdx = startIdx; +this.processingTimeCallback = this::onProcessingTime; Review Comment: Good suggestion👍, override `onProcessingTime` and delete `processingTimeCallback` now. -- 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-35028][runtime] Timer firing under async execution model [flink]
Zakelly commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1569904139 ## flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/KeyAccountingUnit.java: ## @@ -32,6 +32,8 @@ */ public class KeyAccountingUnit { +public static final Object EMPTY_RECORD = "EMPTY_RECORD"; Review Comment: IMO, this should be under `RecordContext` instead of `KeyAccountingUnit`. And how about using `aec.buildContext(null, key)` everywhere instead of import the EMPTY_RECORD everywhere? Further more, it might be better the `EMPTY_RECORD` is `new Object()` to avoid coincidence that happens to be the same as the user input. ## 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 https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model#FLIP425:AsynchronousExecutionModel-Timers>FLIP-425 + * timers section. + * @param Type of timer's key. + * @param Type of the namespace to which timers are scoped. + */ +public class InternalTimerServiceAsyncImpl extends InternalTimerServiceImpl { + +private AsyncExecutionController asyncExecutionController; + +InternalTimerServiceAsyncImpl( +TaskIOMetricGroup taskIOMetricGroup, +KeyGroupRange localKeyGroupRange, +KeyContext keyContext, +ProcessingTimeService processingTimeService, +KeyGroupedInternalPriorityQueue processingTimeTimersQueue, +KeyGroupedInternalPriorityQueue eventTimeTimersQueue, Review Comment: How about adding `>` as type parameter here? ## flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java: ## @@ -133,6 +140,48 @@ public final ThrowingConsumer, Exception> getRecordProcessor getClass().getName(), inputId)); } +/** + * Returns a {@link InternalTimerService} that can be used to query current processing time and + * event time and to set timers. An operator can have several timer services, where each has its + * own namespace serializer. Timer services are differentiated by the string key that is given + * when requesting them, if you call this method with the same key multiple times you will get + * the same timer service instance in subsequent requests. + * + * Timers are always scoped to a key, the currently active key of a keyed stream operation. + * When a timer fires, this key will also be set as the currently active key. + * + * Each timer has attached metadata, the namespace. Different timer services can have a + * different namespace type. If you don't need namespace differentiation you can use {@link + * org.apache.flink.runtime.state.VoidNamespaceSerializer} as the namespace serializer. + * + * @param name The name of the requested timer service. If no service e
Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]
yunfengzhou-hub commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1569902836 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java: ## @@ -117,6 +120,7 @@ public class InternalTimerServiceImpl implements InternalTimerService { startIdx = Math.min(keyGroupIdx, startIdx); } this.localKeyGroupRangeStartIdx = startIdx; +this.processingTimeCallback = this::onProcessingTime; Review Comment: How about making `onProcessingTime` a protected method? This way we won't need to introduce processingTimeCallback. ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManagerImpl.java: ## @@ -179,11 +180,60 @@ InternalTimerServiceImpl registerOrGetTimerService( return timerService; } +@Override +public InternalTimerService getAsyncInternalTimerService( +String name, +TypeSerializer keySerializer, +TypeSerializer namespaceSerializer, +Triggerable triggerable, +AsyncExecutionController asyncExecutionController) { +checkNotNull(keySerializer, "Timers can only be used on keyed operators."); + +// the following casting is to overcome type restrictions. +TimerSerializer timerSerializer = +new TimerSerializer<>(keySerializer, namespaceSerializer); + +InternalTimerServiceAsyncImpl timerService = +registerOrGetAsyncTimerService(name, timerSerializer, asyncExecutionController); + +timerService.startTimerService( +timerSerializer.getKeySerializer(), +timerSerializer.getNamespaceSerializer(), +triggerable); + +return timerService; +} + + InternalTimerServiceAsyncImpl registerOrGetAsyncTimerService( +String name, +TimerSerializer timerSerializer, +AsyncExecutionController asyncExecutionController) { +InternalTimerServiceAsyncImpl timerService = +(InternalTimerServiceAsyncImpl) timerServices.get(name); +if (timerService == null) { + +timerService = +new InternalTimerServiceAsyncImpl<>( +taskIOMetricGroup, +localKeyGroupRange, +keyContext, +processingTimeService, +createTimerPriorityQueue( +PROCESSING_TIMER_PREFIX + name, timerSerializer), +createTimerPriorityQueue(EVENT_TIMER_PREFIX + name, timerSerializer), +cancellationContext, +asyncExecutionController); + +timerServices.put(name, timerService); +} +return timerService; +} + Map> getRegisteredTimerServices() { return Collections.unmodifiableMap(timerServices); } -private +protected KeyGroupedInternalPriorityQueue> createTimerPriorityQueue( Review Comment: This method seems not used in subclasses, so `private` might be enough. Same for `restoreStateForKeyGroup`. -- 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-35028][runtime] Timer firing under async execution model [flink]
fredia commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1568752322 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerServiceAsyncImplTest.java: ## @@ -0,0 +1,219 @@ +/* + * 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.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController; +import org.apache.flink.runtime.asyncprocessing.RecordContext; +import org.apache.flink.runtime.asyncprocessing.StateExecutor; +import org.apache.flink.runtime.asyncprocessing.StateRequest; +import org.apache.flink.runtime.asyncprocessing.StateRequestType; +import org.apache.flink.runtime.mailbox.SyncMailboxExecutor; +import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.PriorityQueueSetFactory; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTaskCancellationContext; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CompletableFuture; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link InternalTimerServiceAsyncImpl}. */ +class InternalTimerServiceAsyncImplTest { Review Comment: Only the logic of `onProcessingTime` and `advanceWatermark` are changed, the logic of `register/delete` timer or `snapshot/restore` timer are same as `InternalTimerServiceImpl`, I think the `InternalTimerServiceAsyncImplTest` is enough, I tend to consider whether to inherit some tests when there are new changes in the future. -- 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-35028][runtime] Timer firing under async execution model [flink]
fredia commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1568731699 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java: ## @@ -117,6 +120,7 @@ public class InternalTimerServiceImpl implements InternalTimerService { startIdx = Math.min(keyGroupIdx, startIdx); } this.localKeyGroupRangeStartIdx = startIdx; +this.processingTimeCallback = this::onProcessingTime; Review Comment: The original idea was to avoid rewriting `startTimerService()` and `registerProcessingTimeTimer` by introducing `processingTimeCallback`, since it needs to be reassigned in the subclass, it cannot be marked as `final`. For the `onAsyncProcessingTime`, I changed it back to `onProcessingTime`, BTW, `onProcessingTime()` is a private method. -- 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-35028][runtime] Timer firing under async execution model [flink]
Zakelly commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1568659887 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java: ## @@ -51,6 +52,21 @@ InternalTimerService getInternalTimerService( TypeSerializer namespaceSerializer, Triggerable triggerable); +/** + * Creates an {@link InternalTimerServiceAsyncImpl} for handling a group of timers identified by + * the given {@code name}. The timers are scoped to a key and namespace. Mainly used by async + * operators. + * + * Some essential order preservation will be added when the given {@link Triggerable} is + * invoked. + */ + InternalTimerService getAsyncInternalTimerService( +String name, +TypeSerializer keySerializer, +TypeSerializer namespaceSerializer, +Triggerable triggerable, +AsyncExecutionController asyncExecutionController); Review Comment: It is better to provide the type parameter `K` from here and any other methods of this class. -- 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-35028][runtime] Timer firing under async execution model [flink]
yunfengzhou-hub commented on code in PR #24672: URL: https://github.com/apache/flink/pull/24672#discussion_r1568257919 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceAsyncImpl.java: ## @@ -0,0 +1,145 @@ +/* + * 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 https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model#FLIP425:AsynchronousExecutionModel-Timers>FLIP-425 + * timers section. + * @param Type of timer's key. + * @param Type of the namespace to which timers are scoped. + */ +public class InternalTimerServiceAsyncImpl extends InternalTimerServiceImpl { + +private AsyncExecutionController asyncExecutionController; + +InternalTimerServiceAsyncImpl( +TaskIOMetricGroup taskIOMetricGroup, +KeyGroupRange localKeyGroupRange, +KeyContext keyContext, +ProcessingTimeService processingTimeService, +KeyGroupedInternalPriorityQueue processingTimeTimersQueue, +KeyGroupedInternalPriorityQueue eventTimeTimersQueue, +StreamTaskCancellationContext cancellationContext, +AsyncExecutionController asyncExecutionController) { +super( +taskIOMetricGroup, +localKeyGroupRange, +keyContext, +processingTimeService, +processingTimeTimersQueue, +eventTimeTimersQueue, +cancellationContext); +this.asyncExecutionController = asyncExecutionController; +this.processingTimeCallback = this::onAsyncProcessingTime; +} + +private void onAsyncProcessingTime(long time) throws Exception { +// null out the timer in case the Triggerable calls registerProcessingTimeTimer() +// inside the callback. +nextTimer = null; + +InternalTimer timer; + +while ((timer = processingTimeTimersQueue.peek()) != null +&& timer.getTimestamp() <= time +&& !cancellationContext.isCancelled()) { +RecordContext recordCtx = +asyncExecutionController.buildContext(EMPTY_RECORD, timer.getKey()); +recordCtx.retain(); +asyncExecutionController.setCurrentContext(recordCtx); +keyContext.setCurrentKey(timer.getKey()); +processingTimeTimersQueue.poll(); +final InternalTimer timerToTrigger = timer; +asyncExecutionController.syncPointRequestWithCallback( +() -> triggerTarget.onProcessingTime(timerToTrigger)); +taskIOMetricGroup.getNumFiredTimers().inc(); +recordCtx.release(); +} + +if (timer != null && nextTimer == null) { +nextTimer = +processingTimeService.registerTimer( +timer.getTimestamp(), this::onAsyncProcessingTime); +} +} + +/** + * Advance one watermark, this will fire some event timers. + * + * @param time the time in watermark. + */ +public voi
Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]
flinkbot commented on PR #24672: URL: https://github.com/apache/flink/pull/24672#issuecomment-2059006200 ## CI report: * 6507dc0d38dec17e91fa3f722cd924ca82197622 UNKNOWN Bot commands The @flinkbot bot supports the following commands: - `@flinkbot run azure` re-run the last Azure build -- 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
[PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]
fredia opened a new pull request, #24672: URL: https://github.com/apache/flink/pull/24672 ## What is the purpose of the change This PR adapts timer firing to async state execution. The entrances are `AbstractAsyncStateStreamOperator ` and `AbstractAsyncStateStreamOperatorV2`, the hot path without this feature is not affected. ## Brief change log - Introduce `InternalTimerServiceAsyncImpl`. - Add `InternalTimeServiceManager#getAsyncInternalTimerService` method. - Override `getInternalTimerService` of `AbstractAsyncStateStreamOperator ` and `AbstractAsyncStateStreamOperatorV2` ## Verifying this change This change added tests and can be verified as follows: - `InternalTimerServiceAsyncImplTest` - `AbstractAsyncStateStreamOperatorTest#testTimerServiceIsAsync` - - `AbstractAsyncStateStreamOperatorV2Test#testTimerServiceIsAsync` ## Does this pull request potentially affect one of the following parts: - Dependencies (does it add or upgrade a dependency): (no) - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no) - The serializers: (no) - The runtime per-record code paths (performance sensitive): (no) - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no) - The S3 file system connector: (no) ## Documentation - Does this pull request introduce a new feature? (yes) - If yes, how is the feature documented? (JavaDocs) -- 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