You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Marcus Eriksson (JIRA)" <ji...@apache.org> on 2014/01/17 20:30:36 UTC
[jira] [Comment Edited] (CASSANDRA-5351) Avoid repairing
already-repaired data by default
[ https://issues.apache.org/jira/browse/CASSANDRA-5351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13875155#comment-13875155 ]
Marcus Eriksson edited comment on CASSANDRA-5351 at 1/17/14 7:30 PM:
---------------------------------------------------------------------
Just pushed a work-in-progress-branch with a bunch of updates: https://github.com/krummas/cassandra/tree/marcuse/5351
* It locks the sstables for the whole duration of the repair, might be better to do a best-effort here, ie, remember what sstables we calculated merkle trees on and then if the sstable is gone during anticompaction, we dont mark that data as repaired - we will catch it next repair round instead.
* Adds a test for anticompaction
* For LCS, we always first check if there are STCS-compactions to do on the unrepaired data, reasoning being that new data is unrepaired, so if you have flushed 4 sstables, there will be a stcs compaction.
* For STCS, we just pick the one with the most sstables, probably not the best way, but we can tweak that heuristic later.
[~lyubent] - I think the bug you had above was because you didn't check if markCompacting was successful or not in AnticompactionSession.lockTables()
It "works" but atleast these things are needed;
* Fix streaming, keep the repairedAt times etc.
* Make full repairs work somehow (anticompacting the whole dataset is probably not preferable)
* Timeouts, if we lock all unrepaired sstables, we must handle stalled repairs somehow.
* Make anticompaction smarter, if we have 10 sstables that we should anticompact, why not combine anticompaction with compaction and create 2 new sstables, one with repaired data and one with unrepaired
* More tests, cleanups, refactor
was (Author: krummas):
Just pushed a work-in-progress-branch with a bunch of updates: https://github.com/krummas/cassandra/tree/marcuse/5351
* It locks the sstables for the whole duration of the repair, might be better to do a best-effort here, ie, remember what sstables we calculated merkle trees on
* Adds a test for anticompaction
* For LCS, we always first check if there are STCS-compactions to do on the unrepaired data, reasoning being that new data is unrepaired, so if you have flushed 4 sstables, there will be a stcs compaction.
* For STCS, we just pick the one with the most sstables, probably not the best way, but we can tweak that heuristic later.
[~lyubent] - I think the bug you had above was because you didn't check if markCompacting was successful or not in AnticompactionSession.lockTables()
It "works" but atleast these things are needed;
* Fix streaming, keep the repairedAt times etc.
* Make full repairs work somehow (anticompacting the whole dataset is probably not preferable)
* Timeouts, if we lock all unrepaired sstables, we must handle stalled repairs somehow.
* Make anticompaction smarter, if we have 10 sstables that we should anticompact, why not combine anticompaction with compaction and create 2 new sstables, one with repaired data and one with unrepaired
* More tests, cleanups, refactor
> Avoid repairing already-repaired data by default
> ------------------------------------------------
>
> Key: CASSANDRA-5351
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5351
> Project: Cassandra
> Issue Type: Task
> Components: Core
> Reporter: Jonathan Ellis
> Assignee: Lyuben Todorov
> Labels: repair
> Fix For: 2.1
>
> Attachments: 5351_node1.log, 5351_node2.log, 5351_node3.log, 5351_nodetool.log
>
>
> Repair has always built its merkle tree from all the data in a columnfamily, which is guaranteed to work but is inefficient.
> We can improve this by remembering which sstables have already been successfully repaired, and only repairing sstables new since the last repair. (This automatically makes CASSANDRA-3362 much less of a problem too.)
> The tricky part is, compaction will (if not taught otherwise) mix repaired data together with non-repaired. So we should segregate unrepaired sstables from the repaired ones.
--
This message was sent by Atlassian JIRA
(v6.1.5#6160)