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



##########
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:
       See the history in the PR. The pb is that now that coordinatorExecutor 
is a ScheduledExecutorService, UncaughtExceptionHandler is useful only for 
exception raised by the threadPool as the JVM now uses a ScheduledFutureTask 
that catches exceptions in its run method. So the exception handler is no more 
for the runnable exceptions but for exceptions raised by the pool itself. It is 
also called in a manual try/catch in the runnable execution in `wrap()` (that 
Till recommended) to workaround the pb with ScheduledFutureTask explained above.




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to