cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Yang Yang (Commented) (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs
Date Mon, 03 Oct 2011 21:54:34 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119659#comment-13119659
] 

Yang Yang commented on CASSANDRA-3269:
--------------------------------------

ok, this looks to be what could lead to the symptoms:


let's say the commitlog position is 0 now.

Table.apply() :

  switchlock.readlock.lock();
          commitlog.instance.add(mutation) ====> executor.add(new LogRecordAdder())
                             // but the Adder is not really executed yet, just submitted.
         
         /// add to memtable
  switchlock.readlock.unlock();



then we have a flush.
  CFS.maybeSwitchMemtable()
 
  switchlock.writelock.lock();
    ctx = Commitlog.instance.getContext();   // returns 0 

  

/// now the logRecordAdder is executed , and advances the position to 199;
    
/// blahblah
   postflusher.executes(
      discardCompletedSegments
                 
             turnOffIfNotWritten() ====> check fails, so the CF written by the last Adder
                                         is not cleaned
  )
  unlock


as  a result, IF the CF is never written again, it will forever remain dirty in the segment.



then we need to maintain the order between the adder and the getContext() call in maybeSwitchMemtable()



                
> accumulated 100GB of commit logs
> --------------------------------
>
>                 Key: CASSANDRA-3269
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3269
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.0
>            Reporter: Yang Yang
>            Assignee: Sylvain Lebresne
>            Priority: Critical
>             Fix For: 1.0.0
>
>         Attachments: 3269-v2.txt, 3269-v3.txt, 3269.patch
>
>
> I ran my cluster for about 2 days. the cluster has 2 nodes. I restarted one box several
times, and the other one was always running. the one always running ended up accumulating
100GB of commit logs.
> this is 1.0.0 code from about Sept 15 in github. I kept the original setting for 
> #commitlog_total_space_in_mb: 4096
> i.e. commented out
> here is some sample of the output:
> -rw-r--r-- 1 yyang yyang 134217857 2011-09-28 03:51 CommitLog-1317181834810.log
> -rw-r--r-- 1 yyang yyang 134217869 2011-09-28 03:50 CommitLog-1317181764105.log
> -rw-r--r-- 1 yyang yyang 134217783 2011-09-28 03:49 CommitLog-1317181694633.log
> -rw-r--r-- 1 yyang yyang 134217750 2011-09-28 02:39 CommitLog-1317176955102.log
> yyang@ip-10-71-21-46:/mnt/cass/log/cassandra$ ls -lt /mnt/cass/lib//cassandra/commitlog/|wc
-l
> 727
> yyang@ip-10-71-21-46:/mnt/cass/log/cassandra$ du -s /mnt/cass/lib/cassandra/commitlog/

> 95095316        /mnt/cass/lib/cassandra/commitlog/

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message