Re: MemtablePostFlusher with high number of pending calls?

2011-05-04 Thread Terje Marthinussen
Yes, some sort of data structure to coordinate this could reduce the problem as well. I made some comments on that at the end of 2558. I believe a coordinator could be in place both to - plan the start of compaction and - to coordinate compaction thread shutdown and tmp file deletion before we com

Re: MemtablePostFlusher with high number of pending calls?

2011-05-04 Thread Jonathan Ellis
Or we could "reserve" space when starting a compaction. On Wed, May 4, 2011 at 2:32 AM, Terje Marthinussen wrote: > Partially, I guess this may be a side effect of multithreaded compactions? > Before running out of space completely, I do see a few of these: >  WARN [CompactionExecutor:448] 2011-0

Re: MemtablePostFlusher with high number of pending calls?

2011-05-04 Thread Terje Marthinussen
Partially, I guess this may be a side effect of multithreaded compactions? Before running out of space completely, I do see a few of these: WARN [CompactionExecutor:448] 2011-05-02 01:08:10,480 CompactionManager.java (line 516) insufficient space to compact all requested files SSTableReader(path=

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Terje Marthinussen
Well, just did not look at these logs very well at all last night First out of disk message: 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 T

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Terje Marthinussen
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 ta

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Jonathan Ellis
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 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

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Terje Marthinussen
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

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Jonathan Ellis
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 wrote

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Terje Marthinussen
So yes, there is currently some 200GB empty disk. On Wed, May 4, 2011 at 3:20 AM, Terje Marthinussen 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

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Terje Marthinussen
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

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Jonathan Ellis
... and are there any exceptions in the log? On Tue, May 3, 2011 at 1:01 PM, Jonathan Ellis wrote: > Does it resolve down to 0 eventually if you stop doing writes? > > On Tue, May 3, 2011 at 12:56 PM, Terje Marthinussen > wrote: >> Cassandra 0.8 beta trunk from about 1 week ago: >> Pool Name    

Re: MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Jonathan Ellis
Does it resolve down to 0 eventually if you stop doing writes? On Tue, May 3, 2011 at 12:56 PM, Terje Marthinussen wrote: > Cassandra 0.8 beta trunk from about 1 week ago: > Pool Name                    Active   Pending      Completed > ReadStage                         0         0              5

MemtablePostFlusher with high number of pending calls?

2011-05-03 Thread Terje Marthinussen
Cassandra 0.8 beta trunk from about 1 week ago: Pool NameActive Pending Completed ReadStage 0 0 5 RequestResponseStage 0 0 87129 MutationStage 0 0 187298 ReadRe