Return-Path: X-Original-To: apmail-cassandra-user-archive@www.apache.org Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id D271710EA4 for ; Mon, 15 Jul 2013 14:09:21 +0000 (UTC) Received: (qmail 38095 invoked by uid 500); 15 Jul 2013 14:09:18 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 37162 invoked by uid 500); 15 Jul 2013 14:09:17 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 37111 invoked by uid 99); 15 Jul 2013 14:09:14 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 15 Jul 2013 14:09:14 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of jakers@gmail.com designates 209.85.212.54 as permitted sender) Received: from [209.85.212.54] (HELO mail-vb0-f54.google.com) (209.85.212.54) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 15 Jul 2013 14:09:07 +0000 Received: by mail-vb0-f54.google.com with SMTP id q12so3131597vbe.41 for ; Mon, 15 Jul 2013 07:08:46 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:from:date:message-id:subject:to :content-type; bh=EMEO50DKPFwfwE1iH2BT4F3qccKezGtcFKhiZoerSDQ=; b=Fu/SlUNbQnmI2g4Xf7XIob8AfMr7VJhzZOXecfdctCRV0YZNqlIO7IPhfCBTJmDY1E hNv+jCgSYsQtZOzm1uUnnhC+oqZ8dProrpB3VkGnj48fhmkqmb81Aq3z4UECDW9srDHR kRldD+44USWzFxAv15rK6MgvSzHISeum1Wb+GF4dOJfKlRuvBrcKXpq5zZ7FpbKg/TeY W0xQTTCZu8dqazEUSAYf/O+RLJ7hKR2d/o5B9QG+UTcy3AV3z4J2Te4pnj4rnL8JuJk7 8fEJ1UiRpWxQAOmM66NzroAVCEnbOOv6qwGpX5PpXVieUCX/Etmhmqo0u+zainjhSko4 6zyg== X-Received: by 10.52.31.68 with SMTP id y4mr4583269vdh.97.1373897326654; Mon, 15 Jul 2013 07:08:46 -0700 (PDT) MIME-Version: 1.0 Received: by 10.58.118.17 with HTTP; Mon, 15 Jul 2013 07:08:26 -0700 (PDT) In-Reply-To: References: From: Jake Luciani Date: Mon, 15 Jul 2013 10:08:26 -0400 Message-ID: Subject: Re: Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader? To: user Content-Type: multipart/alternative; boundary=bcaec51d2c401d02fa04e18d6674 X-Virus-Checked: Checked by ClamAV on apache.org --bcaec51d2c401d02fa04e18d6674 Content-Type: text/plain; charset=windows-1252 Content-Transfer-Encoding: quoted-printable Take a look at https://issues.apache.org/jira/browse/CASSANDRA-5661 On Mon, Jul 15, 2013 at 4:18 AM, sulong wrote: > Thanks for your help. Yes, I will try to increase the sstable size. I hop= e > it can save me. > > 9000 SSTableReader x 10 RandomAccessReader x 64Kb =3D 5.6G memory. If the= re > is only one RandomAccessReader, the memory will be 9000 * 1 * 64Kb =3D 0.= 56G > . Looks great. But I think it must be reasonable to recycle the > RandomAccessReader. > > > On Mon, Jul 15, 2013 at 4:02 PM, Janne Jalkanen wrote: > >> >> I had exactly the same problem, so I increased the sstable size (from 5 >> to 50 MB - the default 5MB is most certainly too low for serious usecase= s). >> Now the number of SSTableReader objects is manageable, and my heap is >> happier. >> >> Note that for immediate effect I stopped the node, removed the *.json >> files and restarted - which put all SSTables to L0, which meant a weeken= d >> full of compactions=85 Would be really cool if there was a way to >> automatically drop all LCS SSTables one level down to make them compact >> earlier without avoiding the >> "OMG-must-compact-everything-aargh-my-L0-is-full" -effect of removing th= e >> JSON file. >> >> /Janne >> >> On 15 Jul 2013, at 10:48, sulong wrote: >> >> > Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader= ? >> The RandomAccessReader objects consums too much memory. >> > >> > I have a cluster of 4 nodes. Every node's cassandra jvm has 8G heap. >> The cassandra's memory is full after about one month, so I have to resta= rt >> the 4 nodes every month. >> > >> > I have 100G data on every node, with LevedCompactionStrategy and 10M >> sstable size, so there are more than 10000 sstable files. By looking >> through the heap dump file, I see there are more than 9000 SSTableReader >> objects in memory, which references lots of RandomAccessReader objects. >> The memory is consumed by these RandomAccessReader objects. >> > >> > I see the PoolingSegementedFile has a recycle method, which puts the >> RandomAccessReader to a queue. Looks like the Queue always grow until th= e >> sstable is compacted. Is there any way to stop the RandomAccessReader >> recycling? Or, set a limit to the recycled RandomAccessReader's number? >> > >> > >> >> > --=20 http://twitter.com/tjake --bcaec51d2c401d02fa04e18d6674 Content-Type: text/html; charset=windows-1252 Content-Transfer-Encoding: quoted-printable


On= Mon, Jul 15, 2013 at 4:18 AM, sulong <sulong1984@gmail.com> wrote:
Thanks for your help. Yes, = I will try to increase the sstable size. I hope it can save me. =A0
9000 SSTableReader x 10 RandomAccessReader x 64Kb =3D 5.6G memory. If = there is only one RandomAccessReader, the memory will be 9000 * 1 * 64Kb = =3D 0.56G . Looks great. But I think it must be reasonable to recycle the R= andomAccessReader.=A0


On Mon, Jul 15, 2013 at 4:02 PM, Jan= ne Jalkanen <janne.jalkanen@ecyrd.com> wrote:

I had exactly the same problem, so I increased the sstable size (from 5 to = 50 MB - the default 5MB is most certainly too low for serious usecases). = =A0Now the number of SSTableReader objects is manageable, and my heap is ha= ppier.

Note that for immediate effect I stopped the node, removed the *.json files= and restarted - which put all SSTables to L0, which meant a weekend full o= f compactions=85 Would be really cool if there was a way to automatically d= rop all LCS SSTables one level down to make them compact earlier without av= oiding the "OMG-must-compact-everything-aargh-my-L0-is-full" -eff= ect of removing the JSON file.

/Janne

On 15 Jul 2013, at 10:48, sulong <sulong1984@gmail.com> wrote:

> Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader= ? The RandomAccessReader objects consums too much memory.
>
> I have a cluster of 4 nodes. Every node's cassandra jvm has 8G hea= p. The cassandra's memory is full after about one month, so I have to r= estart the 4 nodes every month.
>
> I have 100G data on every node, with LevedCompactionStrategy and 10M s= stable size, so there are more than 10000 sstable files. By looking through= the heap dump file, I see there are more than 9000 SSTableReader objects i= n memory, which references lots of =A0RandomAccessReader objects. The memor= y is consumed by these RandomAccessReader objects.
>
> I see the PoolingSegementedFile has a recycle method, which puts the R= andomAccessReader to a queue. Looks like the Queue always grow until the ss= table is compacted. =A0Is there any way to stop the RandomAccessReader recy= cling? Or, set a limit to the recycled RandomAccessReader's number?
>
>





--
= http://twitter.com/t= jake
--bcaec51d2c401d02fa04e18d6674--