You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Yang Yang (Created) (JIRA)" <ji...@apache.org> on 2011/09/28 22:28:45 UTC

[jira] [Created] (CASSANDRA-3269) accumulated 100GB of commit logs

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


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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119163#comment-13119163 ] 

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

this piece of logging is particularly interesting.  the LocationInfo CF is flushed, right after that, it's trying to discard oldest segment, but the discard fails saying that locationInfo bit is dirty. but supposedly the bit should have been turned off




 INFO [main] 2011-10-02 04:22:55,033 CommitLog.java (line 174) Log replay complete, 15600762 replayed mutations
 INFO [main] 2011-10-02 04:22:58,560 ColumnFamilyStore.java (line 651) flush position is ReplayPosition(segmentId=1317524063739,
 position=0)

 INFO [main] 2011-10-02 04:22:59,244 ColumnFamilyStore.java (line 665) Enqueuing flush of Memtable-LocationInfo@2115195389(29/36
 serialized/live bytes, 1 ops)
 INFO [FlushWriter:2] 2011-10-02 04:22:59,244 Memtable.java (line 283) flushing memtable LocationInfo
 INFO [FlushWriter:2] 2011-10-02 04:22:59,244 Memtable.java (line 287) entered lock flushing memtable LocationInfo
 INFO [FlushWriter:2] 2011-10-02 04:22:59,244 Memtable.java (line 290) really start flushing memtable LocationInfo
 INFO [FlushWriter:2] 2011-10-02 04:22:59,245 Memtable.java (line 236) Writing Memtable-LocationInfo@2115195389(29/36 serialized
/live bytes, 1 ops)
 INFO [FlushWriter:2] 2011-10-02 04:22:59,396 Memtable.java (line 272) Completed flushing /mnt/cass/lib/cassandra/data/system/Lo
cationInfo-h-4-Data.db (80 bytes)
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 04:22:59,397 CommitLog.java (line 461) discard completed log segments for ReplayPosition(segmentId=1317524063739, position=0), column family 0.
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 04:22:59,397 CommitLog.java (line 473) trying to discard segment CommitLog-1317524063739.log
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 04:22:59,398 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
 INFO [main] 2011-10-02 04:22:59,447 ColumnFamilyStore.java (line 651) flush position is ReplayPosition(segmentId=1317524063739, position=174)
 INFO [main] 2011-10-02 04:22:59,447 ColumnFamilyStore.java (line 665) Enqueuing flush of Memtable-LocationInfo@2010157886(53/66 serialized/live bytes, 2 ops)
 INFO [FlushWriter:2] 2011-10-02 04:22:59,448 Memtable.java (line 283) flushing memtable LocationInfo
 INFO [FlushWriter:2] 2011-10-02 04:22:59,448 Memtable.java (line 287) entered lock flushing memtable LocationInfo
 INFO [FlushWriter:2] 2011-10-02 04:22:59,448 Memtable.java (line 290) really start flushing memtable LocationInfo
 INFO [FlushWriter:2] 2011-10-02 04:22:59,448 Memtable.java (line 236) Writing Memtable-LocationInfo@2010157886(53/66 serialized/live bytes, 2 ops)
 INFO [FlushWriter:2] 2011-10-02 04:22:59,572 Memtable.java (line 272) Completed flushing /mnt/cass/lib/cassandra/data/system/LocationInfo-h-6-Data.db (163 bytes)
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 04:22:59,573 CommitLog.java (line 461) discard completed log segments for ReplayPosition(segmentId=1317524063739, position=174), column family 0.
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 04:22:59,573 CommitLog.java (line 473) trying to discard segment CommitLog-1317524063739.log
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
 WARN [MutationStage:23] 2011-10-02 04:23:21,015 Memtable.java (line 136) MemoryMeter uninitialized (jamm not specified as java :

                
> 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
>
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119388#comment-13119388 ] 

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

Sylvain:


I did try to move down the threshold to 300MB, but in that case, it does not seem to occur easily (I haven't seen the issue appear after running under a 300MB threshold for 10 hours).


but if you look at  comment https://issues.apache.org/jira/browse/CASSANDRA-3269?focusedCommentId=13119163&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13119163


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.


                
> 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

        

[jira] [Updated] (CASSANDRA-3269) possible early deletion of commit logs

Posted by "Yang Yang (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yang Yang updated CASSANDRA-3269:
---------------------------------

    Summary: possible early deletion of commit logs  (was: accumulated 100GB of commit logs)
    
> possible early deletion 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119106#comment-13119106 ] 

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

the following shows that createNewSegment() sees that for  the oldest segment file xxxxx63739.log , it's forcing out a flush of LocationInfo CF. it does this multiple times (at least 1000 times ... ), but the actualy CFS.forceFlush() always says that the CF is clean. so no flushing happens.



DEBUG [COMMIT-LOG-WRITER] 2011-10-02 12:29:22,217 CommitLog.java (line 572) create new segment
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 12:29:22,217 CommitLog.java (line 523)  commit log total size now:23488116331
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 12:29:22,217 CommitLog.java (line 578) forcing a flush on segments : CommitLog-1317524063739.log out of 176
 WARN [COMMIT-LOG-WRITER] 2011-10-02 12:29:22,217 ColumnFamilyStore.java (line 718) forceFlush requested but everything is clean
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 12:29:22,218 CommitLog.java (line 584) forcing out CF:LocationInfo
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 12:30:15,357 CommitLog.java (line 572) create new segment
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 12:30:15,357 CommitLog.java (line 523)  commit log total size now:23622334131


but when flushing (triggered by other cf) happens, the discardSegment() says that the oldest one is still dirty on LocationInfo.



DEBUG [COMMIT-LOG-WRITER] 2011-10-02 05:11:14,182 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is opsMetrics (1001), LocationInfo (0), Budget (1000), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 06:21:26,481 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), Budget (1000), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 06:21:26,868 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 06:32:15,341 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 06:32:20,905 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 06:32:39,725 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 06:32:54,023 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 06:33:01,934 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 06:33:11,167 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 08:16:28,250 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 08:16:36,208 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 08:16:56,037 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true
DEBUG [COMMIT-LOG-WRITER] 2011-10-02 08:17:18,003 CommitLog.java (line 502) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317524063739.log); dirty is LocationInfo (0), ; hasNext: true



since the isClean() of CFS is determined by columnFamilies.size() and isClean() of segment is determined by lastCFWrite(), it seems that these 2 data structures somehow got out of sync.


also an easy suspect is that the lastCFWrite is not concurrent, but I can't show an exact scenario of how this would lead to the symptom I see




                
> 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
>
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119423#comment-13119423 ] 

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

I changed the commitlog rotation size to 5MB and total to 200MB, see if the symptoms come out more easily
                
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13117493#comment-13117493 ] 

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

I don't think so, I have set the memtable_total_space to be very high
due to my larger memory available, and rely on the
commit_log_total_size to trigger flushing  (also I changed the
getLiveSize() ... as I said in the other JIRA I commented , since my
traffic contains a large portion of counter adds). so essentially the
only trigger for flushing should come from commit_log_total_size
threshold,  but I don't see these flushing requests coming out, it's
not that the flushing started and can't finish.


I'm still running it, let me set it to a smaller threshold and
hopefully get to the point soon.

On Thu, Sep 29, 2011 at 10:59 AM, Jonathan Ellis (Commented) (JIRA)
<ji...@apache.org> wrote:

                
> 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
>
> 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

        

[jira] [Issue Comment Edited] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119219#comment-13119219 ] 

Sylvain Lebresne commented on CASSANDRA-3269:
---------------------------------------------

Actually no. The cfLastWrite position is the last position we started writing at for the given column family, not the last position we finished writing to. More precisely, we populate cfLastWrite in CLS.turnOn() which itself is called in CLS.write(RowMutation) just before the actual write with the position prior to the write. In other words, if lastWritten == flushPosition, it can only mean that the flush grabbed the commit log position *before* the last write (and thus we don't want to turn the segment off).
                
> 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
>
> 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

        

[jira] [Resolved] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Resolved) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yang Yang resolved CASSANDRA-3269.
----------------------------------

    Resolution: Fixed
    
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119382#comment-13119382 ] 

Sylvain Lebresne commented on CASSANDRA-3269:
---------------------------------------------

To be clear, the committed patch is probably not the source of the problem Yang is seeing, so let's keep that open for now.

Yang, I don't see anything yet from the logs you've pasted that seem wrong per se. Can you try to lower the commit_log_total_size
to something small (like 300MB) and see if you can reproduce easily ? (you should hopefully never have more than 3 commit logs (2 even) since when the third one is created, we should be over the limit and the older one should be deleted).
                
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119402#comment-13119402 ] 

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

I'm a bit unclear about the logic in cfLastWrites vs position, it seems that cflastWrites take value from the logwriter position, which always increases, so cfLastWrite can be only smaller or equal than the position, which is the value obtained through maybeSwitchMemtable()---->getContext() .


but in log I see


 WARN [COMMIT-LOG-WRITER] 2011-10-03 15:48:38,038 CommitLogSegment.java (line 205) turnOffIfNotWritten:ad_ip_agent 50020888/3959
3935


the above logging is produced in 



    void turnOffIfNotWritten(Integer cfId, Integer flushPosition)
    {   
        Integer lastWritten = cfLastWrite.get(cfId);
                String keypace = Schema.instance.getCF(cfId).left;

                final ColumnFamilyStore cfs = Table.open(keypace).getColumnFamilyStore(cfId);
                logger.warn("turnOffIfNotWritten:" + cfs.columnFamily + " " + lastWritten + "/" + flushPosition);

        if (lastWritten == null || lastWritten < flushPosition)
            cfLastWrite.remove(cfId);
    }

                
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13120280#comment-13120280 ] 

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

thanks Sylvain.

now I see that cfWrite> flushPosition is  fine .
after I updated to the git HEAD  , the run in the last day has not produced excessive commitlogs. let me keep running and see if it's gone now


Yang
                
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13120257#comment-13120257 ] 

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

yeah I was wrong.

but as stated in comment https://issues.apache.org/jira/browse/CASSANDRA-3269?focusedCommentId=13119643&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13119643

there is hard evidence in that link that cfWrite has appeared to be > flushPosition.

this would indeed lead to the turnOffIfNotWritten() not turning off the dirty bit, this point is right, no???


                
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119359#comment-13119359 ] 

Sylvain Lebresne commented on CASSANDRA-3269:
---------------------------------------------

+1 on v3
                
> 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
>            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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119130#comment-13119130 ] 

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

I was suspecting there could be something wrong with the CommitLog.discardCompletedSegmentsInternal():

            if (segment.id == context.segment)
            {
                // Only unmark this segment if there were not write since the
                // ReplayPosition was grabbed.
                segment.turnOffIfNotWritten(id, context.position);
                maybeDiscardSegment(segment, iter);


then inside CommitLogSegment:
    void turnOffIfNotWritten(Integer cfId, Integer flushPosition)
    {
        Integer lastWritten = cfLastWrite.get(cfId);
        if (lastWritten == null || lastWritten < flushPosition)
            cfLastWrite.remove(cfId);
    }

should the comparison "lastWritten < flushPosition" be "<= " ?? I don't know the exact semantic of the "flushPosition",

then I looked at CommitLogSegment.write(), here the position comes from the underlying writer. but it only calls logWriter.stream.xxxxx , and not any update on the positions 

Jonathan could you please briefly explain the logic on the position update of CommitLogSegment.write() ?

Thanks a lot
Yang
                
> 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
>
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13120244#comment-13120244 ] 

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

bq. That could (should) not happen. The commit log is mono-threaded by mean of its executor.



this is no longer the case since it was changed (particularly getContext() ) in https://issues.apache.org/jira/browse/CASSANDRA-3253 due to possibility of deadlock
                
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13116848#comment-13116848 ] 

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

Thanks Jonathan. 
unfortunately I removed the logs. I'll keep the load testing, if this appears again, I'll update this bug
                
> 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
>
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13120248#comment-13120248 ] 

Sylvain Lebresne commented on CASSANDRA-3269:
---------------------------------------------

Hum, #3253 moved a flush to an unrelated executor, so that a task on the commit log executor don't wait for another task on the commit log executor *because* that commit log executor is mono-threaded and so this would deadlock. But the commit log executor is still mono-thread (all hell would break loose if it wasn't) and CL.getContext() does push a task on that executor.
                
> 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

        

[jira] [Updated] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yang Yang updated CASSANDRA-3269:
---------------------------------

    Comment: was deleted

(was: I was suspecting there could be something wrong with the CommitLog.discardCompletedSegmentsInternal():

            if (segment.id == context.segment)
            {
                // Only unmark this segment if there were not write since the
                // ReplayPosition was grabbed.
                segment.turnOffIfNotWritten(id, context.position);
                maybeDiscardSegment(segment, iter);


then inside CommitLogSegment:
    void turnOffIfNotWritten(Integer cfId, Integer flushPosition)
    {
        Integer lastWritten = cfLastWrite.get(cfId);
        if (lastWritten == null || lastWritten < flushPosition)
            cfLastWrite.remove(cfId);
    }

should the comparison "lastWritten < flushPosition" be "<= " ?? I don't know the exact semantic of the "flushPosition",

then I looked at CommitLogSegment.write(), here the position comes from the underlying writer. but it only calls logWriter.stream.xxxxx , and not any update on the positions 

Jonathan could you please briefly explain the logic on the position update of CommitLogSegment.write() ?

Thanks a lot
Yang)
    
> 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
>
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119171#comment-13119171 ] 

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

I think I found why:



CommitLogSegment.turnOffIfNotWritten() :
    void turnOffIfNotWritten(Integer cfId, Integer flushPosition)
    {
        Integer lastWritten = cfLastWrite.get(cfId);
        if (lastWritten == null || lastWritten < flushPosition)
            cfLastWrite.remove(cfId);
    }

the comparison "<" probably should be "<="

let's say we flush a Memtable, and we have only 1 commitlogsegment in system now. after the flush, no write has come in, now the post-flush job tries to discard segments, 
the context passed in points to the current pointer, which is the same as cfLastWrite, 
then the comparison fails here, and the bit in this logsegment is not removed.

later, either the cf is never flushed, or when it is being forcefully flushed, we find that there is no new writes to the CF, so the CF isclean(), so the flush is never submitted, and the post-flush can never be run, so the dirty bit can never be cleared.....

sounds right?
                
> 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
>
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Jonathan Ellis (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13117475#comment-13117475 ] 

Jonathan Ellis commented on CASSANDRA-3269:
-------------------------------------------

Are you simply writing faster than it can flush?
                
> 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
>
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ 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

        

[jira] [Updated] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Jonathan Ellis (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-3269:
--------------------------------------

    Attachment: 3269-v3.txt

v3 inlines cP entirely.
                
> 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
>            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

        

[jira] [Updated] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Jonathan Ellis (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-3269:
--------------------------------------

    Attachment: 3269-v2.txt

not to bikeshed this too much, but v2 moves context and currentPosition out of the try/catch to make it clear that cP will always be valid in the catch section.  also merging declaration and assignment of cP prevents bogus refactors like the original regression.
                
> 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
>            Priority: Critical
>             Fix For: 1.0.0
>
>         Attachments: 3269-v2.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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Jonathan Ellis (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13116783#comment-13116783 ] 

Jonathan Ellis commented on CASSANDRA-3269:
-------------------------------------------

Hmm.  Would be useful to have a debug log -- it would say this

{noformat}
                logger.debug("Not safe to delete commit log " + segment + "; dirty is " + segment.dirtyString() + "; hasNext: " + iter.hasNext());
{noformat}

whenever a CF was flushed but it can't remove a segment.

For bonus points, you could add a log message in CommitLog before it creates the forceFlush runnable in createNewSegment when the size cap is exceeded.
                
> 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
>
> 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

        

[jira] [Updated] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sylvain Lebresne updated CASSANDRA-3269:
----------------------------------------

    Attachment: 3269.patch

But there is something very wrong in the commit log code. The commit r1171248 broke it (we ended up always setting the position of the LastWrite at -1). That commit is from September 15th, so that could match with what you're seeing, even though it should have resulted in commit log segment being deleted sooner than is safe rather than later. Attaching patch to fix anyway.
                
> 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
>             Fix For: 1.0.0
>
>         Attachments: 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119937#comment-13119937 ] 

Sylvain Lebresne commented on CASSANDRA-3269:
---------------------------------------------

{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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13119643#comment-13119643 ] 

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

I added a warn() to CLS.turnOffIfNotWritten() :

    void turnOffIfNotWritten(Integer cfId, Integer flushPosition)
    {  
        Integer lastWritten = cfLastWrite.get(cfId);
                String keypace = Schema.instance.getCF(cfId).left;

                final ColumnFamilyStore cfs = Table.open(keypace).getColumnFamilyStore(cfId);
                logger.warn("turnOffIfNotWritten:" + cfs.columnFamily + " " + lastWritten + "/" + flushPosition);

        if (lastWritten == null || lastWritten < flushPosition)
            cfLastWrite.remove(cfId);
    }



I saw many cases where lastWritten is > than the flushPosition, this does not make sense. 
I think this is definitely going to lead to the un-purged commit logs symptoms, IF no more writes are added onto the dirty CFs in the oldest segment. so the analysis I gave before still holds, but we just need to find out why  lastWritten > flushPosition.

 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:19,552 CommitLogSegment.java (line 205) turnOffIfNotWritten:ad_impression_session 96030/46355
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:22,688 CommitLogSegment.java (line 205) turnOffIfNotWritten:ad_ip_agent 3070922/46355
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:24,397 CommitLogSegment.java (line 205) turnOffIfNotWritten:multi_click_filter 4996295/46522
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:25,576 CommitLogSegment.java (line 205) turnOffIfNotWritten:session_limit_filter 5242578/59906
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:27,013 CommitLogSegment.java (line 205) turnOffIfNotWritten:measuredSession 5241697/80685
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:28,740 CommitLogSegment.java (line 205) turnOffIfNotWritten:IpFilter 5242052/96877
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:28,919 CommitLogSegment.java (line 205) turnOffIfNotWritten:session_limit_filter 5242475/870755
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:29,187 CommitLogSegment.java (line 205) turnOffIfNotWritten:measuredSession 5242807/890241
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:29,471 CommitLogSegment.java (line 205) turnOffIfNotWritten:IpFilter 5242120/890241
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:43:29,690 CommitLogSegment.java (line 205) turnOffIfNotWritten:IpFilter 3422607/768691
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:45:36,674 CommitLogSegment.java (line 205) turnOffIfNotWritten:ad_impression_session 3552787/165636
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:45:39,484 CommitLogSegment.java (line 205) turnOffIfNotWritten:ad_ip_agent 5242441/165636
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:45:40,832 CommitLogSegment.java (line 205) turnOffIfNotWritten:multi_click_filter 5241748/165636
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:45:41,670 CommitLogSegment.java (line 205) turnOffIfNotWritten:ad_ip_agent 5242691/1387810
 WARN [COMMIT-LOG-WRITER] 2011-10-03 19:45:41,863 CommitLogSegment.java (line 205) turnOffIfNotWritten:multi_click_filter 5242353/1409533



                
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13120266#comment-13120266 ] 

Sylvain Lebresne commented on CASSANDRA-3269:
---------------------------------------------

{quote}
there is hard evidence in that link that cfWrite has appeared to be > flushPosition.
this would indeed lead to the turnOffIfNotWritten() not turning off the dirty bit, this point is right, no???
{quote}

Yes, it can and will happen that cfWrite > flushPosition. And then the dirty bit will not be turning off. That is right. However, if that happen, it means that there is a write for the column family in a memtable that is not flushed yet. And thus the flush of this memtable would clean the dirty bit off. The invariant that the code enforces  (unless there is a bug of course) is that 'for the flush of a given memtable, every write in that memtable will have been written to the commit log at a position that is < the flushPosition for *that* flush'. If follows that if cfWrite > flushPosition, there a write in a unflushed memtable.
                
> 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

        

[jira] [Commented] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13117437#comment-13117437 ] 

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

I ran the code for a while, it created a lot of commit logs (not yet over the 4G limit ), I shut it down, ran with the code with extra debugging. 

then it showed the following messages, saying that not safe to remove. but in the end, the old commit logs were indeed removed, and I have only a new one left. (see the ls output at the end)



 INFO [FlushWriter:2] 2011-09-29 16:50:28,410 Memtable.java (line 236) Writing Memtable-multi_click_filter@1810361645(106184307/1327303837 serialized/live bytes, 1538903 ops)
DEBUG [COMMIT-LOG-WRITER] 2011-09-29 16:50:28,410 CommitLog.java (line 501) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317314880183.log); dirty is ; hasNext: false
 INFO [FlushWriter:2] 2011-09-29 16:50:31,610 Memtable.java (line 272) Completed flushing /mnt/cass/lib/cassandra/data/testBudget_items/multi_click_filter-h-33-Data.db (118127883 bytes)
DEBUG [COMMIT-LOG-WRITER] 2011-09-29 16:50:31,611 CommitLog.java (line 461) discard completed log segments for ReplayPosition(segmentId=1317314880183, position=0), column family 1010.
 INFO [FlushWriter:2] 2011-09-29 16:50:31,611 Memtable.java (line 236) Writing Memtable-IpFilter@1199856819(109065609/1328000737 serialized/live bytes, 1580661 ops)
DEBUG [COMMIT-LOG-WRITER] 2011-09-29 16:50:31,612 CommitLog.java (line 501) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317314880183.log); dirty is ; hasNext: false
 INFO [FlushWriter:2] 2011-09-29 16:50:34,038 Memtable.java (line 272) Completed flushing /mnt/cass/lib/cassandra/data/testBudget_items/IpFilter-h-33-Data.db (107340235 bytes)
DEBUG [COMMIT-LOG-WRITER] 2011-09-29 16:50:34,040 CommitLog.java (line 461) discard completed log segments for ReplayPosition(segmentId=1317314880183, position=0), column family 1013.
 INFO [FlushWriter:2] 2011-09-29 16:50:34,040 Memtable.java (line 236) Writing Memtable-opsMetrics@1355039451(1297090/77250 serialized/live bytes, 15090 ops)
DEBUG [COMMIT-LOG-WRITER] 2011-09-29 16:50:34,040 CommitLog.java (line 501) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317314880183.log); dirty is ; hasNext: false
 INFO [FlushWriter:2] 2011-09-29 16:50:34,129 Memtable.java (line 272) Completed flushing /mnt/cass/lib/cassandra/data/testBudget_counters/opsMetrics-h-19-Data.db (6282 bytes)
DEBUG [COMMIT-LOG-WRITER] 2011-09-29 16:50:34,130 CommitLog.java (line 461) discard completed log segments for ReplayPosition(segmentId=1317314880183, position=0), column family 1001.
DEBUG [COMMIT-LOG-WRITER] 2011-09-29 16:50:34,130 CommitLog.java (line 501) Not safe to delete commit log CommitLogSegment(/mnt/cass/lib/cassandra/commitlog/CommitLog-1317314880183.log); dirty is ; hasNext: false






yyang@ip-10-71-21-46:/mnt/cass/log/cassandra$ ls -lt /mnt/cass/lib/cassandra/commitlog/
total 2474280
-rw-r--r-- 1 yyang yyang         0 2011-09-29 16:48 CommitLog-1317314880183.log
-rw-r--r-- 1 yyang yyang         0 2011-09-29 16:43 CommitLog-1317314635458.log
-rw-r--r-- 1 yyang yyang         0 2011-09-29 16:40 CommitLog-1317314406536.log
-rw-r--r-- 1 yyang yyang         0 2011-09-29 16:36 CommitLog-1317314186503.log
-rw-r--r-- 1 yyang yyang         0 2011-09-29 16:21 CommitLog-1317313281629.log
-rw-r--r-- 1 yyang yyang 115115322 2011-09-29 16:02 CommitLog-1317312032419.log
-rw-r--r-- 1 yyang yyang 134217870 2011-09-29 16:00 CommitLog-1317311854994.log
-rw-r--r-- 1 yyang yyang 134217761 2011-09-29 15:57 CommitLog-1317311685452.log
-rw-r--r-- 1 yyang yyang 134217843 2011-09-29 15:54 CommitLog-1317311520485.log
-rw-r--r-- 1 yyang yyang 134217795 2011-09-29 15:52 CommitLog-1317311347744.log
-rw-r--r-- 1 yyang yyang 134217777 2011-09-29 15:49 CommitLog-1317311185371.log
-rw-r--r-- 1 yyang yyang 134217819 2011-09-29 15:46 CommitLog-1317311021414.log
-rw-r--r-- 1 yyang yyang 134217766 2011-09-29 15:43 CommitLog-1317310849097.log
-rw-r--r-- 1 yyang yyang 134217800 2011-09-29 15:40 CommitLog-1317310682639.log
-rw-r--r-- 1 yyang yyang 134217867 2011-09-29 15:38 CommitLog-1317310520489.log
-rw-r--r-- 1 yyang yyang 134217882 2011-09-29 15:35 CommitLog-1317310339007.log
-rw-r--r-- 1 yyang yyang 134217857 2011-09-29 15:32 CommitLog-1317310162460.log
-rw-r--r-- 1 yyang yyang 134217876 2011-09-29 15:29 CommitLog-1317310000804.log
-rw-r--r-- 1 yyang yyang 134217883 2011-09-29 15:26 CommitLog-1317309831513.log
-rw-r--r-- 1 yyang yyang 134217886 2011-09-29 15:23 CommitLog-1317309658187.log
-rw-r--r-- 1 yyang yyang 134217846 2011-09-29 15:20 CommitLog-1317309490763.log
-rw-r--r-- 1 yyang yyang 134217741 2011-09-29 15:18 CommitLog-1317309323730.log
-rw-r--r-- 1 yyang yyang 134217828 2011-09-29 15:15 CommitLog-1317309154140.log
-rw-r--r-- 1 yyang yyang 134217738 2011-09-29 15:12 CommitLog-1317308989678.log
yyang@ip-10-71-21-46:/mnt/cass/log/cassandra$ ls -lt /mnt/cass/lib/cassandra/commitlog/
total 4
-rw-r--r-- 1 yyang yyang 270 2011-09-29 16:50 CommitLog-1317314880183.log

                
> 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
>
> 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

        

[jira] [Updated] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Sylvain Lebresne (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sylvain Lebresne updated CASSANDRA-3269:
----------------------------------------

    Priority: Critical  (was: Major)
    
> 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
>            Priority: Critical
>             Fix For: 1.0.0
>
>         Attachments: 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

        

[jira] [Updated] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yang Yang updated CASSANDRA-3269:
---------------------------------

    Comment: was deleted

(was: yeah I was wrong.

but as stated in comment https://issues.apache.org/jira/browse/CASSANDRA-3269?focusedCommentId=13119643&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13119643

there is hard evidence in that link that cfWrite has appeared to be > flushPosition.

this would indeed lead to the turnOffIfNotWritten() not turning off the dirty bit, this point is right, no???

)
    
> 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

        

[jira] [Reopened] (CASSANDRA-3269) accumulated 100GB of commit logs

Posted by "Yang Yang (Reopened) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yang Yang reopened CASSANDRA-3269:
----------------------------------


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



                
> 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