dmvk commented on a change in pull request #18610:
URL: https://github.com/apache/flink/pull/18610#discussion_r818762667



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
##########
@@ -257,9 +257,23 @@ public void signalNoMoreSplits(int subtask) {
         notifier.notifyReadyAsync(callable, handler);
     }
 
+    /** {@inheritDoc} If the runnable throws an Exception, the corresponding 
job is failed. */
     @Override
     public void runInCoordinatorThread(Runnable runnable) {
-        coordinatorExecutor.execute(runnable);
+        coordinatorExecutor.execute(wrap(runnable));
+    }
+
+    private Runnable wrap(final Runnable runnable) {
+        return () -> {
+            try {
+                runnable.run();
+            } catch (final Throwable t) {
+                // when using a SheduledThreadPool, uncaught exception handler 
catches only
+                // exceptions thrown by the threadPool, so manually call it 
when the exception is
+                // thrown by the runnable
+                
coordinatorThreadFactory.uncaughtException(Thread.currentThread(), t);

Review comment:
       👍 Makes sense.
   
   I'm wondering why are we using the ScheduledExecutor here at first place 🤔 
   
   ```
           final Executor errorHandlingCoordinatorExecutor =
                   (runnable) ->
                           coordinatorExecutor.execute(
                                   new ThrowableCatchingRunnable(
                                           
this::handleUncaughtExceptionFromAsyncCall, runnable));
   
           this.notifier = new ExecutorNotifier(workerExecutor, 
errorHandlingCoordinatorExecutor);
   ```
   
   We're effectively transforming it into the `Executor` anyway. Would simply 
switching from `Executors.newScheduledThreadPool` to 
`Executors.newSingleThreadExecutor` do the trick?




-- 
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]


Reply via email to