[
https://issues.apache.org/jira/browse/CASSANDRA-2769?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13050081#comment-13050081
]
Alan Liang commented on CASSANDRA-2769:
---------------------------------------
Instead of letting DataTracker#markCompacting modify the subset of sstables to
be compacted, I think it might be cleaner if it didn't and relied on the
CompactionStrategy to select the correct sstables. We can do this by having the
CompactionStrategy get the non compacting sstables from the DataTracker and
work with those to generate the buckets. The strategy should also be
responsible for creating buckets that fit within the min/max thresholds.
#markCompacting would then be changed such that it can either accept/reject a
bucket to be compacted instead of modifying the subset. #markCompacting will
also serve to handle the race condition of the DataTracker being inaccurate,
whereby, it will move on to other buckets.
With this, we can avoid generating buckets that are already compacting and it
gives full control of what actually is compacted by the CompactionStrategy.
What do you guys think?
> 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.2
>
> 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