[
https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782754#action_12782754
]
Stu Hood commented on CASSANDRA-193:
------------------------------------
> 10. AES.validator.prepare() requires buffering all keys in a node in memory.
> Can you push sampling into the key iterator?
If I remember correctly, prepare() samples the SSTable indexes, which are
themselves samples of the data on disk, and which are already sitting in
memory: at worst, it should only create a new list, but not actually clone any
keys.
> 11. I still think using XOR to combine MD5 for each row is simpler. In
> MerkleTree.TreeRange.validateHelper(),
> you can compute the hash directly at each leaf node, instead of having to
> descend to hashdepth.
I really don't think XORing row hashes together is a good idea. Any permutation
of the same set of hashes can cause a collision, and when it is row hashes that
are being XOR'd, the chance of collision goes through the roof. Collisions in
this context mean data won't get repaired, and will continue to not be repaired
until someone writes to the affected range.
> 12. Could you add some comments in MerkleTree that describe the flow of
> TableRepair?
Will do.
> 13. Some suggestions for notation changes : Node --> INode, Hash -- > LNode.
Based on Jonathan's comment, maybe these should just go back to 'Inner' and
'Leaf'.
> 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-1-tree-preparation.diff, 193-2-tree.diff,
> 193-2-tree.diff, 193-2-tree.diff, 193-3-aes-preparation.diff,
> 193-3-aes-preparation.diff, 193-3-aes-preparation.diff, 193-4-aes.diff,
> 193-4-aes.diff, 193-4-aes.diff, 193-5-manual-repair.diff,
> 193-5-manual-repair.diff, 193-6-inverted-filter.diff,
> 193-6-inverted-filter.diff, 193-7-disable-caching-and-fix-minimum-token.diff,
> 193-breakdown.txt, 193-breakdown.txt, 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.