rkhachatryan commented on a change in pull request #10151: [FLINK-14231] Handle
the processing-time timers when closing operators to make endInput semantics on
the operator chain strict
URL: https://github.com/apache/flink/pull/10151#discussion_r355390921
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeServiceImpl.java
##########
@@ -39,11 +78,282 @@ public long getCurrentProcessingTime() {
@Override
public ScheduledFuture<?> registerTimer(long timestamp,
ProcessingTimeCallback target) {
- return timerService.registerTimer(timestamp,
processingTimeCallbackWrapper.apply(target));
+ if (isQuiesced()) {
+ return new NeverCompleteFuture(
+
ProcessingTimeServiceUtil.getProcessingTimeDelay(timestamp,
getCurrentProcessingTime()));
+ }
+
+ final TimerScheduledFuture<?> timer = new
TimerScheduledFuture<Void>(false, this::removeUndoneTimer);
+ undoneTimers.put(timer, Boolean.TRUE);
+
+ // double check to deal with the following race conditions:
+ // 1. canceling timers from the undone table occurs before
putting this timer into the undone table
+ // (see #cancelTimersNotInExecuting())
+ // 2. using the size of the undone table to determine if all
timers have done occurs before putting
+ // this timer into the undone table (see
#tryCompleteTimersDoneFutureIfQuiesced())
+ if (isQuiesced()) {
+ removeUndoneTimer(timer);
+ return new NeverCompleteFuture(
+
ProcessingTimeServiceUtil.getProcessingTimeDelay(timestamp,
getCurrentProcessingTime()));
+ }
+
+ timer.bind(timerService.registerTimer(timestamp,
timer.getCallback(processingTimeCallbackWrapper.apply(target))));
+
+ return timer;
}
@Override
public ScheduledFuture<?> scheduleAtFixedRate(ProcessingTimeCallback
callback, long initialDelay, long period) {
- return
timerService.scheduleAtFixedRate(processingTimeCallbackWrapper.apply(callback),
initialDelay, period);
+ if (isQuiesced()) {
+ return new NeverCompleteFuture(initialDelay);
+ }
+
+ final TimerScheduledFuture<?> timer = new
TimerScheduledFuture<Void>(true, this::removeUndoneTimer);
+ undoneTimers.put(timer, Boolean.TRUE);
+
+ // double check to deal with the following race conditions:
+ // 1. canceling timers from the undone table occurs before
putting this timer into the undone table
+ // (see #cancelTimersNotInExecuting())
+ // 2. using the size of the undone table to determine if all
timers have done occurs before putting
+ // this timer into the undone table (see
#tryCompleteTimersDoneFutureIfQuiesced())
+ if (isQuiesced()) {
+ removeUndoneTimer(timer);
+ return new NeverCompleteFuture(initialDelay);
+ }
+
+ timer.bind(
+ timerService.scheduleAtFixedRate(
+
timer.getCallback(processingTimeCallbackWrapper.apply(callback)), initialDelay,
period));
+
+ return timer;
+ }
+
+ void quiesce() {
+ status.compareAndSet(STATUS_ALIVE, STATUS_QUIESCED);
+ }
+
+ /**
+ * This is an idempotent method to allow to repeatedly call.
+ */
+ CompletableFuture<Void> cancelTimersGracefullyAfterQuiesce() {
+ checkState(status.get() == STATUS_QUIESCED);
+
+ if (!timersDoneFutureAfterQuiescing.isDone()) {
+ if (!cancelTimersNotInExecuting()) {
+ return FutureUtils.completedExceptionally(new
CancellationException("Cancel timers failed"));
+ }
+ tryCompleteTimersDoneFutureIfQuiesced();
+ }
+
+ return timersDoneFutureAfterQuiescing;
+ }
+
+ @VisibleForTesting
+ int getNumUndoneTimers() {
+ return undoneTimers.size();
+ }
+
+ private boolean isQuiesced() {
+ return status.get() == STATUS_QUIESCED;
+ }
+
+ private void removeUndoneTimer(TimerScheduledFuture<?> timer) {
+ undoneTimers.remove(timer);
+ tryCompleteTimersDoneFutureIfQuiesced();
+ }
+
+ private void tryCompleteTimersDoneFutureIfQuiesced() {
+ if (isQuiesced() && getNumUndoneTimers() == 0) {
+ timersDoneFutureAfterQuiescing.complete(null);
+ }
+ }
+
+ private boolean cancelTimersNotInExecuting() {
+ // we should cancel the timers in descending timestamp order
+ TimerScheduledFuture<?>[] timers =
undoneTimers.keySet().toArray(new TimerScheduledFuture<?>[0]);
+ Arrays.sort(timers, Comparator.reverseOrder());
+ for (TimerScheduledFuture<?> timer : timers) {
+ if (!timer.cancel(false) &&
!timer.canDoneIfCancellationFailure()) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ static final class TimerScheduledFuture<V> implements
ScheduledFuture<V> {
+
+ private static final int STATUS_PENDING = 0;
+ private static final int STATUS_EXECUTING = 1;
+ private static final int STATUS_CANCELLED = 2;
+ private static final int STATUS_FAILED = 3;
+ private static final int STATUS_SUCCEEDED = 4;
+
Review comment:
I'd rather have a set of states as `enum`.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services