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 C97D57137 for ; Thu, 1 Dec 2011 19:43:35 +0000 (UTC) Received: (qmail 66178 invoked by uid 500); 1 Dec 2011 19:43:33 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 66153 invoked by uid 500); 1 Dec 2011 19:43:33 -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 66145 invoked by uid 99); 1 Dec 2011 19:43:33 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 01 Dec 2011 19:43:33 +0000 X-ASF-Spam-Status: No, hits=2.8 required=5.0 tests=FREEMAIL_FROM,HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS,URI_HEX X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of sicoe.alexandru@googlemail.com designates 209.85.216.44 as permitted sender) Received: from [209.85.216.44] (HELO mail-qw0-f44.google.com) (209.85.216.44) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 01 Dec 2011 19:43:28 +0000 Received: by qadb15 with SMTP id b15so467967qad.10 for ; Thu, 01 Dec 2011 11:43:08 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=googlemail.com; s=gamma; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=4LNKP6ed8Y9sVMdQ2yZe/oMbPpEbNc4RRUaEl5ynS0Q=; b=BerTbTyH3EWoO87T7HEpWdC9I9JsO6UYw9CowImk6K8vKJ5yFn2dnb1VsXpqC0ncI0 s+O7YpTIt81M98UTYY9CoWN/EfqsMGOFEljq+NKacUakKIBWi/LspJWif6OtluD2ow+H x0w6XaU+vlY/FGZdvrr3PKFpX7s+9cvuE6X3Y= MIME-Version: 1.0 Received: by 10.229.3.141 with SMTP id 13mr1690160qcn.147.1322768587983; Thu, 01 Dec 2011 11:43:07 -0800 (PST) Received: by 10.229.20.84 with HTTP; Thu, 1 Dec 2011 11:43:07 -0800 (PST) In-Reply-To: <4ED7C052.10802@morningstar.com> References: <4ED7C052.10802@morningstar.com> Date: Thu, 1 Dec 2011 20:43:07 +0100 Message-ID: Subject: Re: Insufficient disk space to flush From: Alexandru Dan Sicoe To: user@cassandra.apache.org Content-Type: multipart/alternative; boundary=0016368320fcce8e3f04b30d0f9f --0016368320fcce8e3f04b30d0f9f Content-Type: text/plain; charset=ISO-8859-1 Hi Jeremiah, My commitlog was indeed on another disk. I did what you said and yes the node restart brings back the disk size to the around 50 GB I was expecting. Still I do not understand how the node managed to get itself in the situation of having these tmp files? Could you clarify what these are, how they are produced and why? I've tried to find a clear definition but all I could come up with is hints that they are produced during compaction. I also found a thread that described a similar problem: http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Errors-During-Compaction-td5953493.html as described there it seems like compaction fails and tmp files don't get cleaned up until they fill the disk. Is this what happened in my case? Compactions did not finish properly because the disk utilization was more than half and then more and more files tmp started getting accumulated at each other attempt. The Cassandra log would indicate this because I get many of these: ERROR [CompactionExecutor:22850] 2011-12-01 04:12:15,200 CompactionManager.java (line 513) insufficie nt space to compact even the two smallest files, aborting before I started getting many of these: ERROR [FlushWriter:283] 2011-12-01 04:12:22,917 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[FlushWriter:283,5,main] java.lang.RuntimeException: java.lang.RuntimeException: Insufficient disk space to flush 42531 bytes I just want to clearly understand what happened. Thanks, Alex On Thu, Dec 1, 2011 at 6:58 PM, Jeremiah Jordan < jeremiah.jordan@morningstar.com> wrote: > If you are writing data with QUORUM or ALL you should be safe to restart > cassandra on that node. If the extra space is all from *tmp* files from > compaction they will get deleted at startup. You will then need to run > repair on that node to get back any data that was missed while it was > full. If your commit log was on a different device you may not even have > lost much. > > -Jeremiah > > > On 12/01/2011 04:16 AM, Alexandru Dan Sicoe wrote: > > Hello everyone, > 4 node Cassandra 0.8.5 cluster with RF =2. > One node started throwing exceptions in its log: > > ERROR 10:02:46,837 Fatal exception in thread > Thread[FlushWriter:1317,5,main] > java.lang.RuntimeException: java.lang.RuntimeException: Insufficient disk > space to flush 17296 bytes > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:619) > Caused by: java.lang.RuntimeException: Insufficient disk space to flush > 17296 bytes > at > org.apache.cassandra.db.ColumnFamilyStore.getFlushPath(ColumnFamilyStore.java:714) > at > org.apache.cassandra.db.ColumnFamilyStore.createFlushWriter(ColumnFamilyStore.java:2301) > at > org.apache.cassandra.db.Memtable.writeSortedContents(Memtable.java:246) > at org.apache.cassandra.db.Memtable.access$400(Memtable.java:49) > at > org.apache.cassandra.db.Memtable$3.runMayThrow(Memtable.java:270) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) > ... 3 more > > Checked disk and obviously it's 100% full. > > How do I recover from this without loosing the data? I've got plenty of > space on the other nodes, so I thought of doing a decommission which I > understand reassigns ranges to the other nodes and replicates data to them. > After that's done I plan on manually deleting the data on the node and then > joining in the same cluster position with auto-bootstrap turned off so that > I won't get back the old data and I can continue getting new data with the > node. > > Note, I would like to have 4 nodes in because the other three barely take > the input load alone. These are just long running tests until I get some > better machines. > > On strange thing I found is that the data folder on the ndoe that filled > up the disk is 150 GB (as measured with du) while the data folder on all > other 3 nodes is 50 GB. At the same time, DataStax OpsCenter shows a size > of around 50GB for all 4 nodes. I though that the node was making a major > compaction at which time it filled up the disk....but even that doesn't > make sense because shouldn't a major compaction just be capable of doubling > the size, not triple-ing it? Doesn anyone know how to explain this behavior? > > Thanks, > Alex > > --0016368320fcce8e3f04b30d0f9f Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable Hi Jeremiah,
=A0My commitlog was indeed on another di= sk. I did what you said and yes the node restart brings back the disk size = to the around 50 GB I was expecting. Still I do not understand how the node= managed to get itself in the situation of having these tmp files? Could yo= u clarify what these are, how they are produced and why? I've tried to = find a clear definition but all I could come up with is hints that they are= produced during compaction. I also found a thread that described a similar= problem:
http://cassandra-user-incubator-a= pache-org.3065146.n2.nabble.com/Errors-During-Compaction-td5953493.html=
as described there it seems like compaction fails and tmp files don't g= et cleaned up until they fill the disk. Is this what happened in my case? C= ompactions did not finish properly because the disk utilization was more th= an half and then more and more files tmp started getting accumulated at eac= h other attempt. The Cassandra log would indicate this because I get many o= f these:
ERROR [CompactionExecutor:22850] 2011-12-01 04:12:15,200 CompactionManager.= java (line 513) insufficie
nt space to compact even the two smallest fil= es, aborting

before I started getting many of these:
ERROR [Flush= Writer:283] 2011-12-01 04:12:22,917 AbstractCassandraDaemon.java (line 139)= Fatal exception in thread Thread[FlushWriter:283,5,main]
=09 =09 =09 java.lang.RuntimeException: java.lang.RuntimeE= xception: Insufficient disk space to flush 42531 bytes

I just want to clearly understand what happened.
=
Thanks,
Alex


On Thu, De= c 1, 2011 at 6:58 PM, Jeremiah Jordan <jeremiah.jordan@morningstar.com>= wrote:
=20 =20 =20
If you are writing data with QUORUM or ALL you should be safe to restart cassandra on that node.=A0 If the extra space is all from *tmp* files from compaction they will get deleted at startup.=A0 You will then need to run repair on that node to get back any data that was missed while it was full.=A0 If your commit log was on a different device you may not even have lost much.

-Jeremiah


On 12/01/2011 04:16 AM, Alexandru Dan Sicoe wrote:
Hello everyone,
=A04 node Cassandra 0.8.5 cluster with RF =3D2.
=A0One node started throwing exceptions in its log:

ERROR 10:02:46,837 Fatal exception in thread Thread[FlushWriter:1317,5,main]
java.lang.RuntimeException: java.lang.RuntimeException: Insufficient disk space to flush 17296 bytes
=A0=A0=A0=A0=A0=A0=A0 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:3= 4)
=A0=A0=A0=A0=A0=A0=A0 at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.j= ava:886)
=A0=A0=A0=A0=A0=A0=A0 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:= 908)
=A0=A0=A0=A0=A0=A0=A0 at java.lang.Thread.run(Thread.java:619)
Caused by: java.lang.RuntimeException: Insufficient disk space to flush 17296 bytes
=A0=A0=A0=A0=A0=A0=A0 at org.apache.cassandra.db.ColumnFamilyStore.getFlushPath(ColumnFamilyStore.ja= va:714)
=A0=A0=A0=A0=A0=A0=A0 at org.apache.cassandra.db.ColumnFamilyStore.createFlushWriter(ColumnFamilySto= re.java:2301)
=A0=A0=A0=A0=A0=A0=A0 at org.apache.cassandra.db.Memtable.writeSortedContents(Memtable.java:24= 6)
=A0=A0=A0=A0=A0=A0=A0 at org.apache.cassandra.db.Memtable.access$400(Memtable.java:49)
=A0=A0=A0=A0=A0=A0=A0 at org.apache.cassandra.db.Memtable$3.runMayThrow(Memtable.java:270)
=A0=A0=A0=A0=A0=A0=A0 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:3= 0)
=A0=A0=A0=A0=A0=A0=A0 ... 3 more

Checked disk and obviously it's 100% full.

How do I recover from this without loosing the data? I've got plenty of space on the other nodes, so I thought of doing a decommission which I understand reassigns ranges to the other nodes and replicates data to them. After that's done I plan on manually deleting the data on the node and then joining in the same cluster position with auto-bootstrap turned off so that I won't get back the old data and I can continue getting new data with the node.

Note, I would like to have 4 nodes in because the other three barely take the input load alone. These are just long running tests until I get some better machines.

On strange thing I found is that the data folder on the ndoe that filled up the disk is 150 GB (as measured with du) while the data folder on all other 3 nodes is 50 GB. At the same time, DataStax OpsCenter shows a size of around 50GB for all 4 nodes. I though that the node was making a major compaction at which time it filled up the disk....but even that doesn't make sense because shouldn't a major compaction just be capable of doubling the size= , not triple-ing it? Doesn anyone know how to explain this behavior?
Thanks,
Alex


--0016368320fcce8e3f04b30d0f9f--