You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Nick Bailey (JIRA)" <ji...@apache.org> on 2011/09/02 19:11:16 UTC

[jira] [Commented] (CASSANDRA-2434) node bootstrapping can violate consistency

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

Nick Bailey commented on CASSANDRA-2434:
----------------------------------------

Just as initial feedback, I'm not sure we need a new getRangesWithSources method, especially with so much duplication between them. Seems like strict could be passed to the current method. Also, what about leaving getRangesWithSources how it is and passing strict to getWorkMap? That method can do the endpoint set math if it needs to and throw a more informative exception in the case that strict is set and the endpoint we want to fetch from is dead.

> node bootstrapping can violate consistency
> ------------------------------------------
>
>                 Key: CASSANDRA-2434
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2434
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Peter Schuller
>            Assignee: paul cannon
>             Fix For: 1.1
>
>         Attachments: 2434.patch.txt
>
>
> My reading (a while ago) of the code indicates that there is no logic involved during bootstrapping that avoids consistency level violations. If I recall correctly it just grabs neighbors that are currently up.
> There are at least two issues I have with this behavior:
> * If I have a cluster where I have applications relying on QUORUM with RF=3, and bootstrapping complete based on only one node, I have just violated the supposedly guaranteed consistency semantics of the cluster.
> * Nodes can flap up and down at any time, so even if a human takes care to look at which nodes are up and things about it carefully before bootstrapping, there's no guarantee.
> A complication is that not only does it depend on use-case where this is an issue (if all you ever do you do at CL.ONE, it's fine); even in a cluster which is otherwise used for QUORUM operations you may wish to accept less-than-quorum nodes during bootstrap in various emergency situations.
> A potential easy fix is to have bootstrap take an argument which is the number of hosts to bootstrap from, or to assume QUORUM if none is given.
> (A related concern is bootstrapping across data centers. You may *want* to bootstrap to a local node and then do a repair to avoid sending loads of data across DC:s while still achieving consistency. Or even if you don't care about the consistency issues, I don't think there is currently a way to bootstrap from local nodes only.)
> Thoughts?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira