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 <a
+ *     
href=https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model#FLIP425:AsynchronousExecutionModel-Timers>FLIP-425
+ *     timers section.
+ * @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 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<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::onAsyncProcessingTime);
+        }
+    }
+
+    /**
+     * Advance one watermark, this will fire some event timers.
+     *
+     * @param time the time in watermark.
+     */
+    public void advanceWatermark(long time) throws Exception {

Review Comment:
   nit: it might be better to add `@Override` to this and the following method.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java:
##########
@@ -133,6 +140,40 @@ public final <T> ThrowingConsumer<StreamRecord<T>, 
Exception> getRecordProcessor
                         getClass().getName(), inputId));
     }
 
+    /**
+     * Returns a {@link 
org.apache.flink.streaming.api.operators.InternalTimerServiceAsyncImpl} 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.
+     *
+     * <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, and a {@link
+     * RecordContext} will also be set as the current processing context to 
preserve record order.

Review Comment:
   It might be better to keep descriptions to implementation details as inline 
comments instead of as JavaDocs.



##########
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:
   It might be better to avoid introducing `processingTimeCallback`, or 
introduce it and make it a final variable. Given that the PR has already reused 
the name `advanceWatermark` in `InternalTimerServiceAsyncImpl` instead of using 
`asyncAdvanceWatermark`, it might be better to rename `onAsyncProcessingTime` 
back to `onProcessingTime` as well.



##########
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:
   It might be better to make this class extend from 
`InternalTimerServiceImplTest` to reuse the existing test cases.



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