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

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

GitHub user pnowojski opened a pull request:

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

    [FLINK-7368][metrics] Backport synchronization fix for MetricStore

    ## What is the purpose of the change
    
    This backports adding missing synchronization on MetricStore
    
    ## Verifying this change
    
    This change is already covered by existing tests, such as 
*MetricFetcherTest*.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / **no**)
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes / **no**)
      - The serializers: (yes / **no** / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / 
**no** / don't know)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / **no**)
      - If yes, how is the feature documented? (**not applicable** / docs / 
JavaDocs / not documented)
    


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

    $ git pull https://github.com/pnowojski/flink flink7368-1.3

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

    https://github.com/apache/flink/pull/4841.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 #4841
    
----
commit 4783633167b3008e66da97076b5e38e89e85e2bd
Author: Piotr Nowojski <piotr.nowoj...@gmail.com>
Date:   2017-10-16T15:17:26Z

    [FLINK-7368][metrics] Backport synchronization fix for MetricStore

----


> MetricStore makes cpu spin at 100%
> ----------------------------------
>
>                 Key: FLINK-7368
>                 URL: https://issues.apache.org/jira/browse/FLINK-7368
>             Project: Flink
>          Issue Type: Bug
>          Components: Metrics
>            Reporter: Nico Chen
>            Assignee: Piotr Nowojski
>            Priority: Blocker
>             Fix For: 1.4.0, 1.3.3
>
>         Attachments: MyHashMap.java, MyHashMapInfiniteLoopTest.java, 
> jm-jstack.log
>
>
> Flink's `MetricStore` is not thread-safe. multi-treads may acess java' 
> hashmap inside `MetricStore` and can tirgger hashmap's infinte loop. 
> Recently I met the case that flink jobmanager consumed 100% cpu. A part of 
> stacktrace is shown below. The full jstack is in the attachment.
> {code:java}
> "ForkJoinPool-1-worker-19" daemon prio=10 tid=0x00007fbdacac9800 nid=0x64c1 
> runnable [0x00007fbd7d1c2000]
>    java.lang.Thread.State: RUNNABLE
>         at java.util.HashMap.put(HashMap.java:494)
>         at 
> org.apache.flink.runtime.webmonitor.metrics.MetricStore.addMetric(MetricStore.java:176)
>         at 
> org.apache.flink.runtime.webmonitor.metrics.MetricStore.add(MetricStore.java:121)
>         at 
> org.apache.flink.runtime.webmonitor.metrics.MetricFetcher.addMetrics(MetricFetcher.java:198)
>         at 
> org.apache.flink.runtime.webmonitor.metrics.MetricFetcher.access$500(MetricFetcher.java:58)
>         at 
> org.apache.flink.runtime.webmonitor.metrics.MetricFetcher$4.onSuccess(MetricFetcher.java:188)
>         at akka.dispatch.OnSuccess.internal(Future.scala:212)
>         at akka.dispatch.japi$CallbackBridge.apply(Future.scala:175)
>         at akka.dispatch.japi$CallbackBridge.apply(Future.scala:172)
>         at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
>         at 
> scala.runtime.AbstractPartialFunction.applyOrElse(AbstractPartialFunction.scala:28)
>         at 
> scala.concurrent.Future$$anonfun$onSuccess$1.apply(Future.scala:117)
>         at 
> scala.concurrent.Future$$anonfun$onSuccess$1.apply(Future.scala:115)
>         at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
>         at 
> java.util.concurrent.ForkJoinTask$AdaptedRunnable.exec(ForkJoinTask.java:1265)
>         at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:334)
>         at 
> java.util.concurrent.ForkJoinWorkerThread.execTask(ForkJoinWorkerThread.java:604)
>         at java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:784)
>         at java.util.concurrent.ForkJoinPool.work(ForkJoinPool.java:646)
>         at 
> java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:398)
> {code}
> There are 24 threads show same stacktrace as above to indicate they are 
> spining at HashMap.put(HashMap.java:494) (I am using Java 1.7.0_6). Many 
> posts indicate multi-threads accessing hashmap cause this problem and I 
> reproduce the case as well. The test code is attached. I only modify the 
> HashMap.transfer() by adding concurrent barriers for different treads in 
> order to simulate the timing of creation of cycles in hashmap's Entry.  My 
> program's stacktrace shows it hangs at same line of 
> HashMap(HashMap.put(HashMap.java:494)) as the stacktrace I post above.
>  Even through `MetricFetcher` has a 10 seconds minimum inteverl between each 
> metrics qurey, it still cannot guarntee query responses do not acess 
> `MtricStore`'s hashmap concurrently.  Thus I think it's a bug to fix.
>  



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to