cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Oleg Anastasyev (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-5554) Growing pending compactions
Date Fri, 10 May 2013 06:43:16 GMT

     [ https://issues.apache.org/jira/browse/CASSANDRA-5554?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Oleg Anastasyev updated CASSANDRA-5554:
---------------------------------------

    Description: 
I noticed on one of our new cassandra production server, that "pending compactions" number
is steadily growing. The cluster is under low write load, so "compactions are not keeping
up" is not the case.

A quick investigation shown, that compactions are stopping far before all pending tasks are
completed. I also found, that if concurrent_compactors=1, background compactions are not happening
at all.

The bug is in BackgroundCompactionTask rescheduling logic. The executor pool "room control"
code in CompactionManager.submitBackground() does not reschedule next background cycle, if
executor.getActiveCount reach maximun pool size, so it is lost forever.

So I patched it to always schedules single background cycle.
    
> Growing pending compactions
> ---------------------------
>
>                 Key: CASSANDRA-5554
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-5554
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.2.4
>            Reporter: Oleg Anastasyev
>         Attachments: patch.diff, pending_compactions_fixed.png
>
>
> I noticed on one of our new cassandra production server, that "pending compactions" number
is steadily growing. The cluster is under low write load, so "compactions are not keeping
up" is not the case.
> A quick investigation shown, that compactions are stopping far before all pending tasks
are completed. I also found, that if concurrent_compactors=1, background compactions are not
happening at all.
> The bug is in BackgroundCompactionTask rescheduling logic. The executor pool "room control"
code in CompactionManager.submitBackground() does not reschedule next background cycle, if
executor.getActiveCount reach maximun pool size, so it is lost forever.
> So I patched it to always schedules single background cycle.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message