cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Terje Marthinussen <tmarthinus...@gmail.com>
Subject Re: MemtablePostFlusher with high number of pending calls?
Date Tue, 03 May 2011 21:34:48 GMT
Hm... peculiar.

Post flush is not involved in compactions, right?

May 2nd
01:06 - Out of disk
01:51 - Starts a mix of major and minor compactions on different column
families
It then starts a few minor compactions extra over the day, but given that
there are more than 1000 sstables, and we are talking 3 minor compactions
started, it is not normal I think.
May 3rd 1 minor compaction started.

When I checked today, there was a bunch of tmp files on the disk with last
modify time from 01:something on may 2nd and 200GB empty disk...

Definitely no compaction going on.
Guess I will add some debug logging and see if I get lucky and run out of
disk again.

Terje

On Wed, May 4, 2011 at 5:06 AM, Jonathan Ellis <jbellis@gmail.com> wrote:

> Compaction does, but flush didn't until
> https://issues.apache.org/jira/browse/CASSANDRA-2404
>
> On Tue, May 3, 2011 at 2:26 PM, Terje Marthinussen
> <tmarthinussen@gmail.com> wrote:
> > Yes, I realize that.
> > I am bit curious why it ran out of disk, or rather, why I have 200GB
> empty
> > disk now, but unfortunately it seems like we may not have had monitoring
> > enabled on this node to tell me what happened in terms of disk usage.
> > I also thought that compaction was supposed to resume (try again with
> less
> > data) if it fails?
> > Terje
> >
> > On Wed, May 4, 2011 at 3:50 AM, Jonathan Ellis <jbellis@gmail.com>
> wrote:
> >>
> >> post flusher is responsible for updating commitlog header after a
> >> flush; each task waits for a specific flush to complete, then does its
> >> thing.
> >>
> >> so when you had a flush catastrophically fail, its corresponding
> >> post-flush task will be stuck.
> >>
> >> On Tue, May 3, 2011 at 1:20 PM, Terje Marthinussen
> >> <tmarthinussen@gmail.com> wrote:
> >> > Just some very tiny amount of writes in the background here (some
> hints
> >> > spooled up on another node slowly coming in).
> >> > No new data.
> >> >
> >> > I thought there was no exceptions, but I did not look far enough back
> in
> >> > the
> >> > log at first.
> >> > Going back a bit further now however, I see that about 50 hours ago:
> >> > ERROR [CompactionExecutor:387] 2011-05-02 01:16:01,027
> >> > AbstractCassandraDaemon.java (line 112) Fatal exception in thread
> >> > Thread[CompactionExecutor:387,1,main]
> >> > java.io.IOException: No space left on device
> >> >         at java.io.RandomAccessFile.writeBytes(Native Method)
> >> >         at java.io.RandomAccessFile.write(RandomAccessFile.java:466)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.io.util.BufferedRandomAccessFile.flush(BufferedRandomAccessFile.java:160)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.io.util.BufferedRandomAccessFile.reBuffer(BufferedRandomAccessFile.java:225)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.io.util.BufferedRandomAccessFile.writeAtMost(BufferedRandomAccessFile.java:356)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.io.util.BufferedRandomAccessFile.write(BufferedRandomAccessFile.java:335)
> >> >         at
> >> >
> org.apache.cassandra.io.PrecompactedRow.write(PrecompactedRow.java:102)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:130)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:566)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:146)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:112)
> >> >         at
> >> > java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> >> >         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> >> >         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:662)
> >> > [followed by a few more of those...]
> >> > and then a bunch of these:
> >> > ERROR [FlushWriter:123] 2011-05-02 01:21:12,690
> >> > AbstractCassandraDaemon.java
> >> > (line 112) Fatal exception in thread Thread[FlushWriter:123,5,main]
> >> > java.lang.RuntimeException: java.lang.RuntimeException: Insufficient
> >> > disk
> >> > space to flush 40009184 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:662)
> >> > Caused by: java.lang.RuntimeException: Insufficient disk space to
> flush
> >> > 40009184 bytes
> >> >         at
> >> >
> >> >
> org.apache.cassandra.db.ColumnFamilyStore.getFlushPath(ColumnFamilyStore.java:597)
> >> >         at
> >> >
> >> >
> org.apache.cassandra.db.ColumnFamilyStore.createFlushWriter(ColumnFamilyStore.java:2100)
> >> >         at
> >> >
> org.apache.cassandra.db.Memtable.writeSortedContents(Memtable.java:239)
> >> >         at
> org.apache.cassandra.db.Memtable.access$400(Memtable.java:50)
> >> >         at
> >> > org.apache.cassandra.db.Memtable$3.runMayThrow(Memtable.java:263)
> >> >         at
> >> >
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> >> >         ... 3 more
> >> > Seems like compactions stopped after this (a bunch of tmp tables there
> >> > still
> >> > from when those errors where generated), and I can only suspect the
> post
> >> > flusher may have stopped at the same time.
> >> > There is 890GB of disk for data, sstables are currently using 604G
> >> > (139GB is
> >> > old tmp tables from when it ran out of disk) and "ring" tells me the
> >> > load on
> >> > the node is 313GB.
> >> > Terje
> >> >
> >> >
> >> > On Wed, May 4, 2011 at 3:02 AM, Jonathan Ellis <jbellis@gmail.com>
> >> > wrote:
> >> >>
> >> >> ... and are there any exceptions in the log?
> >> >>
> >> >> On Tue, May 3, 2011 at 1:01 PM, Jonathan Ellis <jbellis@gmail.com>
> >> >> wrote:
> >> >> > Does it resolve down to 0 eventually if you stop doing writes?
> >> >> >
> >> >> > On Tue, May 3, 2011 at 12:56 PM, Terje Marthinussen
> >> >> > <tmarthinussen@gmail.com> wrote:
> >> >> >> Cassandra 0.8 beta trunk from about 1 week ago:
> >> >> >> Pool Name                    Active   Pending      Completed
> >> >> >> ReadStage                         0         0            
 5
> >> >> >> RequestResponseStage              0         0          87129
> >> >> >> MutationStage                     0         0         187298
> >> >> >> ReadRepairStage                   0         0            
 0
> >> >> >> ReplicateOnWriteStage             0         0            
 0
> >> >> >> GossipStage                       0         0        1353524
> >> >> >> AntiEntropyStage                  0         0            
 0
> >> >> >> MigrationStage                    0         0            
10
> >> >> >> MemtablePostFlusher               1       190            108
> >> >> >> StreamStage                       0         0            
 0
> >> >> >> FlushWriter                       0         0            302
> >> >> >> FILEUTILS-DELETE-POOL             0         0            
26
> >> >> >> MiscStage                         0         0            
 0
> >> >> >> FlushSorter                       0         0            
 0
> >> >> >> InternalResponseStage             0         0            
 0
> >> >> >> HintedHandoff                     1         4            
 7
> >> >> >>
> >> >> >> Anyone with nice theories about the pending value on the memtable
> >> >> >> post
> >> >> >> flusher?
> >> >> >> Regards,
> >> >> >> Terje
> >> >> >
> >> >> >
> >> >> >
> >> >> > --
> >> >> > Jonathan Ellis
> >> >> > Project Chair, Apache Cassandra
> >> >> > co-founder of DataStax, the source for professional Cassandra
> support
> >> >> > http://www.datastax.com
> >> >> >
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Jonathan Ellis
> >> >> Project Chair, Apache Cassandra
> >> >> co-founder of DataStax, the source for professional Cassandra support
> >> >> http://www.datastax.com
> >> >
> >> >
> >>
> >>
> >>
> >> --
> >> Jonathan Ellis
> >> Project Chair, Apache Cassandra
> >> co-founder of DataStax, the source for professional Cassandra support
> >> http://www.datastax.com
> >
> >
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of DataStax, the source for professional Cassandra support
> http://www.datastax.com
>

Mime
View raw message