[
https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12776601#action_12776601
]
Stu Hood commented on CASSANDRA-193:
------------------------------------
> 1. The high level question. When should the Merkle tree be computed?
I see that you're right about major compactions not always involving all
sstables: I'll make Validation dependent on all sstables being involved. At
some point, we need to make a decision allowing repairs to run more frequently,
but I don't think it should block this issue.
> Won't the above approach trigger too many unnecessary repairs?
We definitely want to minimize the time lag between the tree being generated on
one node, and then on another.
Absolutely optimal would be to have them begin generating the trees at the
exact same time, immediately after a flush. I don't think that is really
feasible though.
> One approach is to explicitly iterate through keys on all sstables...
Adding this operation seems like a good idea: effectively, it is a 'noop' major
compaction (which would still need to acquire compaction locks), which doesn't
actually output the merged data.
Going back to the "triggering too many repairs" problem: if repair was running
independently of other compactions, perhaps it could choose to only
"noop-compact" sstables that existed when the tree was generated on the other
node? Rather than stopping time, or trying to perform the validation in sync,
we would only repair the older data. With this approach, we might end up, for
example, repairing a key (based on older sstables) that is deleted by newer
sstables.
----
Again, thanks for the review! I'll start incorporating these changes tonight,
and think more about the possibility of repairing only older sstables.
> Proactive repair
> ----------------
>
> Key: CASSANDRA-193
> URL: https://issues.apache.org/jira/browse/CASSANDRA-193
> Project: Cassandra
> Issue Type: New Feature
> Components: Core
> Reporter: Jonathan Ellis
> Assignee: Stu Hood
> Fix For: 0.5
>
> Attachments: 193-1-tree-preparation.diff, 193-2-tree.diff,
> 193-3-aes-preparation.diff, 193-4-aes.diff
>
>
> Currently cassandra supports "read repair," i.e., lazy repair when a read is
> done. This is better than nothing but is not sufficient for some cases (e.g.
> catastrophic node failure where you need to rebuild all of a node's data on a
> new machine).
> Dynamo uses merkle trees here. This is harder for Cassandra given the CF
> data model but I suppose we could just hash the serialized CF value.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.