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 1CA1F200D3A for ; Wed, 15 Nov 2017 13:18:29 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 1AE37160BF4; Wed, 15 Nov 2017 12:18:29 +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 DDDFF160BEA for ; Wed, 15 Nov 2017 13:18:27 +0100 (CET) Received: (qmail 34527 invoked by uid 500); 15 Nov 2017 12:18:21 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list user@flink.apache.org Received: (qmail 34365 invoked by uid 99); 15 Nov 2017 12:18:21 -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; Wed, 15 Nov 2017 12:18:21 +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 113EB180791 for ; Wed, 15 Nov 2017 12:18:21 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.979 X-Spam-Level: * X-Spam-Status: No, score=1.979 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd3-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=data-artisans-com.20150623.gappssmtp.com Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id 4VCL2s9fr7Dn for ; Wed, 15 Nov 2017 12:18:16 +0000 (UTC) Received: from mail-wm0-f47.google.com (mail-wm0-f47.google.com [74.125.82.47]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id BA2C95FBB0 for ; Wed, 15 Nov 2017 12:18:15 +0000 (UTC) Received: by mail-wm0-f47.google.com with SMTP id b189so2591273wmd.0 for ; Wed, 15 Nov 2017 04:18:15 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=data-artisans-com.20150623.gappssmtp.com; s=20150623; h=from:message-id:mime-version:subject:date:in-reply-to:cc:to :references; bh=COUhtMUSvvBR2thTzEbUMoEadxCjRl65L+Bam9va+BI=; b=hId0keO2o8o36xPMggWG/macqg1dsqqQsyOWvLbeUkoYP843JpK0WdV/3YhCsOP0dW QvnacmC1BdEVagtqo9eGJRvtKbtkVAzIFF9b1aS8nHxU7ZFnZ7WEYWGjzJ/vScgadEW5 dDTcjKp6o3IyoZ5GMgaeFMXrPSuXrF3V+V7WGF6IziqDOtZnJnICjsrHDZoPpb4Ug4/D Pcgi5WiJk60gtNRU/cJf9mgF2R4HFsaL+1DlgiopEQhrMLcCrgggJezsk8qSo7xnDBXP +9/LPr5C9vbOYC+HZM75kSCQx86tOIGYYmbBuQFQIYnH58kqehqEo5esc2z7ULHB3fDr z+BQ== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:from:message-id:mime-version:subject:date :in-reply-to:cc:to:references; bh=COUhtMUSvvBR2thTzEbUMoEadxCjRl65L+Bam9va+BI=; b=Fod1egzVjHqzKWbbX018das0O1ZJLEJglNBWGKg9xIacyAOBzOKsCkZMb8xCAK46j3 VHXhSm5RINHOAkudqBJgaxGP4e5qQHr9/inpuT1VRl3UX01V7J2uf/lpFWSyINJ7vmp6 7OoEEXgL+4AKFSuYr8W+cp9MJzYCGvajR+FCJUQT/4ZEX9OJ3wtEZQW/p5a88YBXlCee LXn0MXgOT2gLvMPf2y7xwp6QoHfLYFkkYGYAhHfcrCeRSAVyLvnoGvfJnus5vJla558N c4H5mtBdHOg6yg9lRkkwJTSClUgz1XW8XV4RAd6m0jX6swtibUsG9tjtxx4XvjqAbg50 /Oaw== X-Gm-Message-State: AJaThX5+6IWG3eWmJmlIKeKuTDv2OajQvKtiIkQiKX1nR6gaiX02d3IM KWhHecDtxsTUBBhLiNXdcU7L+A== X-Google-Smtp-Source: AGs4zMZd33MogysW617wKcqvyJhCtNCE/GLg39OfN8JBtz0x8W4+R3YmPD5nHNjcnFQk1gPeWxoodg== X-Received: by 10.80.182.35 with SMTP id b32mr22356424ede.205.1510748295268; Wed, 15 Nov 2017 04:18:15 -0800 (PST) Received: from piotrs-mbp.fritz.box (ip-2-205-80-95.web.vodafone.de. [2.205.80.95]) by smtp.gmail.com with ESMTPSA id a63sm7503715ede.79.2017.11.15.04.18.14 (version=TLS1_2 cipher=ECDHE-RSA-AES128-GCM-SHA256 bits=128/128); Wed, 15 Nov 2017 04:18:14 -0800 (PST) From: Piotr Nowojski Message-Id: Content-Type: multipart/alternative; boundary="Apple-Mail=_C3E63565-3E6E-4752-8A4F-9E34CEDAEB6C" Mime-Version: 1.0 (Mac OS X Mail 11.1 \(3445.4.7\)) Subject: Re: Off heap memory issue Date: Wed, 15 Nov 2017 13:18:13 +0100 In-Reply-To: Cc: Kien Truong , Javier Lopez , Robert Metzger , "user@flink.apache.org" To: Flavio Pompermaier References: <38c6eb07-cd85-ad2c-f0ff-a0e5118431d1@gmail.com> X-Mailer: Apple Mail (2.3445.4.7) archived-at: Wed, 15 Nov 2017 12:18:29 -0000 --Apple-Mail=_C3E63565-3E6E-4752-8A4F-9E34CEDAEB6C Content-Transfer-Encoding: quoted-printable Content-Type: text/plain; charset=utf-8 Hi, I have been able to observe some off heap memory =E2=80=9Cissues=E2=80=9D = by submitting Kafka job provided by Javier Lopez (in different mailing = thread).=20 TL;DR; There was no memory leak, just memory pool =E2=80=9CMetaspace=E2=80=9D = and =E2=80=9CCompressed Class Space=E2=80=9D are growing in size over = time and are only rarely garbage collected. In my test case they = together were wasting up to ~7GB of memory, while my test case could use = as little as ~100MB. Connect with for example jconsole to your JVM, = check their size and cut their size by half by setting: env.java.opts: -XX:CompressedClassSpaceSize=3D***M = -XX:MaxMetaspaceSize=3D***M In flink-conf.yaml. Everything works fine and memory consumption still = too high? Rinse and repeat. Long story: In default settings, with max heap size of 1GB, off heap memory = consumption, memory consumption off non-heap memory pools of = =E2=80=9CMetaspace=E2=80=9D and =E2=80=9CCompressed Class Space=E2=80=9D = was growing in time which seemed like indefinitely, and Metaspace was = always around ~6 times larger compared to compressed class space. = Default max meatspace size is unlimited, while =E2=80=9CCompressed class = space=E2=80=9D has a default max size of 1GB.=20 When I decreased the CompressedClassSpaceSize down to 100MB, memory = consumption grew up to 90MB and then it started bouncing up and down by = couple of MB. =E2=80=9CMetaspace=E2=80=9D was following the same = pattern, but using ~600MB. When I decreased down MaxMetaspaceSize to = 200MB, memory consumption of both pools was bouncing around ~220MB. It seems like there are no general guide lines how to configure those = values, since it=E2=80=99s heavily application dependent. However this = seems like the most likely suspect of the apparent OFF HEAP =E2=80=9Cmemor= y leak=E2=80=9D that was reported couple of times in use cases where = users are submitting hundreds/thousands of jobs to Flink cluster. For = more information please check here: = https://docs.oracle.com/javase/8/docs/technotes/guides/vm/gctuning/conside= rations.html = Please let us know if this solves your issues. Thanks, Piotrek > On 13 Nov 2017, at 16:06, Flavio Pompermaier = wrote: >=20 > Unfortunately the issue I've opened [1] was not a problem of Flink but = was just caused by an ever increasing job plan. > So no help from that..Let's hope to find out the real source of the = problem. > Maybe using -Djdk.nio.maxCachedBufferSize could help (but I didn't = try it yet) >=20 > Best, > Flavio >=20 > [1] https://issues.apache.org/jira/browse/FLINK-7845 = >=20 > On Wed, Oct 18, 2017 at 2:07 PM, Kien Truong > wrote: > Hi, >=20 > We saw a similar issue in one of our job due to ByteBuffer memory = leak[1].=20 > We fixed it using the solution in the article, setting = -Djdk.nio.maxCachedBufferSize >=20 > This variable is available for Java > 8u102 >=20 > Best regards, >=20 > Kien > [1]http://www.evanjones.ca/java-bytebuffer-leak.html = >=20 > On 10/18/2017 4:06 PM, Flavio Pompermaier wrote: >> We also faced the same problem, but the number of jobs we can run = before restarting the cluster depends on the volume of the data to = shuffle around the network. We even had problems with a single job and = in order to avoid OOM issues we had to put some configuration to limit = Netty memory usage, i.e.: >> - Add to flink.yaml -> env.java.opts: = -Dio.netty.recycler.maxCapacity.default=3D1 >> - Edit taskmanager.sh and change TM_MAX_OFFHEAP_SIZE from 8388607T = to 5g >>=20 >> At this purpose we wrote a small test to reproduce the problem and we = opened an issue for that [1]. >> We still don't know if the problems are related however.. >>=20 >> I hope that could be helpful, >> Flavio >>=20 >> [1] https://issues.apache.org/jira/browse/FLINK-7845 = >>=20 >> On Wed, Oct 18, 2017 at 10:48 AM, Javier Lopez = > wrote: >> Hi Robert, >>=20 >> Sorry to reply this late. We did a lot of tests, trying to identify = if the problem was in our custom sources/sinks. We figured out that none = of our custom components is causing this problem. We came up with a = small test, and realized that the Flink nodes run out of non-heap JVM = memory and crash after deployment of thousands of jobs.=20 >>=20 >> When rapidly deploying thousands or hundreds of thousands of Flink = jobs - depending on job complexity in terms of resource consumption - = Flink nodes non-heap JVM memory consumption grows until there is no more = memory left on the machine and the Flink process crashes. Both = TaskManagers and JobManager exhibit the same behavior. The TaskManagers = die faster though. The memory consumption doesn't decrease after = stopping the deployment of new jobs, with the cluster being idle (no = running jobs).=20 >>=20 >> We could replicate the behavior by the rapid deployment of the = WordCount Job provided in the Quickstart with a Python script. We = started 24 instances of the deployment script to run in parallel. >>=20 >> The non-heap JVM memory consumption grows faster with more complex = jobs, i.e. reading from Kafka 10K events and printing to STDOUT( * ). = Thus less deployed jobs are needed until the TaskManagers/JobManager = dies. >>=20 >> We employ Flink 1.3.2 in standalone mode on AWS EC2 t2.large nodes = with 4GB RAM inside Docker containers. For the test, we used 2 = TaskManagers and 1 JobManager. >>=20 >> ( * ) a slightly changed Python script was used, which waited after = deployment 15 seconds for the 10K events to be read from Kafka, then it = canceled the freshly deployed job via Flink REST API. >>=20 >> If you want we can provide the Scripts and Jobs we used for this = test. We have a workaround for this, which restarts the Flink nodes once = a memory threshold is reached. But this has lowered the availability of = our services. >>=20 >> Thanks for your help. >>=20 >> On 30 August 2017 at 10:39, Robert Metzger > wrote: >> I just saw that your other email is about the same issue. >>=20 >> Since you've done a heapdump already, did you see any pattern in the = allocated objects? Ideally none of the classes from your user code = should stick around when no job is running. >> What's the size of the heap dump? I'm happy to take a look at it if = it's reasonably small. >>=20 >> On Wed, Aug 30, 2017 at 10:27 AM, Robert Metzger > wrote: >> Hi Javier, >>=20 >> I'm not aware of such issues with Flink, but if you could give us = some more details on your setup, I might get some more ideas on what to = look for. >>=20 >> are you using the RocksDBStateBackend? (RocksDB is doing some JNI = allocations, that could potentially leak memory) >> Also, are you passing any special garbage collector options? (Maybe = some classes are not unloaded) >> Are you using anything else that is special (such as protobuf or avro = formats, or any other big library)? >>=20 >> Regards, >> Robert >>=20 >>=20 >>=20 >> On Mon, Aug 28, 2017 at 5:04 PM, Javier Lopez = > wrote: >> Hi all, >>=20 >> we are starting a lot of Flink jobs (streaming), and after we have = started 200 or more jobs we see that the non-heap memory in the = taskmanagers increases a lot, to the point of killing the instances. We = found out that every time we start a new job, the committed non-heap = memory increases by 5 to 10MB. Is this an expected behavior? Are there = ways to prevent this? >>=20 >>=20 >>=20 >>=20 >=20 >=20 >=20 > --=20 > Flavio Pompermaier > Development Department >=20 > OKKAM S.r.l. > Tel. +(39) 0461 041809 --Apple-Mail=_C3E63565-3E6E-4752-8A4F-9E34CEDAEB6C Content-Transfer-Encoding: quoted-printable Content-Type: text/html; charset=utf-8 Hi,

I = have been able to observe some off heap memory =E2=80=9Cissues=E2=80=9D = by submitting Kafka job provided by Javier Lopez (in different mailing = thread). 

TL;DR;

There was no memory leak, just memory = pool =E2=80=9CMetaspace=E2=80=9D and =E2=80=9CCompressed Class Space=E2=80= =9D are growing in size over time and are only rarely garbage collected. = In my test case they together were wasting up to ~7GB of memory, while = my test case could use as little as ~100MB. Connect with for example = jconsole to your JVM, check their size and cut their size by half by = setting:

env.java.opts: -XX:CompressedClassSpaceSize=3D***M = -XX:MaxMetaspaceSize=3D***M

In flink-conf.yaml. Everything works = fine and memory consumption still too high? Rinse and repeat.


Long story:

In default settings, with max heap size = of 1GB, off heap memory consumption, memory consumption off non-heap = memory pools of =E2=80=9CMetaspace=E2=80=9D and =E2=80=9CCompressed = Class Space=E2=80=9D was growing in time which seemed like indefinitely, = and Metaspace was always around ~6 times larger compared to compressed = class space. Default max meatspace size is unlimited, while = =E2=80=9CCompressed class space=E2=80=9D has a default max size of = 1GB. 

When = I decreased the CompressedClassSpaceSize down to 100MB, memory = consumption grew up to 90MB and then it started bouncing up and down by = couple of MB. =E2=80=9CMetaspace=E2=80=9D was following the same = pattern, but using ~600MB. When I decreased down MaxMetaspaceSize to 200MB, memory = consumption of both pools was bouncing around ~220MB.

It seems like there are = no general guide lines how to configure those values, since it=E2=80=99s = heavily application dependent. However this seems like the most likely = suspect of the apparent OFF HEAP =E2=80=9Cmemory = leak=E2=80=9D that was reported couple of times in use cases where = users are submitting hundreds/thousands of jobs to Flink cluster. For = more information please check here:


Please let us know if this solves your issues.

Thanks, = Piotrek

On 13 Nov 2017, at 16:06, Flavio Pompermaier = <pompermaier@okkam.it> wrote:

Unfortunately the issue I've opened [1] was not a problem of = Flink but was just caused by an ever increasing job plan.
So no help from that..Let's hope to find out the real source = of the problem.
Maybe using =  -Djdk.nio.maxCachedBufferSize could help (but I didn't try it = yet)

Best,

On Wed, = Oct 18, 2017 at 2:07 PM, Kien Truong <duckientruong@gmail.com> wrote:
=20 =20 =20

Hi,

We saw a similar issue in one of our job = due to ByteBuffer memory leak[1].

We fixed it using the solution in the article, = setting -Djdk.nio.maxCachedBufferSize

This = variable is available for Java > 8u102

Best = regards,

Kien

[1]http://www.evanjones.ca/java-bytebuffer-leak.html


On 10/18/2017 = 4:06 PM, Flavio Pompermaier wrote:
We also faced the same problem, but = the number of jobs we can run before restarting the cluster depends on the volume of the data to shuffle around the network. We even had problems with a single job and in order to avoid OOM issues we had to put some configuration to limit Netty memory usage, i.e.:
 - Add to flink.yaml -> env.java.opts: -Dio.netty.recycler.maxCapacity.default=3D1
 - Edit = taskmanager.sh and = change TM_MAX_OFFHEAP_SIZE from 8388607T to 5g

At this purpose we wrote a small test to = reproduce the problem and we opened an issue for that [1].
We still don't know if the problems are = related however..

I hope that could be helpful,

On Wed, Oct 18, 2017 at 10:48 = AM, Javier Lopez <javier.lopez@zalando.de> wrote:
Hi Robert,

Sorry to reply this late. We did a = lot of tests, trying to identify if the problem was in our custom sources/sinks. We figured out that none of our custom components is causing this problem. We came up with a small test, and realized that the Flink nodes run out of non-heap JVM = memory and crash after deployment of thousands of = jobs. 

When rapidly deploying thousands = or hundreds of thousands of Flink jobs - depending on job complexity in terms of resource consumption - Flink nodes non-heap JVM memory consumption grows until there is no more memory left on the machine and the Flink process crashes. Both TaskManagers and JobManager exhibit the same behavior. The TaskManagers die faster though. The memory consumption doesn't decrease after stopping the deployment of new jobs, with the cluster being idle (no running = jobs). 

We could replicate the behavior by = the rapid deployment of the WordCount Job provided in the Quickstart with a Python script.  We = started 24 instances of the deployment script to run in parallel.

The non-heap JVM memory = consumption grows faster with more complex jobs, i.e. reading from Kafka 10K events and printing to STDOUT( * ). Thus less deployed jobs are needed until the TaskManagers/JobManager dies.

We employ Flink 1.3.2 in = standalone mode on AWS EC2 t2.large nodes with 4GB RAM inside Docker containers. For the test, we used 2 TaskManagers and 1 JobManager.

( * ) a slightly changed Python = script was used, which waited after deployment 15 seconds for the 10K events to be read from Kafka, then it canceled the freshly deployed job via Flink REST API.

If you want we can provide the = Scripts and Jobs we used for this test. We have a workaround for this, which restarts the Flink nodes once a memory threshold is reached. But this has lowered the availability of our services.

Thanks for your help.

On 30 August 2017 at = 10:39, Robert Metzger <rmetzger@apache.org> wrote:
I just saw that your = other email is about the same issue.

Since you've done a heapdump = already, did you see any pattern in the allocated objects? Ideally none of the classes from your user code should stick around when no job is running.
What's the size of the heap = dump? I'm happy to take a look at it if it's reasonably small.

On Wed, Aug 30, 2017 at 10:27 AM, Robert Metzger <rmetzger@apache.org> wrote:
Hi Javier,

I'm not aware of = such issues with Flink, but if you could give us some more details on your setup, I might get some more ideas on what to look for.

are you using the RocksDBStateBackend? (RocksDB is doing some JNI allocations, that could potentially leak = memory)
Also, are you = passing any special garbage collector options? (Maybe some classes are not unloaded)
Are you using = anything else that is special (such as protobuf or avro formats, or any other big library)?

Regards,
Robert



On = Mon, Aug 28, 2017 at 5:04 PM, Javier Lopez <javier.lopez@zalando.de> wrote:
Hi= all,

we are = starting a lot of Flink jobs (streaming), and after we have started 200 or more jobs we see that the non-heap memory in the taskmanagers increases a lot, to the point of killing the instances. We found out that every time we start a new job, the committed non-heap memory increases by 5 to 10MB. Is this an expected behavior? Are there ways to prevent this?







--
Flavio Pompermaier
Development Department

OKKAM = S.r.l.
Tel. +(39) 0461 = 041809

= --Apple-Mail=_C3E63565-3E6E-4752-8A4F-9E34CEDAEB6C--