Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 91B03200D24 for ; Tue, 24 Oct 2017 16:42:07 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 905BB160BF1; Tue, 24 Oct 2017 14:42:07 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id B41BC160BDB for ; Tue, 24 Oct 2017 16:42:06 +0200 (CEST) Received: (qmail 21220 invoked by uid 500); 24 Oct 2017 14:42:05 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 21211 invoked by uid 99); 24 Oct 2017 14:42:05 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 24 Oct 2017 14:42:05 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 8107B18071E for ; Tue, 24 Oct 2017 14:42:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.001 X-Spam-Level: X-Spam-Status: No, score=-100.001 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id sKDxN9ajiNxd for ; Tue, 24 Oct 2017 14:42:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 5A34E5FDCE for ; Tue, 24 Oct 2017 14:42:02 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 7D4D8E0BC8 for ; Tue, 24 Oct 2017 14:42:01 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id A6B0221301 for ; Tue, 24 Oct 2017 14:42:00 +0000 (UTC) Date: Tue, 24 Oct 2017 14:42:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-7368) MetricStore makes cpu spin at 100% MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 24 Oct 2017 14:42:07 -0000 [ https://issues.apache.org/jira/browse/FLINK-7368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16217020#comment-16217020 ] ASF GitHub Bot commented on FLINK-7368: --------------------------------------- Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/4840#discussion_r146582951 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStore.java --- @@ -260,50 +293,66 @@ public String getMetric(String name, String defaultValue) { ? value : defaultValue; } - } - /** - * Sub-structure containing metrics of the JobManager. - */ - public static class JobManagerMetricStore extends ComponentMetricStore { + public static ComponentMetricStore unmodifiable(ComponentMetricStore source) { + if (source == null) { + return null; + } + return new ComponentMetricStore(unmodifiableMap(source.metrics)); + } } /** * Sub-structure containing metrics of a single TaskManager. */ + @ThreadSafe public static class TaskManagerMetricStore extends ComponentMetricStore { - public final Set garbageCollectorNames = new HashSet<>(); + public final Set garbageCollectorNames; + + public TaskManagerMetricStore() { + this(new ConcurrentHashMap<>(), ConcurrentHashMap.newKeySet()); + } + + public TaskManagerMetricStore(Map metrics, Set garbageCollectorNames) { + super(metrics); + this.garbageCollectorNames = checkNotNull(garbageCollectorNames); + } public void addGarbageCollectorName(String name) { garbageCollectorNames.add(name); } + + public static TaskManagerMetricStore unmodifiable(TaskManagerMetricStore source) { --- End diff -- In which case we should rather modify the store to not allow writes in the first place, instead of opting for unmodifiable collections that are pretty much a hack. "here, have an object that fails for 50% of the defined methods"; that's hardly good design is it. Till suggested dedicated read methods that return metrics, something like `List getMetrics(List metricNames)` instead of exposing the metric maps. This would make the interface cleaner and would allow us to simplify the synchronization. > 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)