-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-15966][runtime] Capture callstacks for RPC ask() calls to improve exceptions. #11048
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -47,6 +47,7 @@ | |
import java.lang.annotation.Annotation; | ||
import java.lang.reflect.InvocationHandler; | ||
import java.lang.reflect.Method; | ||
import java.util.Arrays; | ||
import java.util.Objects; | ||
import java.util.concurrent.Callable; | ||
import java.util.concurrent.CompletableFuture; | ||
|
@@ -90,13 +91,16 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaBasedEndpoint, Rpc | |
@Nullable | ||
private final CompletableFuture<Void> terminationFuture; | ||
|
||
private final boolean captureAskCallStack; | ||
|
||
AkkaInvocationHandler( | ||
String address, | ||
String hostname, | ||
ActorRef rpcEndpoint, | ||
Time timeout, | ||
long maximumFramesize, | ||
@Nullable CompletableFuture<Void> terminationFuture) { | ||
String address, | ||
String hostname, | ||
ActorRef rpcEndpoint, | ||
Time timeout, | ||
long maximumFramesize, | ||
@Nullable CompletableFuture<Void> terminationFuture, | ||
boolean captureAskCallStack) { | ||
|
||
this.address = Preconditions.checkNotNull(address); | ||
this.hostname = Preconditions.checkNotNull(hostname); | ||
|
@@ -105,6 +109,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaBasedEndpoint, Rpc | |
this.timeout = Preconditions.checkNotNull(timeout); | ||
this.maximumFramesize = maximumFramesize; | ||
this.terminationFuture = terminationFuture; | ||
this.captureAskCallStack = captureAskCallStack; | ||
} | ||
|
||
@Override | ||
|
@@ -208,20 +213,20 @@ private Object invokeRpc(Method method, Object[] args) throws Exception { | |
|
||
result = null; | ||
} else { | ||
// Capture the call stack. It is significantly faster to do that via an exception than | ||
// via Thread.getStackTrace(), because exceptions lazily initialize the stack trace, initially only | ||
// capture a lightweight native pointer, and convert that into the stack trace lazily when needed. | ||
final Throwable callStackCapture = captureAskCallStack ? new Throwable() : null; | ||
|
||
// execute an asynchronous call | ||
CompletableFuture<?> resultFuture = ask(rpcInvocation, futureTimeout); | ||
|
||
CompletableFuture<?> completableFuture = resultFuture.thenApply((Object o) -> { | ||
if (o instanceof SerializedValue) { | ||
try { | ||
return ((SerializedValue<?>) o).deserializeValue(getClass().getClassLoader()); | ||
} catch (IOException | ClassNotFoundException e) { | ||
throw new CompletionException( | ||
new RpcException("Could not deserialize the serialized payload of RPC method : " | ||
+ methodName, e)); | ||
} | ||
final CompletableFuture<?> resultFuture = ask(rpcInvocation, futureTimeout); | ||
|
||
final CompletableFuture<Object> completableFuture = new CompletableFuture<>(); | ||
resultFuture.whenComplete((resultValue, failure) -> { | ||
if (failure != null) { | ||
completableFuture.completeExceptionally(resolveTimeoutException(failure, callStackCapture, method)); | ||
} else { | ||
return o; | ||
completableFuture.complete(deserializeValueIfNeeded(resultValue, method)); | ||
} | ||
}); | ||
|
||
|
@@ -370,4 +375,33 @@ public String getHostname() { | |
public CompletableFuture<Void> getTerminationFuture() { | ||
return terminationFuture; | ||
} | ||
|
||
static Object deserializeValueIfNeeded(Object o, Method method) { | ||
if (o instanceof SerializedValue) { | ||
try { | ||
return ((SerializedValue<?>) o).deserializeValue(AkkaInvocationHandler.class.getClassLoader()); | ||
} catch (IOException | ClassNotFoundException e) { | ||
throw new CompletionException( | ||
new RpcException( | ||
"Could not deserialize the serialized payload of RPC method : " + method.getName(), e)); | ||
} | ||
} else { | ||
return o; | ||
} | ||
} | ||
|
||
static Throwable resolveTimeoutException(Throwable exception, @Nullable Throwable callStackCapture, Method method) { | ||
if (callStackCapture == null || (!(exception instanceof akka.pattern.AskTimeoutException))) { | ||
return exception; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The null branch implies that the exception type is dependent on the configuration option, which sounds like an accident waiting to happen if someone wants to handle timeouts explicitly as they would have to know what there are 2 types of timeouts that can occur. How problematic would it be to always create a TimeoutException, and only have different behaviors for how the stacktrace is set? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think that would be fine. |
||
} | ||
|
||
final TimeoutException newException = new TimeoutException("Invocation of " + method + " timed out."); | ||
newException.initCause(exception); | ||
|
||
// remove the stack frames coming from the proxy interface invocation | ||
final StackTraceElement[] stackTrace = callStackCapture.getStackTrace(); | ||
newException.setStackTrace(Arrays.copyOfRange(stackTrace, 3, stackTrace.length)); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we filter the stack trace containing o.a.f.runtime.rpc.akka instead of using 3? Because others may change call stack of this function unintentionally in the future. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would like to initially keep it like that. There is a test that guards this, so we should see it if someone breaks this by accident. The check is slightly trickier than it looks, because the package name of the proxy class depends on visibility keyword of the interface, etc. I would suggest to keep it simple for now and fix it when the test tells us that this simple solution is not enough. Then we should also know which cases cause it to break, what the involved classnames and package names are, etc. |
||
|
||
return newException; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
forget to modify this :) ?