cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jun Rao (JIRA)" <j...@apache.org>
Subject [jira] Commented: (CASSANDRA-193) Proactive repair
Date Fri, 13 Nov 2009 17:59:39 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12777586#action_12777586
] 

Jun Rao commented on CASSANDRA-193:
-----------------------------------

Now that I understood how the code really works, I don't have any big issue with the patch.
Some comments below:
1. To get us started, can we trigger a Merkle tree repair manually from nodeprobe. Something
like the following:
    trigger Merkle tree repair for a Keyspace/CF on a node from nodeprobe
    the range to be fixed is (token from left node, token from this node]
    each replica of this range computes Merkle tree independently, in parallel, through a
full scan of all SSTables.
    when done, Merkle trees are compared and repairs triggered, if necessary.
4. The code needs to be better documented. You need to explain the n-ary tree, the complete
binary tree and the relationship btw them. Attach the diagram you uploaded will be useful.
5. You need to name variables and methods better. For example, MerkleTree.SELF and  MerkleTree.CHILD
are not very intuitive. How about FullyInconsisten and PartiallyInconsistent. Ditto for ChildMatch.container().
6. In MerkleTree.difference(), shouldn't you add a case that returns an empty diff when the
trees are consistent?


> 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, mktree-and-binary-tree.png
>
>
> 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.


Mime
View raw message