Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-13806

Metric Fetcher floods the JM log with errors when TM is lost

    XMLWordPrintableJSON

Details

    Description

      When a task manager is lost, the log contains a series of exceptions from the metrics fetcher, making it hard to identify the exceptions from the actual job failure.

      The exception below is contained multiple time (in my example eight times) in a simple 4 TM setup after one TM failure.

      I would suggest to suppress "failed asks" (timeouts) from the metrics fetcher service, because the fetcher has not enough information to distinguish between root cause exceptions and follow-up exceptions. In most cases, these exceptions should be follow-up to a failure that is handled in the scheduler/ExecutionGraph already, and the additional exception logging only add noise to the log.

      2019-08-20 22:00:09,865 WARN  org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcherImpl  - Requesting TaskManager's path for query services failed.
      java.util.concurrent.CompletionException: akka.pattern.AskTimeoutException: Ask timed out on [Actor[akka://flink/user/dispatcher#-1834666306]] after [10000 ms]. Message of type [org.apache.flink.runtime.rpc.messages.LocalFencedMessage]. A typical reason for `AskTimeoutException` is that the recipient actor didn't send a reply.
              at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
              at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
              at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
              at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
              at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
              at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
              at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:871)
              at akka.dispatch.OnComplete.internal(Future.scala:263)
              at akka.dispatch.OnComplete.internal(Future.scala:261)
              at akka.dispatch.japi$CallbackBridge.apply(Future.scala:191)
              at akka.dispatch.japi$CallbackBridge.apply(Future.scala:188)
              at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
              at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:74)
              at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
              at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
              at akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:644)
              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.lang.Thread.run(Thread.java:748)
      Caused by: akka.pattern.AskTimeoutException: Ask timed out on [Actor[akka://flink/user/dispatcher#-1834666306]] after [10000 ms]. Message of type [org.apache.flink.runtime.rpc.messages.LocalFencedMessage]. 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)
              ... 9 more
      

      Attachments

        Issue Links

          Activity

            People

              chesnay Chesnay Schepler
              sewen Stephan Ewen
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 20m
                  20m