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_r355445719
##########
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;
+
+ //
------------------------------------------------------------------------
+
+ private final CompletableFuture<ScheduledFuture<?>>
wrappedFuture;
+
+ private final boolean isPeriodic;
+
+ private final CompletableFuture<TimerScheduledFuture<?>>
onDoneActionFuture;
+
+ /**
+ * This is used to control execution behavior to ensure that
the on-done action is executed
+ * after {@link ProcessingTimeCallback#onProcessingTime(long)}.
+ */
+ private final AtomicInteger status;
+
+ TimerScheduledFuture(boolean isPeriodic,
Consumer<TimerScheduledFuture<?>> onDoneAction) {
+ this.isPeriodic = isPeriodic;
+
+ this.wrappedFuture = new CompletableFuture<>();
+ this.onDoneActionFuture = new CompletableFuture<>();
+ this.status = new AtomicInteger(STATUS_PENDING);
+
+ // if "onDoneAction" is executed in the timer thread,
the execution of "onDoneAction" may
+ // be interrupted when canceling the timer, which will
cause some state uncertainty, so we
+ // execute it asynchronously
+
onDoneActionFuture.thenAcceptAsync(checkNotNull(onDoneAction));
+ }
+
+ void bind(ScheduledFuture<?> future) {
+ checkState(!this.wrappedFuture.isDone());
+ this.wrappedFuture.complete(future);
+ }
+
+ ProcessingTimeCallback getCallback(ProcessingTimeCallback
callback) {
+ return isPeriodic ? wrapPeriodicCallback(callback) :
wrapOneShotCallback(callback);
+ }
+
+ private ProcessingTimeCallback
wrapOneShotCallback(ProcessingTimeCallback callback) {
+
+ return timestamp -> {
+ if (status.compareAndSet(STATUS_PENDING,
STATUS_EXECUTING)) {
+
+ try {
+
callback.onProcessingTime(timestamp);
+
+
status.compareAndSet(STATUS_EXECUTING, STATUS_SUCCEEDED);
+ } catch (Throwable throwable) {
+ // this may be a true execution
failure or a cancellation
+ // that can be interrupted to
running
+
+
status.compareAndSet(STATUS_EXECUTING, STATUS_FAILED);
+ throw throwable;
+ } finally {
+
onDoneActionFuture.complete(this);
+ }
+ }
+ };
+ }
+
+ private ProcessingTimeCallback
wrapPeriodicCallback(ProcessingTimeCallback callback) {
+
+ return timestamp -> {
+ if (status.compareAndSet(STATUS_PENDING,
STATUS_EXECUTING)) {
+
+ callback.onProcessingTime(timestamp);
+ if
(!status.compareAndSet(STATUS_EXECUTING, STATUS_PENDING)) {
+ checkState(status.get() ==
STATUS_CANCELLED);
+
onDoneActionFuture.complete(this);
+ }
+ } else if (status.get() == STATUS_CANCELLED) {
+ // when the underlying future is not
successfully cancelled, this ensures that it
+ // will not run at idle all the time
+ // (see the logic of canceling from the
"STATUS_PENDING" state in #cancel())
+ checkState(wrappedFuture.isDone());
+ wrappedFuture.get().cancel(false);
+ }
+ };
+ }
+
+ private ScheduledFuture<?> getUnderlyingFuture() {
+ try {
+ return wrappedFuture.get();
+ } catch (Throwable t) {
+ throw new RuntimeException(t);
+ }
+ }
+
+ @Override
+ public boolean cancel(boolean mayInterruptIfRunning) {
+ ScheduledFuture<?> future = getUnderlyingFuture();
+
+ if (status.compareAndSet(STATUS_PENDING,
STATUS_CANCELLED)) {
+ // we don't need to worry about whether the
cancellation is successful, even if it is
+ // not, the callback that is wrapped will not
be executed
+ future.cancel(mayInterruptIfRunning);
+
+ onDoneActionFuture.complete(this);
+ return true;
+ }
+
+ if (mayInterruptIfRunning) {
+ if (future.cancel(true)) {
+ status.set(STATUS_CANCELLED);
+ onDoneActionFuture.complete(this);
+
+ return true;
+ }
+ } else if (isPeriodic) {
+ if (future.cancel(false)) {
+ // if the timer is still executing,
"onDoneActionFuture" must be completed by the
+ // timer thread to ensure that
"onDoneAction" runs at the right time
+ if
(!status.compareAndSet(STATUS_EXECUTING, STATUS_CANCELLED)) {
+
checkState(status.compareAndSet(STATUS_PENDING, STATUS_CANCELLED));
Review comment:
Race condition?
```
t1 starts execution, state is currently PENDING
t2 calls cancels(interrupt = false)
t2 checks state - it's PENDING (line 300)
t1 CASes state from PENDING to EXECUTING
t2 expects PENDING - and fails (line 301)
```
----------------------------------------------------------------
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