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 <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.
+ */
+@Internal
+public class InternalTimerServiceAsyncImpl<K, N> extends 
InternalTimerServiceImpl<K, N> {
+
+    private AsyncExecutionController<K> asyncExecutionController;
+
+    InternalTimerServiceAsyncImpl(
+            TaskIOMetricGroup taskIOMetricGroup,
+            KeyGroupRange localKeyGroupRange,
+            KeyContext keyContext,
+            ProcessingTimeService processingTimeService,
+            KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, N>> 
processingTimeTimersQueue,
+            KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, N>> 
eventTimeTimersQueue,
+            StreamTaskCancellationContext cancellationContext,
+            AsyncExecutionController<K> 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 @@ <N> InternalTimerServiceImpl<K, N> 
registerOrGetTimerService(
         return timerService;
     }
 
+    @Override
+    public <N> InternalTimerService<N> getAsyncInternalTimerService(
+            String name,
+            TypeSerializer<K> keySerializer,
+            TypeSerializer<N> namespaceSerializer,
+            Triggerable<K, N> triggerable,
+            AsyncExecutionController<K> asyncExecutionController) {
+        checkNotNull(keySerializer, "Timers can only be used on keyed 
operators.");
+
+        // the following casting is to overcome type restrictions.
+        TimerSerializer<K, N> timerSerializer =
+                new TimerSerializer<>(keySerializer, namespaceSerializer);
+
+        InternalTimerServiceAsyncImpl<K, N> timerService =
+                registerOrGetAsyncTimerService(name, timerSerializer, 
asyncExecutionController);
+
+        timerService.startTimerService(
+                timerSerializer.getKeySerializer(),
+                timerSerializer.getNamespaceSerializer(),
+                triggerable);
+
+        return timerService;
+    }
+
+    <N> InternalTimerServiceAsyncImpl<K, N> registerOrGetAsyncTimerService(
+            String name,
+            TimerSerializer<K, N> timerSerializer,
+            AsyncExecutionController<K> asyncExecutionController) {
+        InternalTimerServiceAsyncImpl<K, N> timerService =
+                (InternalTimerServiceAsyncImpl<K, N>) timerServices.get(name);
+        if (timerService == null) {
+

Review Comment:
   nit. remove this empty line



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