[ https://issues.apache.org/jira/browse/FLINK-8721?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16410005#comment-16410005 ]
ASF GitHub Bot commented on FLINK-8721: --------------------------------------- Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/5737#discussion_r176515453 --- Diff: flink-core/src/main/java/org/apache/flink/util/OptionalFailure.java --- @@ -0,0 +1,112 @@ +/* + * 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.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Objects; +import java.util.function.Supplier; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Wrapper around an object representing either a success (with a given value) or a failure cause. + */ +public class OptionalFailure<T> implements Serializable { + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(OptionalFailure.class); + + @Nullable + private T value; + + @Nullable + private Throwable failureCause; + + private OptionalFailure(@Nullable T value, @Nullable Throwable failureCause) { + this.value = value; + this.failureCause = failureCause; + } + + public static <T> OptionalFailure<T> of(T value) { + return new OptionalFailure<>(value, null); + } + + public static <T> OptionalFailure<T> ofFailure(Throwable failureCause) { + return new OptionalFailure<>(null, failureCause); + } + + /** + * @return wrapped {@link OptionalFailure} returned by {@code valueSupplier} or wrapped failure if + * {@code valueSupplier} has thrown a {@link RuntimeException}. + */ + public static <T> OptionalFailure<T> createFrom(Supplier<T> valueSupplier) { + try { + return OptionalFailure.of(valueSupplier.get()); + } + catch (RuntimeException ex) { + LOG.error("Failed to archive accumulators", ex); + return OptionalFailure.ofFailure(ex); + } + } + + /** + * @return stored value or throw a {@link FlinkRuntimeException} with {@code failureCause}. + */ + public T get() throws FlinkRuntimeException { --- End diff -- I think `get` should throw a checked exception and not an unchecked exception. Otherwise users won't be aware of it. We could provide a method `getUnchecked` where we throw an unchecked exception. > Client blocks indefinitely if job archiving fails > ------------------------------------------------- > > Key: FLINK-8721 > URL: https://issues.apache.org/jira/browse/FLINK-8721 > Project: Flink > Issue Type: Improvement > Components: Client, JobManager > Affects Versions: 1.5.0 > Reporter: Chesnay Schepler > Assignee: Piotr Nowojski > Priority: Blocker > Labels: flip-6 > Fix For: 1.5.0 > > > While porting the {{AccumulatorErrorITCase}} i noticed that, for FLIP-6, if > the job archiving fails (in this case due to a custom accumulator throwing an > exception in #merge) no response is sent to the client. > {code} > 3547 [flink-akka.actor.default-dispatcher-2] ERROR > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor - Caught exception > while executing runnable in main thread. > org.apache.flink.test.accumulators.AccumulatorErrorITCase$CustomException > at > org.apache.flink.test.accumulators.AccumulatorErrorITCase$FaultyAccumulator.clone(AccumulatorErrorITCase.java:113) > at > org.apache.flink.test.accumulators.AccumulatorErrorITCase$FaultyAccumulator.clone(AccumulatorErrorITCase.java:107) > at > org.apache.flink.api.common.accumulators.AccumulatorHelper.mergeInto(AccumulatorHelper.java:52) > at > org.apache.flink.runtime.executiongraph.ExecutionJobVertex.getAggregatedUserAccumulatorsStringified(ExecutionJobVertex.java:599) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex.<init>(ArchivedExecutionJobVertex.java:49) > at > org.apache.flink.runtime.executiongraph.ExecutionJobVertex.archive(ExecutionJobVertex.java:612) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:313) > at > org.apache.flink.runtime.jobmaster.JobMaster.jobStatusChanged(JobMaster.java:985) > at > org.apache.flink.runtime.jobmaster.JobMaster.access$1400(JobMaster.java:136) > at > org.apache.flink.runtime.jobmaster.JobMaster$JobManagerJobStatusListener.lambda$jobStatusChanges$0(JobMaster.java:1181) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:292) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:147) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleMessage(FencedAkkaRpcActor.java:66) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$onReceive$0(AkkaRpcActor.java:129) > at > akka.actor.ActorCell$$anonfun$become$1.applyOrElse(ActorCell.scala:544) > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) > at akka.dispatch.Mailbox.run(Mailbox.scala:224) > at akka.dispatch.Mailbox.exec(Mailbox.scala:234) > at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > at > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > at > scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > at > scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)