Skip to content

Commit

Permalink
[FLINK-2067] [runtime] Unwrap the ExceptionInChainedOperatorException…
Browse files Browse the repository at this point in the history
… exceptions to clean up stack traces
  • Loading branch information
StephanEwen committed May 2, 2017
1 parent c9623be commit 6181302
Show file tree
Hide file tree
Showing 5 changed files with 120 additions and 13 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* 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:https://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 javax.annotation.Nonnull;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A runtime exception that is explicitly used to wrap non-runtime exceptions.
*
* <p>The exception is recognized (for example by the Task when reporting exceptions as
* failure causes) and unwrapped to avoid including the wrapper's stack trace in the reports.
* That way, exception traces are keeping to the important parts.
*/
public class WrappingRuntimeException extends FlinkRuntimeException {

private static final long serialVersionUID = 1L;

public WrappingRuntimeException(@Nonnull Throwable cause) {
super(checkNotNull(cause));
}

public WrappingRuntimeException(String message, @Nonnull Throwable cause) {
super(message, checkNotNull(cause));
}

/**
* Recursively unwraps this WrappingRuntimeException and its causes, getting the first
* non wrapping exception.
*
* @return The first cause that is not a wrapping exception.
*/
public Throwable unwrap() {
Throwable cause = getCause();
return (cause instanceof WrappingRuntimeException) ? ((WrappingRuntimeException) cause).unwrap() : cause;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,8 @@
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;

import org.apache.flink.util.WrappingRuntimeException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -727,6 +729,11 @@ else if (current == ExecutionState.CANCELING) {
}
catch (Throwable t) {

// unwrap wrapped exceptions to make stack traces more compact
if (t instanceof WrappingRuntimeException) {
t = ((WrappingRuntimeException) t).unwrap();
}

// ----------------------------------------------------------------
// the execution failed. either the invokable code properly failed, or
// an exception was thrown as a side effect of cancelling
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,11 +59,13 @@
import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
import org.apache.flink.util.SerializedValue;
import org.apache.flink.util.TestLogger;
import org.apache.flink.util.WrappingRuntimeException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;

import javax.annotation.Nonnull;
import java.io.IOException;
import java.lang.reflect.Field;
import java.net.URL;
Expand Down Expand Up @@ -117,9 +119,9 @@ public class TaskTest extends TestLogger {

@Before
public void createQueuesAndActors() {
taskManagerMessages = new LinkedBlockingQueue<Object>();
jobManagerMessages = new LinkedBlockingQueue<Object>();
listenerMessages = new LinkedBlockingQueue<Object>();
taskManagerMessages = new LinkedBlockingQueue<>();
jobManagerMessages = new LinkedBlockingQueue<>();
listenerMessages = new LinkedBlockingQueue<>();
taskManagerGateway = new ForwardingActorGateway(taskManagerMessages);
jobManagerGateway = new ForwardingActorGateway(jobManagerMessages);
listenerGateway = new ForwardingActorGateway(listenerMessages);
Expand Down Expand Up @@ -335,6 +337,32 @@ public void testExecutionFailsInInvoke() {
fail(e.getMessage());
}
}

@Test
public void testFailWithWrappedException() {
try {
Task task = createTask(FailingInvokableWithChainedException.class);
task.registerExecutionListener(listener);

task.run();

assertEquals(ExecutionState.FAILED, task.getExecutionState());
assertTrue(task.isCanceledOrFailed());

Throwable cause = task.getFailureCause();
assertTrue(cause instanceof IOException);

validateTaskManagerStateChange(ExecutionState.RUNNING, task, false);
validateUnregisterTask(task.getExecutionId());

validateListenerMessage(ExecutionState.RUNNING, task, false);
validateListenerMessage(ExecutionState.FAILED, task, true);
}
catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
}

@Test
public void testCancelDuringInvoke() {
Expand Down Expand Up @@ -1232,4 +1260,27 @@ public void invoke() throws Exception {
public void cancel() throws Exception {
}
}

public static final class FailingInvokableWithChainedException extends AbstractInvokable {

@Override
public void invoke() throws Exception {
throw new TestWrappedException(new IOException("test"));
}

@Override
public void cancel() {}
}

// ------------------------------------------------------------------------
// test exceptions
// ------------------------------------------------------------------------

private static class TestWrappedException extends WrappingRuntimeException {
private static final long serialVersionUID = 1L;

public TestWrappedException(@Nonnull Throwable cause) {
super(cause);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,13 @@
import static java.util.Objects.requireNonNull;

import org.apache.flink.annotation.Internal;
import org.apache.flink.util.WrappingRuntimeException;

/**
* A special exception that signifies that the cause exception came from a chained operator.
*/
@Internal
public class ExceptionInChainedOperatorException extends RuntimeException {
public class ExceptionInChainedOperatorException extends WrappingRuntimeException {

private static final long serialVersionUID = 1L;

Expand All @@ -37,12 +38,4 @@ public ExceptionInChainedOperatorException(Throwable cause) {
public ExceptionInChainedOperatorException(String message, Throwable cause) {
super(message, requireNonNull(cause));
}

public Throwable getOriginalCause() {
Throwable ex = this;
do {
ex = ex.getCause();
} while (ex instanceof ExceptionInChainedOperatorException);
return ex;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.util.List;
import java.util.Map;
import java.util.Random;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
Expand Down Expand Up @@ -54,6 +55,7 @@
import org.apache.flink.streaming.runtime.streamstatus.StreamStatusProvider;
import org.apache.flink.util.OutputTag;
import org.apache.flink.util.XORShiftRandom;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -525,7 +527,7 @@ protected <X> void pushToOperator(StreamRecord<X> record) {
operator.setKeyContextElement1(copy);
operator.processElement(copy);
} catch (Exception e) {
throw new RuntimeException("Could not forward element to next operator", e);
throw new ExceptionInChainedOperatorException(e);
}

}
Expand Down

0 comments on commit 6181302

Please sign in to comment.