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 <tmarthinus...@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 <jbel...@gmail.com> wrote: >> >> ... and are there any exceptions in the log? >> >> On Tue, May 3, 2011 at 1:01 PM, Jonathan Ellis <jbel...@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 >> > <tmarthinus...@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