Skip to content

Conversation

StephanEwen
Copy link
Contributor

@StephanEwen StephanEwen commented Feb 10, 2020

What is the purpose of the change

This change preserves the call stack from the invocation of RPC ask() calls. When the call fails, this allows us to add a meaningful exception pointing to the original call site, rather than having just some weird stack trace from akka.

Note: While it is possible to do this for all types of ask() failures, this PR uses this approach for now only for timeouts.

Former exception message and stack trace:

akka.pattern.AskTimeoutException: Ask timed out on [Actor[akka://flink/user/test_name#-585757974]] after [1 ms]. Message of type [org.apache.flink.runtime.rpc.messages.LocalRpcInvocation]. A typical reason for `AskTimeoutException` is that the recipient actor didn't send a reply.
	at akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635)
	at akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635)
	at akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:648)
	at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:205)
	at scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
	at scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
	at scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
	at akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:328)
	at akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:279)
	at akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:283)
	at akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:235)
	at java.base/java.lang.Thread.run(Thread.java:834)

New exceptions and stack trace:

java.util.concurrent.TimeoutException: Invocation of java.util.concurrent.CompletableFuture org.apache.flink.runtime.rpc.akka.TimeoutCallStackTest$TestingGateway.callThatTimesOut(org.apache.flink.api.common.time.Time) timed out.
	at org.apache.flink.runtime.rpc.akka.TimeoutCallStackTest.testTimeoutException(TimeoutCallStackTest.java:87)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	...

Brief change log

  • Add a new config option akka.ask.callstack (default: true)
  • Capture call stack at call site
  • Replace exception on timeout

Verifying this change

You can verify this with any Flink deployment where something times out, for example deploy/cancel calls when a TaskManager is killed

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): no
  • The public API, i.e., is any changed class annotated with @Public(Evolving): no
  • The serializers: no
  • The runtime per-record code paths (performance sensitive): no
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: no
  • The S3 file system connector: no

Documentation

  • Does this pull request introduce a new feature? yes
  • If yes, how is the feature documented? config docs

@StephanEwen StephanEwen changed the title Ask timeouts [FLINK-15966][runtime] Capture callstacks for RPC ask() calls to improve exceptions. Feb 10, 2020
@flinkbot
Copy link
Collaborator

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit 04fe5b7 (Mon Feb 10 11:57:00 UTC 2020)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.


The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@flinkbot
Copy link
Collaborator

flinkbot commented Feb 10, 2020

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run travis re-run the last Travis build
  • @flinkbot run azure re-run the last Azure build

public class AkkaOptions {

/**
* Timeout for akka ask calls.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

forget to modify this :) ?


// remove the stack frames coming from the proxy interface invocation
final StackTraceElement[] stackTrace = callStackCapture.getStackTrace();
newException.setStackTrace(Arrays.copyOfRange(stackTrace, 3, stackTrace.length));
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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.


static Throwable resolveTimeoutException(Throwable exception, @Nullable Throwable callStackCapture, Method method) {
if (callStackCapture == null || (!(exception instanceof akka.pattern.AskTimeoutException))) {
return exception;
Copy link
Contributor

Choose a reason for hiding this comment

The 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?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that would be fine.
The main reason that I wanted this configurable is to have a way to turn call stack capture off, because during large RPC storms, this might add significantly to the memory consumption.
But always using a TimeoutException and referring to the RPC method name should be uncritical.

@StephanEwen
Copy link
Contributor Author

Merged in 2746e6a

@StephanEwen StephanEwen deleted the ask_timeouts branch February 19, 2020 14:57
Copy link
Contributor

@tillrohrmann tillrohrmann left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm a bit late to the party but the changes look really nice. This will help big time when debugging RPC timeout exceptions :-)

@tisonkun
Copy link
Member

This is a really nice feature for debugging, indeed!

@StephanEwen
Copy link
Contributor Author

Should we backport this to 1.10, switched off by default?
The one caveat is that it changes the resulting exception type, from akka.AskTimeoutException to java.util.concurrent.TimeoutException.

@tillrohrmann
Copy link
Contributor

I couldn't find any direct usages of akka.AskTimeoutException in the production code. There is only usage in ResourceManagerTaskExecutorTest which might be affected by this change.

Apart from that the danger might be that users rely on AskTimeoutException for some exception matching. My gut feeling would be that this is more of a theoretical problem since the AskTimeoutException does not really add a lot of context and I would be surprised if people would really rely on it.

Hence, we could backport it but if we want to be on the safe side, then we only introduce it with Flink 1.11.

@tisonkun
Copy link
Member

@tillrohrmann actually when I internally implement a job client depend on RPC, to support properly retry it depends on AskTimeoutException. However, it should be more elegant to wrap it into TimeoutException which is a Java standard Exception without pollute outside world with a Scala thirdparty library Exception.

@tillrohrmann
Copy link
Contributor

Happy to do it, but not in a bug fix release.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants