You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jonathan Ellis (JIRA)" <ji...@apache.org> on 2009/05/21 21:48:45 UTC

[jira] Created: (CASSANDRA-193) Proactive repair

Proactive repair
----------------

                 Key: CASSANDRA-193
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
             Project: Cassandra
          Issue Type: New Feature
            Reporter: Jonathan Ellis
             Fix For: 0.4


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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: CASSANDRA-193.diff

Here is a first cut of the MerkleTree datastructure. There are a few issues that still need work:

 * TreeRange.validate(Iterator) will need to take an iterator over ColumnFamily (Token,hash) pairs. It currently takes an iterator over Tokens, and uses a dummy hash value,
 * We're still blocked by CASSANDRA-242: this patch contains a hacked implementation of Range.midpoint(Token, Token) that only works with BigIntegerToken.

I'll start looking at CASSANDRA-242 for now so we can actually gets this integrated.

> 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: CASSANDRA-193.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.


[jira] Issue Comment Edited: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722054#action_12722054 ] 

Stu Hood edited comment on CASSANDRA-193 at 6/20/09 9:16 PM:
-------------------------------------------------------------

The current plan is for an AntiEntropyService per table to maintain a Merkle Tree per column family.

The tree will be implemented as a full, randomized binary tree in memory (a ''Treap'': http://en.wikipedia.org/wiki/Treap ), where every item in the tree represents a range bounded by the dht.Tokens of its left and right neighbors. By placing a bound on the total number of nodes in the tree, we can limit the memory usage. We can compact or split ranges in the tree by removing or adding Tokens. The algorithm for deciding which ranges to compact/split will be described below.

When a write comes in for a given table, we will place 'invalidation' operations in a queue for all affected column families. The ExecutorService for the table will read from the queue and perform the 'invalidations' as fast as it can. For a given Key/Token, if any column family tree is marked as 'invalid', the entire row needs to be read from disk and repaired (at some point in the future).

An 'invalidation' operation does a binary search in the Merkle Tree and marks the matching range as 'invalid', deleting its hash. We will also take advantage of this step to optimize the tree: A ''Treap'' stores a random priority (P) on each node, and by generating a random P' and replacing P for a node iff P' < P as we invalidate it, more frequently invalidated ranges will shift to the top of the tree.

The AEService maintaining the tree for a table will occasionally need to exchange portions of the tree with other nodes. In order to do this, subtrees that both nodes are interested in from all CF trees will have to be locked long enough to recalculate all 'invalid' children, and then the locks can flow down the tree as progressively smaller ranges are exhanged. Doing this locking efficiently is going to be interesting (aka: I haven't thought about it).

Implementing the exchange between nodes is blocked by CASSANDRA-242 because in order to align the subtrees on different nodes, we need to be able to deterministically split two ranges.

In order to fill in 'invalid' ranges in the tree, the MerkleTree will provide an operation that builds a list of invalid ranges to be fetched from disk. During this step, we can also compact/split ranges. Because of our Treap maintenance, frequently invalidated ranges will be nearer to the top of the tree, and stable ranges will be closer to the bottom. By compacting the deepest N leaves and expanding the shallowest N, we can minimize the size of the ranges that are affected by invalidations in the future.

Given the list of 'invalid' ranges (and pointers directly to the tree nodes), the AEService will fetch the ranges from the current MemTable and SSTables for the CF, hash them, and store the hashes into the relevant nodes. After this operation, we can recursively calculate hashes for inner nodes.

      was (Author: stuhood):
    The current plan is for an AntiEntropyService per table to maintain a Merkle Tree per column family.

The tree will be a implemented as a randomized binary tree in memory (a ''Treap'': http://en.wikipedia.org/wiki/Treap), where every item in the tree represents a range bounded by the dht.Tokens of its left and right neighbors. By placing a bound on the total number of nodes in the tree, we can limit the memory usage. We can compact or split ranges in the tree by removing or adding Tokens. The algorithm for deciding which ranges to compact/split will be described below.

When a write comes in for a given table, we will place 'invalidation' operations in a queue for all affected column families. The ExecutorService for the table will read from the queue and perform the 'invalidations' as fast as it can. For a given Key/Token, if any column family tree is marked as 'invalid', the entire row needs to be read from disk and repaired (at some point in the future).

An 'invalidation' operation does a binary search in the Merkle Tree and marks the matching range as 'invalid', deleting its hash. We will also take advantage of this step to optimize the tree: A ''Treap'' stores a random priority (P) on each node, and by generating a random P' and replacing P for a node iff P' < P as we invalidate it, more frequently invalidated ranges will shift to the top of the tree.

The AEService maintaining the tree for a table will occasionally need to exchange portions of the tree with other nodes. In order to do this, subtrees that both nodes are interested in from all CF trees will have to be locked long enough to recalculate all 'invalid' children, and then the locks can flow down the tree as progressively smaller ranges are exhanged. Doing this locking efficiently is going to be interesting (aka: I haven't thought about it).

Implementing the exchange between nodes is blocked by CASSANDRA-242 because in order to align the subtrees on different nodes, we need to be able to deterministically split two ranges.

In order to fill in 'invalid' ranges in the tree, the MerkleTree will provide an operation that builds a list of invalid ranges to be fetched from disk. During this step, we can also compact/split ranges. Because of our Treap maintenance, frequently invalidated ranges will be near the top of the tree, and stable ranges will be closer to the leaves. By compacting the deepest N nodes and expanding the shallowest N, we can minimizing the size of the ranges that are affected by invalidations in the future.

Given the list of 'invalid' ranges (and pointers directly to the tree nodes), the AEService will fetch the ranges from the current MemTable and SSTables for the CF, hash them, and store the hashes into the relevant nodes. After this operation, we can recursively calculate hashes for inner nodes.
  
> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12723647#action_12723647 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

So you are basically looking at doing a bulk scan no matter what.

Then the question becomes, is it worth trying to keep (partial) results of that scan in memory to avoid re-doing the work next time around.  If writes are randomly distributed across the range then ISTM the answer is a clear No, but I'm not sure how close real-world workloads would come to that.

Does it make sense to start with a non-caching version like I describe?  I think that a lot of the functionality implemented would be reusable, and it would give us a useful starting point to give us a better feel for the additional complexity your "full" version would entail.

Incidently, I don't think we should worry about the kind of locking you mentioned.  The point of the merkle tree is to save us from exchanging 90% of keys; if we don't lock and the trees end up slightly de-synced and we exchange a few keys at the edges that we wouldn't have to with a "perfect" algorithm, that is a price I'm happy to pay for a much less complicated solution.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment:     (was: CASSANDRA-193.diff)

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780912#action_12780912 ] 

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

The comment explains it better, but you need something like XOR, which is associative: (1 ^ 2) ^ 3 == 1 ^ (2 ^ 3) in order to cache partially computed values. MD5 needs to be computed sequentially over all of the inputs, so it can be used for leaves, but not inner nodes.

> 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.


[jira] Issue Comment Edited: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12776600#action_12776600 ] 

Stu Hood edited comment on CASSANDRA-193 at 11/12/09 4:52 AM:
--------------------------------------------------------------

Thanks a lot for the review Jun! I'll respond to #1 in a separate comment, since it is a design issue that still needs a lot of discussion.

> 2
> 2.1 In MerkleTree.Node.insert, why do you increment the depth of the left child even when the node doesn't split?
Node.insert() is only used during split operations (perhaps it is misnamed... but this is not quite a traditional B-Tree). The child to the left is the node that contains the Token we were splitting on, and whenever we split a range we increment its depth to indicate how far it is from being a complete (0,0] range. I'll add a comment to this effect.

> 2.2 In the same function, if the node does split, where is the code to shrink the children list in the splitted node to half?
Node.insert() uses List.subList() and List.clear() inline to take half of its neighbor's children.

> 2.3 In the same function, do you have to keep calling invalidate during insertion?
The original design assumed that the tree was going to live for a while in memory, and be maintained between repair sessions, so the split operation is intended to be used on a tree that might be partially valid. We might be able to have the initial building of the tree skip this check somehow, but I don't think the 'hash == null? hash = null;' check is too intensive.

> 3
> 3.1 In Validator.add, there is comment about generating a new range. However, no code does that.
The "private MerkleTree.TreeRangeIterator ranges" variable is an iterator generated by the MerkleTree: it iterates in order over all of the ranges in the tree that have null hashes.

> 3.2 In TreeRange.validateHelper [...] Why do you have to compute multiple hash values recursively?
The reasoning here is that a MerkleTree is supposed to be a sparse representation of a 'perfect/complete' binary tree. Each leaf of the perfect tree represents a hashed range, and each inner node represents a binary hash of its two children. The perfect tree is of depth "hashdepth", so when validateHelper() reaches the maximum/hashdepth, it is in one of the leaves of the perfect tree, and rows are hashed sequentially there. <EDIT comment="Ignore everything in in this tag">If a single call to validate() starts in a TreeRange that is at depth == maxdepth, then what is stored in the MkT.Node is the value of a perfect leaf, otherwise, the MkT.Node is storing the value of a perfect inner node.</EDIT>

I'll probably copy and past this exact explanation into the next revision =x

> 4. I need some text description to really follow the Differencer code.
I'll make sure this gets in the next revision, but basically:
 1. It recurses using midpoint as long as both trees have the resolution to continue, and are not equal.
 2a. If it finds a range that has only one invalid child, it adds that child range, since that is the smallest possible invalid range contained in the parent.
 2b. Otherwise, if both children are invalid (and since it can't recurse deeper), the parent range is entirely invalid, and recursion keeps rolling up until 2a is met.

> 5. The Hashable class is confusing.
You're right: I definitely should have worried more about overloading the word "hash": I'll see what I can do about this one.

> 6. The repair logic is missing in Differencer.
The repair logic is the one FIXME for this ticket: CASSANDRA-520 deals with implementing the actual repair logic.

Thanks again for taking time out to look at this! 

      was (Author: stuhood):
    Thanks a lot for the review Jun! I'll respond to #1 in a separate comment, since it is a design issue that still needs a lot of discussion.

> 2
> 2.1 In MerkleTree.Node.insert, why do you increment the depth of the left child even when the node doesn't split?
Node.insert() is only used during split operations (perhaps it is misnamed... but this is not quite a traditional B-Tree). The child to the left is the node that contains the Token we were splitting on, and whenever we split a range we increment its depth to indicate how far it is from being a complete (0,0] range. I'll add a comment to this effect.

> 2.2 In the same function, if the node does split, where is the code to shrink the children list in the splitted node to half?
Node.insert() uses List.subList() and List.clear() inline to take half of its neighbor's children.

> 2.3 In the same function, do you have to keep calling invalidate during insertion?
The original design assumed that the tree was going to live for a while in memory, and be maintained between repair sessions, so the split operation is intended to be used on a tree that might be partially valid. We might be able to have the initial building of the tree skip this check somehow, but I don't think the 'hash == null? hash = null;' check is too intensive.

> 3
> 3.1 In Validator.add, there is comment about generating a new range. However, no code does that.
The "private MerkleTree.TreeRangeIterator ranges" variable is an iterator generated by the MerkleTree: it iterates in order over all of the ranges in the tree that have null hashes.

> 3.2 In TreeRange.validateHelper [...] Why do you have to compute multiple hash values recursively?
The reasoning here is that a MerkleTree is supposed to be a sparse representation of a 'perfect/complete' binary tree. Each leaf of the perfect tree represents a hashed range, and each inner node represents a binary hash of its two children. The perfect tree is of depth "hashdepth", so when validateHelper() reaches the maximum/hashdepth, it is in one of the leaves of the perfect tree, and rows are hashed sequentially there. If a single call to validate() starts in a TreeRange that is at depth == maxdepth, then what is stored in the MkT.Node is the value of a perfect leaf, otherwise, the MkT.Node is storing the value of a perfect inner node.

I'll probably copy and past this exact explanation into the next revision =x

> 4. I need some text description to really follow the Differencer code.
I'll make sure this gets in the next revision, but basically:
 1. It recurses using midpoint as long as both trees have the resolution to continue, and are not equal.
 2a. If it finds a range that has only one invalid child, it adds that child range, since that is the smallest possible invalid range contained in the parent.
 2b. Otherwise, if both children are invalid (and since it can't recurse deeper), the parent range is entirely invalid, and recursion keeps rolling up until 2a is met.

> 5. The Hashable class is confusing.
You're right: I definitely should have worried more about overloading the word "hash": I'll see what I can do about this one.

> 6. The repair logic is missing in Differencer.
The repair logic is the one FIXME for this ticket: CASSANDRA-520 deals with implementing the actual repair logic.

Thanks again for taking time out to look at this! 
  
> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: mktree-and-binary-tree.png

> 3.2 Why is Merkle tree a binary tree? I thought it's an n-ary tree since you
> can specify the order of each node.
This MerkleTree data structure is an n-ary tree that represents a perfect binary hash tree.

> Since a leaf node corresponds to a single range, why do you need a list of hashes?
> I thought each leaf node maintains a single hash value for rows its range and each
> inner node maintains a single hash that is the logical AND of the hashes of all its
> children. Is this not correct?
That is not correct... I'm sorry, I mispoke above... I'll edit that comment and remove the bad information. It is the leaves of the perfect tree which contain sequentially hashed values, and the inner nodes of the perfect tree contain the logical AND hash of their children.

I'm attaching an example. This perfect binary tree has a max depth of 2, so the maximum depth of a MkT.Hash in the MkTree representing it is 2 as well. In this case though, the MkTree has only been split twice, so it contains a single MkT.Leaf and three MkT.Hashes. Each MkT.Hash represents a node from the perfect tree, but they are not all at the same depth in the tree. MkT.Nodes will not always correspond to a node from the binary tree (because the orders are not equal), but when they do, we can cache a value in that MkT.Node.

The important thing to notice, is that if the MkTree had been split at 8 and 12 instead of 4 and 8, it would still represent the same perfect binary tree, which is why we can compare MkTrees generated by different Cassandra endpoints.

The MkTree.Hash for (8, 0] represents an inner node in the perfect tree, so in order to calculate it in the same manner no matter how a Cassandra endpoint chooses to split the MkTree, during validateHelper() we always generate a perfect tree down to the maximum hash depth, sequentially hash rows into the perfect leaves, and then calculate the logical AND back up to the range needed by validate().

> 2.1 Still confused. So node.depth doesn't mean the depth of the node in the tree?
All depths in the code refer to the depth of the perfect binary tree that a Node or Hash represents. This is why MkT.Nodes can only occasionally contain hash values: it's fairly common for a N-ary tree to contain a node representing a range that isn't represented by a single node in a binary tree.

> 3.1 The ranges don't change after the iterator is generated, right? But inside the
> while loop in Validator.add, there is comment about adding a new range.
The comment says "generate a new range": it is asking for TreeRangeIterator.next(), which hides the generation of the next range that needs to be hashed, and returns it.

----

I hope this helps! Thanks again for checking it out.

> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-5-manual-repair.diff
                193-4-aes.diff

> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment:     (was: 193-breakdown.txt)

> 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-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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-breakdown.txt
                193-6-inverted-filter.diff

Adding one more patch: I did some system level testing today and noticed a really dumb bug that slipped through unit testing. Other than that, the system tests look good. I feel good about this patchset being ready for merge.

(The filter to remove the local node from the list of natural endpoints was inverted, but the Gossiper only has one endpoint in tests, which was hiding the fact that we weren't caching a generated tree locally.)

> 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-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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12786221#action_12786221 ] 

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

The new patch looks good to me. I think this is ready to be committed unless someone objects.

Could you rebase eveything to trunk? I can't seem to apply patch-6 to my local branch.

> 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-6-repair-explanation-and-node-rename.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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-breakdown.txt

> 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-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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12778749#action_12778749 ] 

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

> 1. To get us started, can we trigger a Merkle tree repair manually from nodeprobe.
I got most of this working tonight: the code needed some serious cleanup in order to not duplicate effort or send trees back and forth in an infinite loop.

> 4. The code needs to be better documented.
Next patch should be improved in this area.

> 5. You need to name variables and methods better. For example...
These were particularly bad... you're right. I've changed them.

> 6. In MerkleTree.difference(), shouldn't you add a case that returns an empty diff when the trees are consistent?
Good catch.

I'll wrap up a new version of this patch on Tuesday before COB, I promise.

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12786462#action_12786462 ] 

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

Committed. Thanks Stu.

> 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-1-tree-preparation.diff, 193-2-tree.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-3-aes-preparation.diff, 193-4-aes.diff, 193-4-aes.diff, 193-4-aes.diff, 193-4-aes.diff, 193-5-manual-repair.diff, 193-5-manual-repair.diff, 193-5-manual-repair.diff, 193-6-inverted-filter.diff, 193-6-inverted-filter.diff, 193-6-repair-explanation-and-node-rename.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-193:
-------------------------------------

    Fix Version/s:     (was: 0.4)
                   0.5

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722054#action_12722054 ] 

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

The current plan is for an AntiEntropyService per table to maintain a Merkle Tree per column family.

The tree will be a implemented as a randomized binary tree in memory (a ''Treap'': http://en.wikipedia.org/wiki/Treap), where every item in the tree represents a range bounded by the dht.Tokens of its left and right neighbors. By placing a bound on the total number of nodes in the tree, we can compact or split ranges in the tree by removing or adding Tokens. The algorithm for deciding which ranges to compact/split will be described below.

When a write comes in for a given table, we will place 'invalidation' operations in a queue for all affected column families. The ExecutorService for the table will read from the queue and perform the 'invalidations' as fast as it can. For a given Key/Token, if any column family tree is marked as 'invalid', the entire row needs to be read from disk and repaired (at some point in the future).

An 'invalidation' operation does a binary search in the Merkle Tree and marks the matching range as 'invalid', deleting its hash. We will also take advantage of this step to optimize the tree: A ''Treap'' stores a random priority (P) on each node, and by generating a random P' and replacing P for a node iff P' < P as we invalidate it, more frequently invalidated ranges will shift to the top of the tree.

The AEService maintaining the tree for a table will occasionally need to exchange portions of the tree with other nodes. In order to do this, subtrees that both nodes are interested in from all CF trees will have to be locked long enough to recalculate all 'invalid' children, and then the locks can flow down the tree as progressively smaller ranges are exhanged. Doing this locking efficiently is going to be interesting (aka: I haven't thought about it).

Implementing the exchange between nodes is blocked by CASSANDRA-242 because in order to align the subtrees on different nodes, we need to be able to deterministically split two ranges.

In order to fill in 'invalid' ranges in the tree, the MerkleTree will provide an operation that builds a list of invalid ranges to be fetched from disk. During this step, we can also compact/split ranges. Because of our Treap maintenance, frequently invalidated ranges will be near the top of the tree, and stable ranges will be closer to the leaves. By compacting the deepest N nodes and expanding the shallowest N, we can minimizing the size of the ranges that are affected by invalidations in the future.

Given the list of 'invalid' ranges (and pointers directly to the tree nodes), the AEService will fetch the ranges from the current MemTable and SSTables for the CF, hash them, and store the hashes into the relevant nodes. After this operation, we can recursively calculate hashes for inner nodes.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780443#action_12780443 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

It would be nice if the new code followed the style guide: http://wiki.apache.org/cassandra/CodeStyle

> 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-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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12786612#action_12786612 ] 

Hudson commented on CASSANDRA-193:
----------------------------------

Integrated in Cassandra #279 (See [http://hudson.zones.apache.org/hudson/job/Cassandra/279/])
    add manual repair through NodeProbe; patched by Stu Hood, reviewed by junrao for 
add AntiEntropyService; patched by Stu Hood, reviewed by junrao for 
preparation for AntiEntropyService; patched by Stu Hood, reviewed by junrao for 
add Merkle tree; patched by Stu Hood, reviewed by junrao for 
preparation for Merkle tree; patched by Stu Hood, reviewed by junrao for 


> 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-1-tree-preparation.diff, 193-2-tree.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-3-aes-preparation.diff, 193-4-aes.diff, 193-4-aes.diff, 193-4-aes.diff, 193-4-aes.diff, 193-5-manual-repair.diff, 193-5-manual-repair.diff, 193-5-manual-repair.diff, 193-6-inverted-filter.diff, 193-6-inverted-filter.diff, 193-6-repair-explanation-and-node-rename.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722860#action_12722860 ] 

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

So after a little bit of discussion, we decided to drop the Treap idea, and go with a B+Tree instead. The Treap would have been slightly faster for invalidations because of the priority-optimization, but it would be much less memory efficient because of excess inner nodes. Also, the invalidation count can just as easily be represented as a counter on each leaf node as it could have been by the height of the leaf in the Treap.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12777171#action_12777171 ] 

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

> Another complexity arises because you assume that the ranges in 2 different Merkle trees are different.
This is another decision that arose from the idea of maintaining the tree over a longer period by invalidating ranges, and only compacting the ranges that had changed recently. The replicas would end up with divergent splits in the trees. If we go back to assuming that we never want to maintain a tree between compactions, then compact() and invalidate() could be removed, and differences() could be simplified.

But considering the fact that we are hesitant to trigger a major compaction for every repair, maintaining the tree between repairs becomes a more interesting option.

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12785095#action_12785095 ] 

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

10. I was referring to the following line, it basically returns all keys for a CF on a node.
            List<DecoratedKey> keys = SSTableReader.getIndexedDecoratedKeysFor(cfpred, DKPRED);


> 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-6-repair-explanation-and-node-rename.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782539#action_12782539 ] 

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

Some comments:
10. AES.validator.prepare() requires buffering all keys in a node in memory. Can you push sampling into the key iterator?
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.
12. Could you add some comments in MerkleTree that describe the flow of TableRepair?
13. Some suggestions for notation changes : Node --> INode, Hash -- > LNode.


> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782563#action_12782563 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

We use IFoo as a convention for denoting interfaces, so naming a class INode doesn't seem like a good idea to me.

> 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.


[jira] Issue Comment Edited: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722054#action_12722054 ] 

Stu Hood edited comment on CASSANDRA-193 at 6/19/09 3:10 PM:
-------------------------------------------------------------

The current plan is for an AntiEntropyService per table to maintain a Merkle Tree per column family.

The tree will be a implemented as a randomized binary tree in memory (a ''Treap'': http://en.wikipedia.org/wiki/Treap), where every item in the tree represents a range bounded by the dht.Tokens of its left and right neighbors. By placing a bound on the total number of nodes in the tree, we can limit the memory usage. We can compact or split ranges in the tree by removing or adding Tokens. The algorithm for deciding which ranges to compact/split will be described below.

When a write comes in for a given table, we will place 'invalidation' operations in a queue for all affected column families. The ExecutorService for the table will read from the queue and perform the 'invalidations' as fast as it can. For a given Key/Token, if any column family tree is marked as 'invalid', the entire row needs to be read from disk and repaired (at some point in the future).

An 'invalidation' operation does a binary search in the Merkle Tree and marks the matching range as 'invalid', deleting its hash. We will also take advantage of this step to optimize the tree: A ''Treap'' stores a random priority (P) on each node, and by generating a random P' and replacing P for a node iff P' < P as we invalidate it, more frequently invalidated ranges will shift to the top of the tree.

The AEService maintaining the tree for a table will occasionally need to exchange portions of the tree with other nodes. In order to do this, subtrees that both nodes are interested in from all CF trees will have to be locked long enough to recalculate all 'invalid' children, and then the locks can flow down the tree as progressively smaller ranges are exhanged. Doing this locking efficiently is going to be interesting (aka: I haven't thought about it).

Implementing the exchange between nodes is blocked by CASSANDRA-242 because in order to align the subtrees on different nodes, we need to be able to deterministically split two ranges.

In order to fill in 'invalid' ranges in the tree, the MerkleTree will provide an operation that builds a list of invalid ranges to be fetched from disk. During this step, we can also compact/split ranges. Because of our Treap maintenance, frequently invalidated ranges will be near the top of the tree, and stable ranges will be closer to the leaves. By compacting the deepest N nodes and expanding the shallowest N, we can minimizing the size of the ranges that are affected by invalidations in the future.

Given the list of 'invalid' ranges (and pointers directly to the tree nodes), the AEService will fetch the ranges from the current MemTable and SSTables for the CF, hash them, and store the hashes into the relevant nodes. After this operation, we can recursively calculate hashes for inner nodes.

      was (Author: stuhood):
    The current plan is for an AntiEntropyService per table to maintain a Merkle Tree per column family.

The tree will be a implemented as a randomized binary tree in memory (a ''Treap'': http://en.wikipedia.org/wiki/Treap), where every item in the tree represents a range bounded by the dht.Tokens of its left and right neighbors. By placing a bound on the total number of nodes in the tree, we can compact or split ranges in the tree by removing or adding Tokens. The algorithm for deciding which ranges to compact/split will be described below.

When a write comes in for a given table, we will place 'invalidation' operations in a queue for all affected column families. The ExecutorService for the table will read from the queue and perform the 'invalidations' as fast as it can. For a given Key/Token, if any column family tree is marked as 'invalid', the entire row needs to be read from disk and repaired (at some point in the future).

An 'invalidation' operation does a binary search in the Merkle Tree and marks the matching range as 'invalid', deleting its hash. We will also take advantage of this step to optimize the tree: A ''Treap'' stores a random priority (P) on each node, and by generating a random P' and replacing P for a node iff P' < P as we invalidate it, more frequently invalidated ranges will shift to the top of the tree.

The AEService maintaining the tree for a table will occasionally need to exchange portions of the tree with other nodes. In order to do this, subtrees that both nodes are interested in from all CF trees will have to be locked long enough to recalculate all 'invalid' children, and then the locks can flow down the tree as progressively smaller ranges are exhanged. Doing this locking efficiently is going to be interesting (aka: I haven't thought about it).

Implementing the exchange between nodes is blocked by CASSANDRA-242 because in order to align the subtrees on different nodes, we need to be able to deterministically split two ranges.

In order to fill in 'invalid' ranges in the tree, the MerkleTree will provide an operation that builds a list of invalid ranges to be fetched from disk. During this step, we can also compact/split ranges. Because of our Treap maintenance, frequently invalidated ranges will be near the top of the tree, and stable ranges will be closer to the leaves. By compacting the deepest N nodes and expanding the shallowest N, we can minimizing the size of the ranges that are affected by invalidations in the future.

Given the list of 'invalid' ranges (and pointers directly to the tree nodes), the AEService will fetch the ranges from the current MemTable and SSTables for the CF, hash them, and store the hashes into the relevant nodes. After this operation, we can recursively calculate hashes for inner nodes.
  
> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12759994#action_12759994 ] 

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

More thoughts (in no particular order, and probably disagreeing with previous decisions):
 1. Since Memtables are always flushed after a bounded period (I think?) it is possible to completely ignore them.
 2. Due to 1, invalidating a range for every Memtable change is dumb/overkill. It would be much more efficient to only invalidate during minor compactions.
 3. Again, due to 1, we only care about the data affected during a major compaction, so we can validate the tree during every major compaction.

If we hook tree validation into major compactions:
 * Handling TreeRequests/TreeResponses would be queued until the next major compaction,
 * The compactor would acquire a lock on the tree for the CF, validate it, and then handle outstanding TreeRequests/TreeResponses.

> 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: CASSANDRA-193.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12719386#action_12719386 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

IRC: i am considering storing the hashes for an SSTable in another SSTable, to prevent them from being recalculated

If it were useful to have the per-sstable hashes precalculated, we could keep them in the SSTable itself (with the appropriate format change).  The problem is, that we need to hash the entire row, which means merging columns from multiple sstables, which means no per-sstable hash will be useful.

Unless I am missing something, in which case, hurrah! :)

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.4
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722313#action_12722313 ] 

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

Shoot. Just noticed a fundamental problem with using a Treap: going to need to think about the problem a bit more.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Comment: was deleted

(was: Shoot. Just noticed a fundamental problem with using a Treap: going to need to think about the problem a bit more.)

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12717914#action_12717914 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

To start with the good news: one thing which may seem on the face of it to be a problem, isn't really.  That is, how do you get nodes replicating a given token range to agree where to freeze or snapshot the data set to be repaired, in the face of continuing updates?  The answer is, you don't; it doesn't matter.  If we repair a few columnfamilies that don't really need it (because one of the nodes was just a bit slower to process an update than the other), that's no big deal.  We accept that and move on.

The bad news is, I don't see a clever solution for performing broad-based repair against the Memtable/SSTable model similar to Merkle trees for Dynamo/bdb.  (Of course, that is no guarantee that none such exists. :)

There are several difficulties.  (In passing, it's worth noting that Bigtable sidesteps these issues by writing both commit logs and sstables to GFS, which takes care of durability.  Here we have to do more work in exchange for a simpler model and better performance on ordinary reads and writes.)

One difficulty lies in how data in one SSTable may be pre-empted by another.  Because of this, any hash-based "summary" of a row may be obsoleted by rows in another.  For some workloads, particularly ones in which most keys are updated infrequently, caching such a summary in the sstable or index file might still be useful, but it should be kept in mind that in the worst case these will just be wasted effort.

(I think it would be a mistake to address this by forcing a major compaction -- combining all sstables for the columnfamily into one -- as a prerequisite to repair.  Reading and rewriting _all_ the data for _each_ repair is a significant amount of extra I/O.)

Another is that token regions do not correspond 1:1 to sstables, because each node is responsible for N token regions -- the regions for which is is the primary, secondar, tertiary, etc. repository for -- all intermingled in the SSTable files.  So any precomputation would need to be done separately N times.

Finally, we can't assume that sstable or even just row key names will fit into the heap, which limits the kind of in-memory structures we can build.

So from what I do not think it is worth the complexity to attempt to cache per-row hashes or summaries of the sstable data in the sstable or index files.

So the approach I propose is simply to iterate through the key space on a per-CF basis, compute a hash, and repair if there is a mismatch.  The code to iterate keys is already there (for the compaction code) and so is the code to compute hashes and repair if a mismatch is found (for read repair).  I think it will be worth flushing the current memtable first to avoid having to take a read lock on it.

Enhancements could include building a merkle tree from each batch of hashes to minimize round trips -- although unfortunately I think that is not going to be a bottleneck for Cassandra compared to the hash computation -- and fixing the compaction and hash computation code to iterate through columns in a CF rather than deserializing each ColumnFamily in its entirety.  These could definitely be split into separate tickets.


> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.4
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12777137#action_12777137 ] 

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

> a. Why do you want to use an n-ary tree to represent a binary tree?
The main reason was due to memory concerns... for 256 subranges, an order 256 B-Tree only needs 1 inner node (total 257), while a binary tree would need 127 inner nodes (total 383). Also, in order to get reasonable performance out of a binary tree over its lifetime, you would probably want to implement a self balancing tree, which isn't much simpler than a B-Tree. Finally, B-Trees are arguably faster than binary tree implementations: http://idlebox.net/2007/stx-btree/stx-btree-0.8.3/doxygen-html/speedtest.html

> b. Suppose + is the bit-wise AND [...] you can compute the hash of a range directly (since now + is commutative and transitive),
> without a bottom-up traversal from the leaves of the complete binary tree.
Ack, this was more of me glossing over details: we don't use AND or any other commutative operations in the tree, since it would be more likely to cause hash collisions (switched subtrees might compare equally).

All hashing is accomplished by a single method: Hashable.hash(byte[], byte[]), which is currently implemented using MD5.

> 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.


[jira] Issue Comment Edited: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781641#action_12781641 ] 

Stu Hood edited comment on CASSANDRA-193 at 11/23/09 9:46 PM:
--------------------------------------------------------------

> see whether we lose any precision by simply using bitwise XOR to combine row hashes
Doing the math on this one was simpler than actually testing it: using a commutative hash function like XOR means that the number of possible inputs goes from being a Permutation of the leaves to being a Combination of the leaves (since a set of leaves in any order are equal).

For MD5 you have:
(2^127)! / (2^127 - 2^16)! == number of possible permutations of 2^16 hashes of length 127
And for XOR:
(2^127)! / (2^16)! * (2^127 - 2^16)! == number of possible combinations of 2^16 hashes of length 127

I wouldn't think it would be possible to notice such a small difference. Nonetheless, I think it is a moot point:

> Using XOR further simplifies how hash values for internal nodes are computed.
I don't think that using XOR is significantly more efficient. Because XOR is associative, it is possible to hash arbitrary sequential leaves together, which is impossible with MD5, but we never do this: all of our comparison happens on the boundaries defined by IPartitioner.midpoint(), so the tree structure containing predefined/precomputed values can contain any value required for comparison.

      was (Author: stuhood):
    > see whether we lose any precision by simply using bitwise XOR to combine row hashes
Doing the math on this one was simpler than actually testing it: using a commutative hash function like XOR means that the number of possible inputs goes from being a Permutation of the leaves to being a Combination of the leaves (since a set of leaves in any order are equal).

For XOR you have:
(2^127)! / (2^127 - 2^16)! == number of possible permutations of 2^16 hashes of length 127
And for MD5:
(2^127)! / (2^16)! * (2^127 - 2^16)! == number of possible combinations of 2^16 hashes of length 127

I wouldn't think it would be possible to notice such a small difference. Nonetheless, I think it is a moot point:

> Using XOR further simplifies how hash values for internal nodes are computed.
I don't think that using XOR is significantly more efficient. Because XOR is associative, it is possible to hash arbitrary sequential leaves together, which is impossible with MD5, but we never do this: all of our comparison happens on the boundaries defined by IPartitioner.midpoint(), so the tree structure containing predefined/precomputed values can contain any value required for comparison.
  
> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-breakdown.txt
                193-5-manual-repair.diff
                193-4-aes.diff

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781484#action_12781484 ] 

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

While you are working on refactoring, do you think that you can do some anecdotal tests to see whether we lose any precision by simply using bitwise XOR to combine row hashes? Using XOR further simplifies how hash values for internal nodes are computed.

> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-4-aes.diff

This one should finally be ready for review! The patchset:
 1. Prepares to add the MerkleTree datastructure,
 2. Adds the datastructure,
 3. Prepares to add AntiEntropyService,
 4. Adds the service.

There should only be one FIXME in the patch, which refers to CASSANDRA-520.

> 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, CASSANDRA-193.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12723696#action_12723696 ] 

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

> Then the question becomes, is it worth trying to keep (partial) results of that scan in memory to avoid re-doing the work next time around. If writes are randomly distributed across the range then ISTM the answer is a clear No, but I'm not sure how close real-world workloads would come to that. 
You're right that the tree is basically a 'range hash cache', but I don't think that writes will be randomly distributed. Especially since we allow complex values, I think people are more likely to have 'hot' keys. Adding in the OrderPreservingPartitioner makes it even more likely to have hot ranges.

> Does it make sense to start with a non-caching version like I describe?
Perhaps: we could initialize a new MerkleTree at repair time, use the range hashing API I've described, and throw it away at the end of the repair. Next, we could implement maintaining/invalidating the tree between repairs. I'm not sure how much simpler this is (since the invalidation of ranges is probably the simplest part of the whole deal).

> Incidently, I don't think we should worry about the kind of locking you mentioned.
You're right: since a separate thread/agent/executor is maintaining the tree, the locking should be completely unnecessary. Whenever we're performing a repair, we're not accepting invalidations, so we're looking at a snapshot of the tree.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Assigned: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood reassigned CASSANDRA-193:
----------------------------------

    Assignee: Stu Hood

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12785117#action_12785117 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

that's just the "indexed" keys (the 1/128 of keys we keep in memory to bsearch on), so I would think that would be fine

> 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-6-repair-explanation-and-node-rename.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12776600#action_12776600 ] 

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

Thanks a lot for the review Jun! I'll respond to #1 in a separate comment, since it is a design issue that still needs a lot of discussion.

> 2
> 2.1 In MerkleTree.Node.insert, why do you increment the depth of the left child even when the node doesn't split?
Node.insert() is only used during split operations (perhaps it is misnamed... but this is not quite a traditional B-Tree). The child to the left is the node that contains the Token we were splitting on, and whenever we split a range we increment its depth to indicate how far it is from being a complete (0,0] range. I'll add a comment to this effect.

> 2.2 In the same function, if the node does split, where is the code to shrink the children list in the splitted node to half?
Node.insert() uses List.subList() and List.clear() inline to take half of its neighbor's children.

> 2.3 In the same function, do you have to keep calling invalidate during insertion?
The original design assumed that the tree was going to live for a while in memory, and be maintained between repair sessions, so the split operation is intended to be used on a tree that might be partially valid. We might be able to have the initial building of the tree skip this check somehow, but I don't think the 'hash == null? hash = null;' check is too intensive.

> 3
> 3.1 In Validator.add, there is comment about generating a new range. However, no code does that.
The "private MerkleTree.TreeRangeIterator ranges" variable is an iterator generated by the MerkleTree: it iterates in order over all of the ranges in the tree that have null hashes.

> 3.2 In TreeRange.validateHelper [...] Why do you have to compute multiple hash values recursively?
The reasoning here is that a MerkleTree is supposed to be a sparse representation of a 'perfect/complete' binary tree. Each leaf of the perfect tree represents a hashed range, and each inner node represents a binary hash of its two children. The perfect tree is of depth "hashdepth", so when validateHelper() reaches the maximum/hashdepth, it is in one of the leaves of the perfect tree, and rows are hashed sequentially there. If a single call to validate() starts in a TreeRange that is at depth == maxdepth, then what is stored in the MkT.Node is the value of a perfect leaf, otherwise, the MkT.Node is storing the value of a perfect inner node.

I'll probably copy and past this exact explanation into the next revision =x

> 4. I need some text description to really follow the Differencer code.
I'll make sure this gets in the next revision, but basically:
 1. It recurses using midpoint as long as both trees have the resolution to continue, and are not equal.
 2a. If it finds a range that has only one invalid child, it adds that child range, since that is the smallest possible invalid range contained in the parent.
 2b. Otherwise, if both children are invalid (and since it can't recurse deeper), the parent range is entirely invalid, and recursion keeps rolling up until 2a is met.

> 5. The Hashable class is confusing.
You're right: I definitely should have worried more about overloading the word "hash": I'll see what I can do about this one.

> 6. The repair logic is missing in Differencer.
The repair logic is the one FIXME for this ticket: CASSANDRA-520 deals with implementing the actual repair logic.

Thanks again for taking time out to look at this! 

> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-6-inverted-filter.diff

New version of patch 6 to resolve the Cachetable/rendezvous issues I noticed last night.

I don't see any more bugs, but I did some testing with a table containing 10^6 keys, and a Readonly compaction took 2 minutes (!). Hopefully a little bit of profiling will expose the issue quickly, because I can understand that performance like that should prevent merging this patch. I'll try and get a new version out this evening.

> 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-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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12723154#action_12723154 ] 

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

Ah, I see what you mean regarding the random reads. I've mostly been thinking about finding ranges to repair, rather than the actual repair steps.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780899#action_12780899 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

"the inner nodes need to use an associative hash function if their values are going to be cached"

huh?

> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-6-repair-explanation-and-node-rename.diff

Adding 193-6-repair-explanation[...] to address Jun's latest comments: no other changes.

> 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-6-repair-explanation-and-node-rename.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722058#action_12722058 ] 

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

Definitely... since the tree maintenance is implemented as a separate service, we can implement maintaining the tree first, and then think about how to exchange portions later.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12777115#action_12777115 ] 

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

Now I start to understand the code a bit better. A couple of other questions:
a. Why do you want to use an n-ary tree to represent a binary tree? Why can't you just implement Merkle tree as an in-complete binary tree itself? This way, there is less confusing about whether a node refers to one in the n-ary tree or the binary tree.
b. Suppose + is the bit-wise AND btw 2 hash values and you compute the hash of a range as the sum of the hash of each row in the range. Then, it seems that you can compute the hash of a range directly (since now + is commutative and transitive), without a bottom-up traversal from the leaves of the complete binary tree. Wouldn't this be simpler?

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12775448#action_12775448 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

it's major compaction, not just "compaction," right?  major compactions are never done automatically, so if we assume they are done by cron or dsh or some other similarly "reasonably synchronous" method it should be fine.  right?  (although it should flush first, dunno if it does.)

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12723151#action_12723151 ] 

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

> The more I think about this the less convinced I am that the partially-invalidated live tree is going to be worth the overhead of maintaining it (and initializing it on startup). 
There is no need to initialize the tree on startup: it can be done lazily when the first tree 
exchange requests come in.

> If you instead just create a mini-merkle tree from the first N keys and exchange that with the replica nodes, then repeat for the next N, you still get a big win on network traffic (which is the main concern here)...
Yes, network traffic is important, but the whole point of maintaining the tree in memory is that it prevents us from having to read entire SSTables from disk in order to do repairs (similar to BloomFilters for random lookups). Any portions of the tree that survive (which should be large portions, assuming we do invalidations correctly) mean that we can use the SSTable index to seek() past chunks of the file.

> but you have no startup overhead, no complicated extra maintenance to perform on insert, better performance in the worst case and (probably) in the average case, since you are avoiding random reads in favor of (a potentially greater number of) streaming reads...
 * No startup overhead necessary,
 * B+Tree invalidations will only involve marking a leaf node invalid: aka, do a lookup and increment a counter,
 * There won't be any random reads... I'm not sure where you read that: in order to validate regions of the tree we will be iterating over the keys in the CF in sorted order, skipping regions that are valid.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12723572#action_12723572 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

If there is no tree-building phase on startup, what is the behavior after a fresh start if a Repair phase begins (either automatically or from human intervention)?

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-3-aes-preparation.diff
                193-2-tree.diff
                193-1-tree-preparation.diff

Alright, here is a rebased version of the patchset containing only 5 patches again, which match up with the files changed in the earlier patchsets.

Since the comment I made on Nov 21st, I refactored the MerkleTree structure into a binary tree, which dropped 750 lines from the patch, and made things much much clearer in general. Additionally, the MerkleTree now almost exclusively uses methods from dht.Range when searching for ranges, which removed a whole bunch special casing for the minimum token. No other files changed by more than 5-10 lines, and in general, I feel very happy to have admitted defeat on the b-tree idea.

I've subjected this to system tests with all partitioners, and it passed with flying colors. 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-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.


[jira] Issue Comment Edited: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12759990#action_12759990 ] 

Stu Hood edited comment on CASSANDRA-193 at 9/26/09 11:00 PM:
--------------------------------------------------------------

I've been working on this ticket a bit more in the past few days:
 * Added o.a.c.service.AntiEntropyService - Maintains trees for each CF, and accepts invalidations when values change.

Still TODO:
 * Implement TreeRequestVerbHandler/TreeResponseVerbHandler - The AEService on a first endpoint will periodically wake up and send a TreeRequest to a replica. The replica endpoint will handle the TreeRequest by validating one or all of its MerkleTrees, and responding with a TreeResponse. Handling the TreeResponse on the first endpoint will involve validating the local tree, and then comparing the two trees.
   * Validation is the only part that is fuzzy here: we need to iterate over keys in each CF (essentially, a major compaction, except that we can skip processing for anything that is still valid in the tree).
 * Begin implementing the actual repair step - There isn't a design for this part yet: any thoughts would be appreciated. The output of the TreeRequest/TreeResponse conversation will be a list of ranges in a given CF that disagree between the two endpoints.

EDIT: The code is still located at: http://github.com/stuhood/cassandra-anti-entropy/

      was (Author: stuhood):
    I've been working on this ticket a bit more in the past few days:
 * Added o.a.c.service.AntiEntropyService - Maintains trees for each CF, and accepts invalidations when values change.

Still TODO:
 * Implement TreeRequestVerbHandler/TreeResponseVerbHandler - The AEService on a first endpoint will periodically wake up and send a TreeRequest to a replica. The replica endpoint will handle the TreeRequest by validating one or all of its MerkleTrees, and responding with a TreeResponse. Handling the TreeResponse on the first endpoint will involve validating the local tree, and then comparing the two trees.
   * Validation is the only part that is fuzzy here: we need to iterate over keys in each CF (essentially, a major compaction, except that we can skip processing for anything that is still valid in the tree).
 * Begin implementing the actual repair step - There isn't a design for this part yet: any thoughts would be appreciated. The output of the TreeRequest/TreeResponse conversation will be a list of ranges in a given CF that disagree between the two endpoints.
  
> 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: CASSANDRA-193.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12776519#action_12776519 ] 

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

First, thanks Stu for this big patch. This is a lot of work. Here are some review comments.
1. The high level question. When should the Merkle tree be computed? The patch piggybacks the computation on a regular compaction. Even if it's moved to major compaction, it's still not enough. This is because there is an upper limit on file size. Therefore, not all sstables are necessarily read during a major compaction, which means the Merkle tree may not see all keys in a particular key range.

One approach is to explicitly iterate through keys on all sstables in a particular range, compute the Merkle tree, send the Merkle tree to replicas. Each replica then computes its own Merkle tree and do the comparison. We can trigger this process through a nodeprobe command.

I spent quite some time reading through the code and I am still confused in several places. Perhaps some more text description on each main method (e.g, split, validate, difference) will help.

2. It's not clear to me exactly how splitting in Merkle tree works.
2.1 In MerkleTree.Node.insert, why do you increment the depth of the left child even when the node doesn't split?
2.2 In the same function, if the node does split, where is the code to shrink the children list in the splitted node to half?
2.3 In the same function, do you have to keep calling invalidate during insertion? It seems to me that it would be simpler if you first split the tree to what you want, then make a pass of the tree to invalidate all nodes before computing the hashes.

3. I am not exactly clear on how the validator works.
3.1 In Validator.add, there is comment about generating a new range. However, no code does that.
3.2 In TreeRange.validateHelper, you are trying to compute the hash for a set of rows in a range. Why do you have to compute multiple hash values recursively?

4. I need some text description to really follow the Differencer code.

5. The Hashable class is confusing. By its name, I expect it to be really about just the hash. However, the comparator is actually on token. HashableToken is probably a better name.

6. The repair logic is missing in Differencer.


> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722055#action_12722055 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

It sounds like I could wrap my head around this if I thought hard about it for a few days like you have. :)

Is there any way this can be split up into independent changes that are easily understood and reviewed?


> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722050#action_12722050 ] 

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

> IRC: i am considering storing the hashes for an SSTable in another SSTable, to prevent them from being recalculated 
I've reconsidered this portion: this would waste a ton of CPU time hashing something that we may never need.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12759990#action_12759990 ] 

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

I've been working on this ticket a bit more in the past few days:
 * Added o.a.c.service.AntiEntropyService - Maintains trees for each CF, and accepts invalidations when values change.

Still TODO:
 * Implement TreeRequestVerbHandler/TreeResponseVerbHandler - The AEService on a first endpoint will periodically wake up and send a TreeRequest to a replica. The replica endpoint will handle the TreeRequest by validating one or all of its MerkleTrees, and responding with a TreeResponse. Handling the TreeResponse on the first endpoint will involve validating the local tree, and then comparing the two trees.
   * Validation is the only part that is fuzzy here: we need to iterate over keys in each CF (essentially, a major compaction, except that we can skip processing for anything that is still valid in the tree).
 * Begin implementing the actual repair step - There isn't a design for this part yet: any thoughts would be appreciated. The output of the TreeRequest/TreeResponse conversation will be a list of ranges in a given CF that disagree between the two endpoints.

> 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: CASSANDRA-193.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12776780#action_12776780 ] 

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

2.1 Still confused. So node.depth doesn't mean the depth of the node in the tree?

3.1 The ranges don't change after the iterator is generated, right? But inside the while loop in Validator.add, there is comment about adding a new range. So, are you really adding a new range there or the comment is wrong?

3.2 Why is Merkle tree a binary tree? I thought it's an n-ary tree since you can specify the order of each node.

Since a leaf node corresponds to a single range, why do you need a list of hashes? I thought each leaf node maintains a single hash value for rows its range and each inner node maintains a single hash that is the logical AND of the hashes of all its children. Is this not correct?


> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12775105#action_12775105 ] 

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

Started reviewing this patch. Here is a high-level question. From the code, the process roughly looks as the follows:
1. Each node N periodically computes a Merkle tree for rows in each new SSTable generated through compaction.
2. The Merkle tree is sent and registered to other nodes that share key ranges with N.
3. The locally computed Merkle tree will be compared with those registered remote Merkle trees. If there is any difference, trigger a repair.

Since compaction is triggered independently at each node, the rows in two compacted SSTables generated in two neighboring nodes are unlikely to match. Won't the above approach trigger too many unnecessary repairs?


> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12733076#action_12733076 ] 

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

I'm making steady progress on this ticket. So far, the work has all been on implementing the merkle tree data structure, but I'd like to start thinking about integrating the structure soon.

I've been working in the (misnamed) branch 'binary-merkle' on http://github.com/stuhood/cassandra-anti-entropy/. I may submit a patch here containing just the data structure sometime soon, but it still has a wart that I'm not happy about.

> 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
>
>
> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-5-manual-repair.diff
                193-4-aes.diff

Rebased: back down to only 5 patches.

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-1-tree-preparation.diff, 193-1-tree-preparation.diff, 193-2-tree.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-3-aes-preparation.diff, 193-4-aes.diff, 193-4-aes.diff, 193-4-aes.diff, 193-4-aes.diff, 193-5-manual-repair.diff, 193-5-manual-repair.diff, 193-5-manual-repair.diff, 193-6-inverted-filter.diff, 193-6-inverted-filter.diff, 193-6-repair-explanation-and-node-rename.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12723639#action_12723639 ] 

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

> ...what is the behavior after a fresh start if a Repair phase begins?
Well, you would have a single large invalid range (or some number of smaller ranges summing to the full range), which could be fetched sequentially from disk.

Rather than being completely lazy, (depending on how much extra load it would cause) we could hook the AEService into compactions that are happening for other reasons, so that before the compaction begins, the compactor fetches the current list of invalid ranges and fills them in based on the merged data. I'm not sure how much of a win this would be, since we probably don't want to slow down compactions, but if they aren't CPU bound, then it shouldn't hurt.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-3-aes-preparation.diff
                193-2-tree.diff
                193-1-tree-preparation.diff

> 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, CASSANDRA-193.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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722062#action_12722062 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

That's true.  That's a logical division.

I'm still a little scared by the performance implications of basically doing mini read-only compactions to keep the tree accurate.  I'm going to mark this as requiring CASSANDRA-234.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12723089#action_12723089 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

The more I think about this the less convinced I am that the partially-invalidated live tree is going to be worth the overhead of maintaining it (and initializing it on startup).

If you instead just create a mini-merkle tree from the first N keys and exchange that with the replica nodes, then repeat for the next N, you still get a big win on network traffic (which is the main concern here) but you have no startup overhead, no complicated extra maintenance to perform on insert, better performance in the worst case and (probably) in the average case, since you are avoiding random reads in favor of (a potentially greater number of) streaming reads which assuming a constant workload profile (i.e. the same proportion of keys being overwritten) is always going to be a win for the streaming case.

Implementation detail: you'd want to add an internal message [merkle startkey] where startkey is initially "" and after each iteration you update it to the N'th key _after_ merging any missing ones.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Issue Comment Edited: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780912#action_12780912 ] 

Stu Hood edited comment on CASSANDRA-193 at 11/22/09 3:07 AM:
--------------------------------------------------------------

EDIT: After a little more consideration, the caching bug had nothing to do with our hash function. It was a disconnect between the binary tree, and the b-tree we are using to store it. To be honest, I don't want to merge something so complex that even the person who created it still has trouble reasoning about it.

I'm going to refactor the b-tree into a binary tree tomorrow.

      was (Author: stuhood):
    The comment explains it better, but you need something like XOR, which is associative: (1 ^ 2) ^ 3 == 1 ^ (2 ^ 3) in order to cache partially computed values. MD5 needs to be computed sequentially over all of the inputs, so it can be used for leaves, but not inner nodes.
  
> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781641#action_12781641 ] 

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

> see whether we lose any precision by simply using bitwise XOR to combine row hashes
Doing the math on this one was simpler than actually testing it: using a commutative hash function like XOR means that the number of possible inputs goes from being a Permutation of the leaves to being a Combination of the leaves (since a set of leaves in any order are equal).

For XOR you have:
(2^127)! / (2^127 - 2^16)! == number of possible permutations of 2^16 hashes of length 127
And for MD5:
(2^127)! / (2^16)! * (2^127 - 2^16)! == number of possible combinations of 2^16 hashes of length 127

I wouldn't think it would be possible to notice such a small difference. Nonetheless, I think it is a moot point:

> Using XOR further simplifies how hash values for internal nodes are computed.
I don't think that using XOR is significantly more efficient. Because XOR is associative, it is possible to hash arbitrary sequential leaves together, which is impossible with MD5, but we never do this: all of our comparison happens on the boundaries defined by IPartitioner.midpoint(), so the tree structure containing predefined/precomputed values can contain any value required for comparison.

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12777156#action_12777156 ] 

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

Another complexity arises because you assume that the ranges in 2 different Merkle trees are different. Why can't you first build the key ranges from a sample of the keys in one node and then reuse those key ranges to build a Merkle on every replica? That would simplify both the computation and the diff of the hash values.

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722300#action_12722300 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

Note: it's difficult to tell what's changed when you edit a large comment.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>            Assignee: Stu Hood
>             Fix For: 0.5
>
>
> 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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-3-aes-preparation.diff
                193-2-tree.diff
                193-1-tree-preparation.diff

> 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-1-tree-preparation.diff, 193-2-tree.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-3-aes-preparation.diff, 193-4-aes.diff, 193-4-aes.diff, 193-4-aes.diff, 193-4-aes.diff, 193-5-manual-repair.diff, 193-5-manual-repair.diff, 193-5-manual-repair.diff, 193-6-inverted-filter.diff, 193-6-inverted-filter.diff, 193-6-repair-explanation-and-node-rename.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.


[jira] Updated: (CASSANDRA-193) Proactive repair

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stu Hood updated CASSANDRA-193:
-------------------------------

    Attachment: 193-3-aes-preparation.diff
                193-2-tree.diff
                193-1-tree-preparation.diff

> 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.


[jira] Commented: (CASSANDRA-193) Proactive repair

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12719376#action_12719376 ] 

Jonathan Ellis commented on CASSANDRA-193:
------------------------------------------

to be specific, for key iteration, start with ColumnFamilyStore.getKeyRange.

for the hashing, look at StorageProxy.strongRead -- first it sends a "digest" message to the non-primary nodes; this computes a hash, and if there is a mismatch then it does a repair step in the catch block.

> Proactive repair
> ----------------
>
>                 Key: CASSANDRA-193
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-193
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jonathan Ellis
>             Fix For: 0.4
>
>
> 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.