Zakelly commented on code in PR #25773:
URL: https://github.com/apache/flink/pull/25773#discussion_r1879329642
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceAsyncImpl.java:
##########
@@ -97,20 +102,27 @@ void onProcessingTime(long time) throws Exception {
* @param time the time in watermark.
*/
@Override
- public void advanceWatermark(long time) throws Exception {
+ public CompletableFuture<Void> advanceWatermark(long time) throws
Exception {
currentWatermark = time;
-
+ List<CompletableFuture<Void>> futures = new ArrayList<>();
InternalTimer<K, N> timer;
-
while ((timer = eventTimeTimersQueue.peek()) != null
&& timer.getTimestamp() <= time
&& !cancellationContext.isCancelled()) {
eventTimeTimersQueue.poll();
final InternalTimer<K, N> timerToTrigger = timer;
+ CompletableFuture<Void> future = new CompletableFuture<>();
maintainContextAndProcess(
- timerToTrigger, () ->
triggerTarget.onEventTime(timerToTrigger));
+ timerToTrigger, () ->
triggerTarget.onEventTime(timerToTrigger))
+ .thenAccept(v -> future.complete(null));
+ futures.add(future);
taskIOMetricGroup.getNumFiredTimers().inc();
}
+ CompletableFuture<?>[] futuresArray = new
CompletableFuture[futures.size()];
+ for (int i = 0; i < futures.size(); i++) {
+ futuresArray[i] = futures.get(i);
+ }
+ return CompletableFuture.allOf(futuresArray);
Review Comment:
```suggestion
return CompletableFuture.allOf(futures.toArray(new
CompletableFuture[0]));
```
##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##########
@@ -387,6 +390,14 @@ public void drainInflightRecords(int targetNum) {
}
}
+ /** A helper function to drain in-flight requests emitted by timer. */
+ public void drainWithTimerIfNeeded(CompletableFuture<Void> timerFuture) {
+ if (epochParallelMode == ParallelMode.SERIAL_BETWEEN_EPOCH) {
+ drainInflightRecords(0);
+ assert timerFuture.isDone();
Review Comment:
Hmmmm.... How about using the `Preconditions.checkState()`
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]