1u0 commented on a change in pull request #8523: [FLINK-12481][runtime] Invoke 
timer callback in task thread (via mailbox)
URL: https://github.com/apache/flink/pull/8523#discussion_r288239644
 
 

 ##########
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java
 ##########
 @@ -253,89 +243,53 @@ int getNumTasksScheduled() {
        // 
------------------------------------------------------------------------
 
        /**
-        * Internal task that is invoked by the timer service and triggers the 
target.
+        * A context to which {@link ProcessingTimeCallback} would be passed to 
be invoked when a timer is up.
         */
-       private static final class TriggerTask implements Runnable {
+       public interface ScheduledCallbackExecutionContext {
 
-               private final AtomicInteger serviceStatus;
-               private final Object lock;
-               private final ProcessingTimeCallback target;
-               private final long timestamp;
-               private final AsyncExceptionHandler exceptionHandler;
-
-               private TriggerTask(
-                               final AtomicInteger serviceStatus,
-                               final AsyncExceptionHandler exceptionHandler,
-                               final Object lock,
-                               final ProcessingTimeCallback target,
-                               final long timestamp) {
-
-                       this.serviceStatus = 
Preconditions.checkNotNull(serviceStatus);
-                       this.exceptionHandler = 
Preconditions.checkNotNull(exceptionHandler);
-                       this.lock = Preconditions.checkNotNull(lock);
-                       this.target = Preconditions.checkNotNull(target);
-                       this.timestamp = timestamp;
-               }
+               void invoke(ProcessingTimeCallback callback, long timestamp) 
throws InterruptedException;
+       }
 
-               @Override
-               public void run() {
-                       synchronized (lock) {
-                               try {
-                                       if (serviceStatus.get() == 
STATUS_ALIVE) {
-                                               
target.onProcessingTime(timestamp);
-                                       }
-                               } catch (Throwable t) {
-                                       TimerException asyncException = new 
TimerException(t);
-                                       
exceptionHandler.handleAsyncException("Caught exception while processing 
timer.", asyncException);
-                               }
-                       }
-               }
+       private Runnable wrapOnTimerCallback(ProcessingTimeCallback callback, 
long timestamp) {
+               return new TimeTrackingDelegate(status, 
callbackExecutionContext, callback, timestamp, 0);
        }
 
-       /**
-        * Internal task which is repeatedly called by the processing time 
service.
-        */
-       private static final class RepeatedTriggerTask implements Runnable {
+       private Runnable wrapOnTimerCallback(ProcessingTimeCallback callback, 
long nextTimestamp, long period) {
+               return new TimeTrackingDelegate(status, 
callbackExecutionContext, callback, nextTimestamp, period);
+       }
 
+       private static final class TimeTrackingDelegate implements Runnable {
                private final AtomicInteger serviceStatus;
-               private final Object lock;
-               private final ProcessingTimeCallback target;
-               private final long period;
-               private final AsyncExceptionHandler exceptionHandler;
+               private final ScheduledCallbackExecutionContext 
callbackExecutionContext;
+               private final ProcessingTimeCallback callback;
 
                private long nextTimestamp;
+               private final long period;
 
-               private RepeatedTriggerTask(
-                               final AtomicInteger serviceStatus,
-                               final AsyncExceptionHandler exceptionHandler,
-                               final Object lock,
-                               final ProcessingTimeCallback target,
-                               final long nextTimestamp,
-                               final long period) {
-
-                       this.serviceStatus = 
Preconditions.checkNotNull(serviceStatus);
-                       this.lock = Preconditions.checkNotNull(lock);
-                       this.target = Preconditions.checkNotNull(target);
+               TimeTrackingDelegate(
+                               AtomicInteger serviceStatus,
+                               ScheduledCallbackExecutionContext 
callbackExecutionContext,
+                               ProcessingTimeCallback callback,
+                               long timestamp,
+                               long period) {
+                       this.serviceStatus = serviceStatus;
+                       this.callbackExecutionContext = 
callbackExecutionContext;
+                       this.callback = callback;
+                       this.nextTimestamp = timestamp;
                        this.period = period;
-                       this.exceptionHandler = 
Preconditions.checkNotNull(exceptionHandler);
-
-                       this.nextTimestamp = nextTimestamp;
                }
 
                @Override
                public void run() {
-                       synchronized (lock) {
-                               try {
-                                       if (serviceStatus.get() == 
STATUS_ALIVE) {
-                                               
target.onProcessingTime(nextTimestamp);
-                                       }
-
-                                       nextTimestamp += period;
-                               } catch (Throwable t) {
-                                       TimerException asyncException = new 
TimerException(t);
-                                       
exceptionHandler.handleAsyncException("Caught exception while processing 
repeated timer task.", asyncException);
-                               }
+                       if (serviceStatus.get() != STATUS_ALIVE) {
+                               return;
 
 Review comment:
   This change assumes, that if status is not `STATUS_ALIVE`, then the service 
is not working as active timer anymore and it cannot go to status 
`STATUS_ALIVE` 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to