cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Stu Hood (JIRA)" <j...@apache.org>
Subject [jira] Commented: (CASSANDRA-193) Proactive repair
Date Wed, 18 Nov 2009 06:36:39 GMT

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

Stu Hood commented on CASSANDRA-193:
------------------------------------

Here is a rebased version of the patchset which should resolve the issues found during Jun's
review.

There are significant changes in patch 4: the list of "observers" that were waiting for valid
trees to be generated was too abstract. Instead, we keep a Cachetable of valid trees, and
when an endpoint receives a tree from another node (via TreeResponse) or generates a tree
locally, it tries to 'rendezvous' with trees from appropriate neighbors. See uses of AntiEntropyService.register().

There is also a new patch 5 to add a manual manual repair operation to nodeprobe, which sends
out TreeRequests to all natural endpoints for the target nodes token. The nodeprobe target
is the 'initiator' for the TreeRequests, so once the trees are generated on their respective
nodes, they are broadcast out as TreeResponses, and they will rendezvous on the target/initiator
to be differenced.

> 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-1-tree-preparation.diff, 193-2-tree.diff,
193-2-tree.diff, 193-3-aes-preparation.diff, 193-3-aes-preparation.diff, 193-4-aes.diff, 193-4-aes.diff,
193-5-manual-repair.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