[ https://issues.apache.org/jira/browse/CASSANDRA-2769?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Sylvain Lebresne updated CASSANDRA-2769: ---------------------------------------- Attachment: 0002-Only-compact-what-has-been-succesfully-marked-as-com.patch 0001-Do-compact-only-smallerSSTables.patch 0001-0.8.0-Remove-useless-unmarkCompacting-in-doCleanup.patch Alright, there is a bunch of problems, one of which affects 0.8 and trunk and could cause this stackTrace. The others are due to CASSANDRA-1610 and thus only affect trunk (but one of those can also result in the attached stackTrace). The problem affecting 0.8 and trunk is related to a left over line in doCleanup() that is wrongly unmarking a sstable from the compacting set before having removed it from the active set of sstables. Thus another compaction could start compacting this sstable and we'll end up marking the file as compacted twice (and we would have duplicated the sstable, which is a problem for counters). Patch 0001-0.8.0-Remove-useless-unmarkCompacting-in-doCleanup.patch removes it and is against 0.8. Trunk has a few problems of its own: * If disk space is not sufficient to compact all sstables, it computes the smallestSSTables set that fits, but doesn't use it. Attached first patch (0001-Do-compact-only-smallerSSTables.patch) fixes that. * The CompactionTask logic wrongly decorrelates the set of sstables that are successfully marked from the ones it did compact. That is, it grabs a list of sstables it wants to compact, then call markCompacting on them, but does not check if all of them are successfully marked and compact the original list instead. In effect, a task will recompact sstables that are already being compacted by other task and the given file will be compacted twice (or more) and marked compacted multiple times. Attached patch (0002-Only-compact-what-has-been-succesfully-marked-as-com.patch) fixes this by changing the sstables set of a given CompactionTask to whatever has been successfully marked only. Since the marking involves updating the task, I've move the logic to AbstractCompactionTask where it seems to make more sense to me. * For some reason, the markCompacting added for CompactionTasks was refusing to mark (and compact) anything if the set of sstable was bigger that MaxCompactionThreshold. This means that as soon as the number of sstables (of same size) in the column family would exceed the threshold, no compaction would be started. This is not the expected behavior. The second patch also fixes this by reusing the original markCompacting that handles this correctly. > Cannot Create Duplicate Compaction Marker > ----------------------------------------- > > Key: CASSANDRA-2769 > URL: https://issues.apache.org/jira/browse/CASSANDRA-2769 > Project: Cassandra > Issue Type: Bug > Affects Versions: 0.8.0 > Reporter: Benjamin Coverston > Assignee: Sylvain Lebresne > Fix For: 0.8.1, 1.0 > > Attachments: > 0001-0.8.0-Remove-useless-unmarkCompacting-in-doCleanup.patch, > 0001-Do-compact-only-smallerSSTables.patch, > 0002-Only-compact-what-has-been-succesfully-marked-as-com.patch > > > Concurrent compaction can trigger the following exception when two threads > compact the same sstable. DataTracker attempts to prevent this but apparently > not successfully. > java.io.IOError: java.io.IOException: Unable to create compaction marker > at > org.apache.cassandra.io.sstable.SSTableReader.markCompacted(SSTableReader.java:638) > at > org.apache.cassandra.db.DataTracker.removeOldSSTablesSize(DataTracker.java:321) > at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:294) > at > org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:255) > at > org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:932) > at > org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:173) > at > org.apache.cassandra.db.compaction.CompactionManager$1.call(CompactionManager.java:119) > at > org.apache.cassandra.db.compaction.CompactionManager$1.call(CompactionManager.java:102) > at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) > at java.util.concurrent.FutureTask.run(FutureTask.java:138) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:680) > Caused by: java.io.IOException: Unable to create compaction marker > at > org.apache.cassandra.io.sstable.SSTableReader.markCompacted(SSTableReader.java:634) > ... 12 more -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira