cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sylvain Lebresne (Issue Comment Edited) (JIRA)" <j...@apache.org>
Subject [jira] [Issue Comment Edited] (CASSANDRA-3269) accumulated 100GB of commit logs
Date Tue, 04 Oct 2011 07:39:34 GMT

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

Sylvain Lebresne edited comment on CASSANDRA-3269 at 10/4/11 7:37 AM:
----------------------------------------------------------------------

{noformat}
switchlock.writelock.lock();
ctx = Commitlog.instance.getContext(); // returns 0

/// now the logRecordAdder is executed , and advances the position to 199;
{noformat}

That could (should) not happen. The commit log is mono-threaded by mean of its executor. And
every action on the commit log happens in a task on that executor. In particular Commitlog.instance.getContext()
push a task on the commit log executor. This means that a logRecordAdder that has been pushed
before the switchlock is grabbed for the flush cannot return a position that is after the
position return by getContext() at the beginning of the flush (i.e, it either return a greater
position in the same segment or a position in a newer segment). So thanks to the switchlock
(that stops writes momentarily), we know that that the ctx position for the flush is after
every write that has been done in the memtable we are flushing.
Now we call discardCompletedSegments later, so what can happen is that there has been writes
to the commit log between the time we had grabbed that flush position and the time discardCompletedSegments
is called. That is the goal of CL.lastWrite. If when we discard segments, there has been no
write on this segment after the flush position we are considering, then the segment can be
turnOff, otherwise there is still "active" write for the column family so we don't turn it
off. But if that happens (i.e, if lastWrite >= flushPosition), it means that the writes
have been done in a newer memtable than the one we just flushed. So the segment will be turnOff
when the flush for that newer memtable happens.

{quote}
you see that LocationInfo being flushed already, twice, once at position 0, once at 174, then
right after that, it tries to discard the segment, but complains that it's dirty, and the
only dirty bit is locationInfo itself. that is the part that doesn't sound right.
{quote}

That log is not fully conclusive, because it is entirely possible that there has been some
write to that commit log after the second flush position. In which case it's ok to not unmark
LocationInfo.

In particular, the last line of the log:
{noformat}
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 04:22:59,573 CommitLog.java (line 502) Not safe to delete
commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log);
dirty is LocationInfo (0), ; hasNext: false
{noformat}
shows that the segment is the active one (hasNext == false), so it's perfectly reasonable
to think there has been some write since position 174. The log is missing the value of lastWrite
in the message to be able to say if it's the case.
                
      was (Author: slebresne):
    {noformat}
switchlock.writelock.lock();
ctx = Commitlog.instance.getContext(); // returns 0

/// now the logRecordAdder is executed , and advances the position to 199;
{noformat}

That could (should) not happen. The commit log is mono-threaded by mean of its executor. And
every action on the commit log happens in a task on that executor. In particular Commitlog.instance.getContext()
push a task on the commit log executor. This means that a logRecordAdder that has been pushed
before the switchlock is grabbed for the flush cannot return a position that is after the
position return by getContext() at the beginning of the flush (i.e, it either return a greater
position in the same segment or a position in a newer segment). So thanks to the switchlock
(that stops writes momentarily), we know that that the ctx position for the flush is after
every write that has been done in the memtable we are flushing.
Now we call discardCompletedSegments later, so what can happen is that there has been writes
to the commit log between the time we had grabbed that flush position and the time discardCompletedSegments
is called. That is the goal of CL.lastWrite. If when we discard segments, there has been no
write on this segment after the flush position we are considering, then the segment can be
turnOff, otherwise there is still "active" write for the column family so we don't turn it
off. But if that happens (i.e, if lastWrite >= flushPosition), it means that the writes
have been done in a newer memtable than the one we just flushed. So the segment will be turnOff
when the flush for that newer memtable happens.

{quote}
you see that LocationInfo being flushed already, twice, once at position 0, once at 174, then
right after that, it tries to discard the segment, but complains that it's dirty, and the
only dirty bit is locationInfo itself. that is the part that doesn't sound right.
{quote}

That log is not fully conclusive, because it is entirely possible that there has been some
write to that commit log after the second flush position. In which case it's ok to not unmark
LocationInfo.

In particular, the last line of the log:
{noformat}
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 04:22:59,573 CommitLog.java (line 502) Not safe to delete
commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log);
dirty is LocationInfo (0), ; hasNext: false
{norormat}
shows that the segment is the active one (hasNext == false), so it's perfectly reasonable
to think there has been some write since position 174. The log is missing the value of lastWrite
in the message to be able to say if it's the case.
                  
> 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