rkhachatryan commented on a change in pull request #10336: [FLINK-14935][task,runtime] Use RunnableWithException in the Mailbox URL: https://github.com/apache/flink/pull/10336#discussion_r351329419
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureTaskWithException.java ########## @@ -0,0 +1,78 @@ +/* + * 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.concurrent; + +import org.apache.flink.util.function.FunctionUtils; +import org.apache.flink.util.function.RunnableWithException; + +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * Adapter from {@link FutureTask} to {@link RunnableWithException}. + */ +public class FutureTaskWithException<V> implements Future<V>, RunnableWithException { + private final FutureTask<V> futureTask; + + public FutureTaskWithException(RunnableWithException command) { + this(FunctionUtils.asCallable(command, null)); + } + + public FutureTaskWithException(Callable<V> callable) { + this(new FutureTask<>(callable)); + } + + public FutureTaskWithException(FutureTask<V> futureTask) { + this.futureTask = futureTask; + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return futureTask.cancel(mayInterruptIfRunning); + } + + @Override + public boolean isCancelled() { + return futureTask.isCancelled(); + } + + @Override + public boolean isDone() { + return futureTask.isDone(); + } + + @Override + public V get() throws InterruptedException, ExecutionException { + return futureTask.get(); + } + + @Override + public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + return futureTask.get(timeout, unit); + } + + @Override + public void run() throws Exception { + futureTask.run(); + } Review comment: If an exception is thrown inside futureTask.run() then it is set as futureTask outcome, which is ignored in this method, so effectively it gets swallowed. ---------------------------------------------------------------- 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: [email protected] With regards, Apache Git Services
