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

vishnu viswanath commented on FLINK-3962:
-----------------------------------------

I am getting this exception in Local execution aswell. (from IDE or when run 
through `flink run`)

{code}
17:16:38,095 ERROR org.apache.flink.metrics.reporter.JMXReporter                
 - A metric with the name 
org.apache.flink.metrics:key0=localhost,key1=taskmanager,key2=f7bccd0c64bc9c3643647913eced7104,key3=Streaming_Job,key4=Map,key5=0,name=numRecordsIn
 was already registered.
javax.management.InstanceAlreadyExistsException: 
org.apache.flink.metrics:key0=localhost,key1=taskmanager,key2=f7bccd0c64bc9c3643647913eced7104,key3=Streaming_Job,key4=Map,key5=0,name=numRecordsIn
        at com.sun.jmx.mbeanserver.Repository.addMBean(Repository.java:437)
        at 
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerWithRepository(DefaultMBeanServerInterceptor.java:1898)
        at 
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerDynamicMBean(DefaultMBeanServerInterceptor.java:966)
        at 
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerObject(DefaultMBeanServerInterceptor.java:900)
        at 
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerMBean(DefaultMBeanServerInterceptor.java:324)
        at 
com.sun.jmx.mbeanserver.JmxMBeanServer.registerMBean(JmxMBeanServer.java:522)
        at 
org.apache.flink.metrics.reporter.JMXReporter.notifyOfAddedMetric(JMXReporter.java:109)
        at 
org.apache.flink.metrics.MetricRegistry.register(MetricRegistry.java:174)
        at 
org.apache.flink.metrics.groups.AbstractMetricGroup.addMetric(AbstractMetricGroup.java:194)
        at 
org.apache.flink.metrics.groups.AbstractMetricGroup.counter(AbstractMetricGroup.java:150)
        at 
org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:141)
        at 
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:66)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:235)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:588)
        at java.lang.Thread.run(Thread.java:745)
{code}

> JMXReporter doesn't properly register/deregister metrics
> --------------------------------------------------------
>
>                 Key: FLINK-3962
>                 URL: https://issues.apache.org/jira/browse/FLINK-3962
>             Project: Flink
>          Issue Type: Bug
>          Components: TaskManager
>    Affects Versions: 1.1.0
>            Reporter: Maximilian Michels
>            Assignee: Stephan Ewen
>             Fix For: 1.1.0
>
>
> The following fails our Yarn tests because it checks for errors in the 
> jobmanager/taskmanager logs:
> {noformat}
> 2016-05-23 19:20:02,349 ERROR org.apache.flink.metrics.reporter.JMXReporter   
>               - A metric with the name 
> org.apache.flink.metrics:key0=testing-worker-linux-docker-05a6b382-3386-linux-4,key1=taskmanager,key2=9398ca9392af615e9d1896d0bd7ff52a,key3=Flink_Java_Job_at_Mon_May_23_19-20-00_UTC_2016,key4=<task_name>,name=numBytesIn
>  was already registered.
> javax.management.InstanceAlreadyExistsException: 
> org.apache.flink.metrics:key0=testing-worker-linux-docker-05a6b382-3386-linux-4,key1=taskmanager,key2=9398ca9392af615e9d1896d0bd7ff52a,key3=Flink_Java_Job_at_Mon_May_23_19-20-00_UTC_2016,key4=<task_name>,name=numBytesIn
>       at com.sun.jmx.mbeanserver.Repository.addMBean(Repository.java:437)
>       at 
> com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerWithRepository(DefaultMBeanServerInterceptor.java:1898)
>       at 
> com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerDynamicMBean(DefaultMBeanServerInterceptor.java:966)
>       at 
> com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerObject(DefaultMBeanServerInterceptor.java:900)
>       at 
> com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerMBean(DefaultMBeanServerInterceptor.java:324)
>       at 
> com.sun.jmx.mbeanserver.JmxMBeanServer.registerMBean(JmxMBeanServer.java:522)
>       at 
> org.apache.flink.metrics.reporter.JMXReporter.notifyOfAddedMetric(JMXReporter.java:76)
>       at 
> org.apache.flink.metrics.MetricRegistry.register(MetricRegistry.java:177)
>       at 
> org.apache.flink.metrics.groups.AbstractMetricGroup.addMetric(AbstractMetricGroup.java:191)
>       at 
> org.apache.flink.metrics.groups.AbstractMetricGroup.counter(AbstractMetricGroup.java:144)
>       at 
> org.apache.flink.metrics.groups.IOMetricGroup.<init>(IOMetricGroup.java:40)
>       at 
> org.apache.flink.metrics.groups.TaskMetricGroup.<init>(TaskMetricGroup.java:68)
>       at 
> org.apache.flink.metrics.groups.JobMetricGroup.addTask(JobMetricGroup.java:74)
>       at 
> org.apache.flink.metrics.groups.TaskManagerMetricGroup.addTaskForJob(TaskManagerMetricGroup.java:86)
>       at 
> org.apache.flink.runtime.taskmanager.TaskManager.submitTask(TaskManager.scala:1092)
>       at 
> org.apache.flink.runtime.taskmanager.TaskManager.org$apache$flink$runtime$taskmanager$TaskManager$$handleTaskMessage(TaskManager.scala:441)
>       at 
> org.apache.flink.runtime.taskmanager.TaskManager$$anonfun$handleMessage$1.applyOrElse(TaskManager.scala:283)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
>       at 
> org.apache.flink.runtime.LeaderSessionMessageFilter$$anonfun$receive$1.applyOrElse(LeaderSessionMessageFilter.scala:36)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:33)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:28)
>       at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:118)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.applyOrElse(LogMessages.scala:28)
>       at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
>       at 
> org.apache.flink.runtime.taskmanager.TaskManager.aroundReceive(TaskManager.scala:124)
>       at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
>       at akka.actor.ActorCell.invoke(ActorCell.scala:487)
>       at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:254)
>       at akka.dispatch.Mailbox.run(Mailbox.scala:221)
>       at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
>       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)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to