dmvk commented on a change in pull request #18610: URL: https://github.com/apache/flink/pull/18610#discussion_r818704738
########## 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.function.Consumer; + +/** + * 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 Consumer<Throwable> failureAction; + + public FailJobExceptionHandler(Consumer<Throwable> failureAction) { + this.failureAction = failureAction; + } + + @Override + public void uncaughtException(Thread t, Throwable e) { + LOG.error( + "Job failed: Thread '{}' produced an uncaught exception. Stopping the job...", Review comment: ```suggestion "Thread '{}' produced an uncaught exception. Failing the job.", ``` ########## 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.function.Consumer; + +/** + * 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 { Review comment: The name seems bit misleading. This doesn't really fail anything, it just calls a provided exception consumer. Since it's only used by the `CoordinatorExecutorThreadFactory`, would it make sense to convert it into an inner class / lambda? ########## 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: I don't think I understand this. The provided `Runnable` should be always executed by the thread from the pool. Any uncaught exception from the `Runnable` should be passed to the uncaught exception handler of the thread pool. Why do we need this safeguard? Is there anything I'm missing? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java ########## @@ -155,6 +156,40 @@ private void testAssignSplitToUnregisterdReader(boolean fromCoordinatorExecutor) "Cannot assign splits"); } + @Test + public void testExceptionInRunnableFailsTheJob() + throws InterruptedException, ExecutionException { + ManuallyTriggeredScheduledExecutorService manualWorkerExecutor = + new ManuallyTriggeredScheduledExecutorService(); + // need the factory to have the exception handler set + final ManuallyTriggeredScheduledExecutorService coordinatorExecutorWithExceptionHandler = + new ManuallyTriggeredScheduledExecutorService(1, coordinatorThreadFactory); Review comment: This implementation executes `Runnables` directly in the thread that has executed the `trigger...` method, so the exception handler is basically ignored. I guess this might have been the reason for introducing the `wrap` method on the context, otherwise this test shouldn't pass. Also the changes to `ManuallyTriggeredScheduledExecutorService` don't seem to have any effect, so we should be able to get rid of them. Can we do something as simple as ... ? ```java final ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor( new SourceCoordinatorProvider.CoordinatorExecutorThreadFactory("test", operatorCoordinatorContext)); ``` Another option would be adding support for the exception handler to `ManuallyTriggeredScheduledExecutorService` (basically wrapping `next.run();` in the `trigger` method with `try/catch`. ########## File path: flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/ManuallyTriggeredScheduledExecutorService.java ########## @@ -54,6 +57,14 @@ private boolean shutdown; + public ManuallyTriggeredScheduledExecutorService() { + super(DEFAULT_NB_THREADS); + } + + public ManuallyTriggeredScheduledExecutorService(int nbThreads, ThreadFactory factory) { + super(nbThreads, factory); Review comment: This change IMO doesn't have any effect. Can you please elaborate? -- 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]
