flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-5464) MetricQueryService throws NullPointerException on JobManager
Date Mon, 16 Jan 2017 12:31:26 GMT

    [ https://issues.apache.org/jira/browse/FLINK-5464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15823880#comment-15823880
] 

ASF GitHub Bot commented on FLINK-5464:
---------------------------------------

GitHub user zentol opened a pull request:

    https://github.com/apache/flink/pull/3128

    [FLINK-5464] Improve MetricDumpSerialization error handling

    Rework of #3103.
    
    The key change introduced in the previous PR remains; if a gauge returns null it is not
serialized.
    
    However, I've extended the PR to harden the entire serialization process against exceptions.
The major gain here is that a single failed serialization does no longer destroys the entire
dump; instead it is simply omitted.
    
    In order to allow that I had to replace the ```OutputStream```s with a ```ByteBuffer```.
The former doesn't really allow you to handle failures in between serialization steps, as
you can't reset the stream in any way. The ```ByteBuffer``` is manually resized if a ```BufferOverflowException```
occurs.
    
    * ```MetricDump(De)Serializer#(de)serialize``` will no longer throw any exception but
catch and log them instead
    * Exceptions during the serialization of a metric will cause that metric to be skipped.
    * added test for handling of gauge returning null
    * added test for manual resizing of backing array

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/zentol/flink 5464_mqs_npe

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/3128.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3128
    
----
commit 8610a47c407afe2140cd4b5651ebc794ef3feec8
Author: zentol <chesnay@apache.org>
Date:   2017-01-12T11:41:56Z

    [FLINK-5464] [metrics] Ignore metrics that are null

commit 442c0a4dee002b73e5b86d6c7bb274484a8900ac
Author: zentol <chesnay@apache.org>
Date:   2017-01-16T10:25:58Z

    [hotfix] Remove unused variable in MetricDumpSerializerTest

commit 0f813ebf53414b1b68c6dfe8e3e1dbc896054c36
Author: zentol <chesnay@apache.org>
Date:   2017-01-12T11:42:26Z

    [FLINK-5464] [metrics] Improve MetricDumpSerialization exception handling

----


> MetricQueryService throws NullPointerException on JobManager
> ------------------------------------------------------------
>
>                 Key: FLINK-5464
>                 URL: https://issues.apache.org/jira/browse/FLINK-5464
>             Project: Flink
>          Issue Type: Bug
>          Components: Webfrontend
>    Affects Versions: 1.2.0
>            Reporter: Robert Metzger
>            Assignee: Chesnay Schepler
>
> I'm using Flink 699f4b0.
> My JobManager log contains many of these log entries:
> {code}
> 2017-01-11 19:42:05,778 WARN  org.apache.flink.runtime.webmonitor.metrics.MetricFetcher
    - Fetching metrics failed.
> akka.pattern.AskTimeoutException: Ask timed out on [Actor[akka://flink/user/MetricQueryService#-970662317]]
after [10000 ms]
> 	at akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:334)
> 	at akka.actor.Scheduler$$anon$7.run(Scheduler.scala:117)
> 	at scala.concurrent.Future$InternalCallbackExecutor$.scala$concurrent$Future$InternalCallbackExecutor$$unbatchedExecute(Future.scala:694)
> 	at scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:691)
> 	at akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(Scheduler.scala:474)
> 	at akka.actor.LightArrayRevolverScheduler$$anon$8.executeBucket$1(Scheduler.scala:425)
> 	at akka.actor.LightArrayRevolverScheduler$$anon$8.nextTick(Scheduler.scala:429)
> 	at akka.actor.LightArrayRevolverScheduler$$anon$8.run(Scheduler.scala:381)
> 	at java.lang.Thread.run(Thread.java:745)
> 2017-01-11 19:42:07,765 WARN  org.apache.flink.runtime.metrics.dump.MetricQueryService
     - An exception occurred while processing a message.
> java.lang.NullPointerException
> 	at org.apache.flink.runtime.metrics.dump.MetricDumpSerialization.serializeGauge(MetricDumpSerialization.java:162)
> 	at org.apache.flink.runtime.metrics.dump.MetricDumpSerialization.access$300(MetricDumpSerialization.java:47)
> 	at org.apache.flink.runtime.metrics.dump.MetricDumpSerialization$MetricDumpSerializer.serialize(MetricDumpSerialization.java:90)
> 	at org.apache.flink.runtime.metrics.dump.MetricQueryService.onReceive(MetricQueryService.java:109)
> 	at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:167)
> 	at akka.actor.Actor$class.aroundReceive(Actor.scala:467)
> 	at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:97)
> 	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> 	at akka.actor.ActorCell.invoke(ActorCell.scala:487)
> 	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
> 	at akka.dispatch.Mailbox.run(Mailbox.scala:220)
> 	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
> 	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
(v6.3.4#6332)

Mime
View raw message