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



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java
##########
@@ -155,6 +156,41 @@ private void testAssignSplitToUnregisterdReader(boolean 
fromCoordinatorExecutor)
                 "Cannot assign splits");
     }
 
+    @Test
+    public void testExceptionInRunnableFailsTheJob() throws 
InterruptedException {
+        ManuallyTriggeredScheduledExecutorService manualWorkerExecutor =
+                new ManuallyTriggeredScheduledExecutorService();
+        // do not use ManuallyTriggeredScheduledExecutorService for 
coordinatorExecutor
+        // as it needs to use factory.newThread() so that the 
FailJobExceptionHandler is set
+        // use a custom ThreadPool to gain access to the underlying thread
+        // to wait for it before asserting
+        final ThreadAccessibleThreadPool coordinatorExecutor =
+                new ThreadAccessibleThreadPool(coordinatorThreadFactory);
+        SourceCoordinatorContext<MockSourceSplit> testingContext =
+                new SourceCoordinatorContext<>(
+                        coordinatorExecutor,
+                        manualWorkerExecutor,
+                        coordinatorThreadFactory,
+                        operatorCoordinatorContext,
+                        new MockSourceSplitSerializer(),
+                        splitSplitAssignmentTracker);
+
+        testingContext.runInCoordinatorThread(
+                () -> {
+                    throw new RuntimeException();
+                });
+
+        manualWorkerExecutor.triggerAll();
+        // testingContext.close(); already shutdowns coordinatorExecutor and 
blocks until tasks are
+        // finished
+        testingContext.close();
+        // wait that the thread handles the exception through the provided 
exception handler.
+        // Using ThreadPoolExecutor#awaitTermination is not enough to wait for 
exception handling.
+        // It just waits for the runnable
+        coordinatorExecutor.getSingleThread().join();

Review comment:
       Instead of waiting on the thread to finish, could we change the 
`MockOperatorCoordinatorContext` to return a future for when the job is failed?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/util/FailJobExceptionHandler.java
##########
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util;
+
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handler for uncaught exceptions that will log the exception and fail the 
job afterwards.
+ *
+ * <p>This guarantees that critical exceptions are not accidentally lost and 
leave the system
+ * running in an inconsistent state.
+ */
+public final class FailJobExceptionHandler implements 
Thread.UncaughtExceptionHandler {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FailJobExceptionHandler.class);
+    private final OperatorCoordinator.Context context;

Review comment:
       Could we change this into a `Consumer<Throwable>`? That way it is 
independent of the `OperatorCoordinator.Context`.




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