Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]

2024-04-22 Thread via GitHub


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]

2024-04-22 Thread via GitHub


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]

2024-04-21 Thread via GitHub


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 

Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]

2024-04-19 Thread via GitHub


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]

2024-04-19 Thread via GitHub


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]

2024-04-18 Thread via GitHub


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 

Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]

2024-04-18 Thread via GitHub


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]

2024-04-18 Thread via GitHub


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 

Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]

2024-04-18 Thread via GitHub


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]

2024-04-18 Thread via GitHub


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]

2024-04-17 Thread via GitHub


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 

Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]

2024-04-17 Thread via GitHub


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]

2024-04-17 Thread via GitHub


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]

2024-04-17 Thread via GitHub


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]

2024-04-17 Thread via GitHub


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]

2024-04-17 Thread via GitHub


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 

Re: [PR] [FLINK-35028][runtime] Timer firing under async execution model [flink]

2024-04-16 Thread via GitHub


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]

2024-04-16 Thread via GitHub


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