[
https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Stu Hood updated CASSANDRA-193:
-------------------------------
Attachment: 193-7-disable-caching-and-fix-minimum-token.diff
Alright, here is one final patch (7) that adds some tests for issues discovered
in the last 2 days. The patchset is made up of the latest versions of patches 1
through 7.
All performance issues have been resolved: from my tests, the new default value
of 64 for MerkleTree.RECOMMENDED_DEPTH is much more reasonable, and only adds
15% runtime to major compactions. The patch has been tested with all of the
partitioners and stress.py.
Additionally, I fixed a bug due to the minimum token being sorted first in
SSTables, while being contained in the last range of the MerkleTree. See the
changes in Validator.add(row).
There was also one fundamental issue: we were using MD5 for the sequential hash
of the perfect leaf nodes, but also for the binary hash of the perfect inner
nodes. As Cliff of the Dynamite project warned me (ages ago) the inner nodes
need to use an associative hash function if their values are going to be
cached. The performance without caching is sufficient, and I didn't want to
rush into picking an algorithm, so for now, caching is simply disabled, with a
TODO mentioning that we should pick an algorithm.
Please take a look! I'd really like to get this in before 0.5. Thanks!
> 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, 193-6-inverted-filter.diff,
> 193-6-inverted-filter.diff, 193-7-disable-caching-and-fix-minimum-token.diff,
> 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.