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<K> {
 
+    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 <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,

Review Comment:
   How about adding `<TimerHeapInternalTimer<K, N>>` 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 <T> ThrowingConsumer<StreamRecord<T>, 
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.
+     *
+     * <p>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.
+     *
+     * <p>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 
exists under the given
+     *     name a new one will be created and returned.
+     * @param namespaceSerializer {@code TypeSerializer} for the timer 
namespace.
+     * @param triggerable The {@link Triggerable} that should be invoked when 
timers fire
+     * @param <N> The type of the timer namespace.
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public <K, N> InternalTimerService<N> getInternalTimerService(
+            String name, TypeSerializer<N> namespaceSerializer, Triggerable<K, 
N> triggerable) {

Review Comment:
   Check `isAsyncStateProcessingEnabled()` first and provide fallback?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java:
##########
@@ -117,6 +120,7 @@ public class InternalTimerServiceImpl<K, N> implements 
InternalTimerService<N> {
             startIdx = Math.min(keyGroupIdx, startIdx);
         }
         this.localKeyGroupRangeStartIdx = startIdx;
+        this.processingTimeCallback = this::onProcessingTime;

Review Comment:
   Is it feasible to make `onProcessingTime` package-private and override this 
in `InternalTimerServiceAsyncImpl `?



##########
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:
   Actually, providing batch operation in async execution is dangerous. This 
should be carefully evaluated. We could discuss this later.



##########
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();

Review Comment:
   Can we extract the common part of `onProcessingTime`, `advanceWatermark` and 
`foreachTimer` into one helper method? e.g. `maintainContextAndProcess(K key, 
ThrowingRunnable<Exception> runnable)`.



-- 
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

Reply via email to