[
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-v2.patch
0001-Do-compact-only-smallerSSTables-v2.patch
bq. For trunk patches, I'm not comfortable w/ 0001 reassigning the sstables
field on general principles either. We could have the compaction proceed using
smallerSSTables as a simpler alternative, but in general this organization
feels like negative progress from the 0.8
doCompaction/doCompactionWithoutSizeEstimation.
Attaching v2 that doesn't reassign the sstables field.
bq. I think Alan has a good point. I don't think it's an appropriate role of
the data tracker to modify the set of sstables to be compacted in a task.
I do not disagree with that. However I'd like that we fix trunk as a first
priority. It's a pain to work on other issues (CASSANDRA-2521 for instance)
while it is broken (and the goal must be to do our best to always have a
working trunk). The attached patches doesn't really change any behavior, it
just fixes the bugs, so let's get that in first before thinking about
refactoring.
> 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-v2.patch,
> 0001-Do-compact-only-smallerSSTables.patch,
> 0002-Only-compact-what-has-been-succesfully-marked-as-com-v2.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