incubator-cassandra-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jun Rao <jun...@almaden.ibm.com>
Subject Re: handling deletes
Date Wed, 01 Apr 2009 18:46:08 GMT

My reply is inlined below.

Jun
IBM Almaden Research Center
K55/B1, 650 Harry Road, San Jose, CA  95120-6099

junrao@almaden.ibm.com


Jonathan Ellis <jbellis@gmail.com> wrote on 04/01/2009 10:50:37 AM:

>
> On Wed, Apr 1, 2009 at 11:41 AM, Jun Rao <junrao@almaden.ibm.com> wrote:
> > I am wondering if this is part of the bigger issue on data consistency.
> >
> > Following your example: a row x is replicated to node A, B, and C. C
goes
> > down. A and B delete x. When C comes back, C should contact other nodes
that
> > hold hinted handoff data intended for C. So, in theory, the missing
deletion
> > of x will be propagated to C at some point and not lost. However, the
> > problem is that those hinted handoff nodes can die before the handoff
> > completes. Then C need some other way to sync itself up. Node A and B
are
> > the only possible sources. Unfortunately, data in A and B are
accumulated
> > independently from C, and therefore syncing them up is a bit
challenging.
>
> Right.  Or you could have a network partition when C comes back up
> preventing the handoff.  There's lots of things that can go wrong.
> Hence the "eventual" part of "eventually consistent." :)
>
> > In the short run, I am not sure if I really like the solution you
suggested
> > here. However, I don't have a better solution either.
>
> Like I said; it's not perfect, but it's better than the alternatives
> I've seen.  I'd much rather have an imperfect solution than none at
> all.
>
> > In the long run, maybe we should look into peer-to-peer replication
> > techniques, instead of relying on hinted handoff. In P2P replication,
an
> > update can be directed to any replica, which will try to push it to its
> > peers. The push will be almost real time if the peers are up. If a peer
is
> > down, changes for it will be accumulated and re-pushed when it's up
again.
> > Because an update is always initiated from one replica, it's easier to
sync
> > up the replicas through log shipping.
>
> There's a huge amount of complexity you're glossing over, though: what
> if the replica responsible for the initiation goes down?  Then you
> have to elect a new one.  This is (a) very complicated and (b) causes
> loss of availability.  I prefer the existing system.  (If you want
> consistency over availability then hbase or hypertable is a better
> choice since that is what they design for.)

P2P replication definitely adds complexity and it is just one of the
alternatives. However, there is also complexity in hinted handoff + read
repair + merkle tree (when it's added). Not sure which one is more
complicated. In P2P replication, since you can initiate a write on any
replica, you just need to pick a live replica for writes. As for
availability, a lot have to do with how quickly a failed node is detected.
Today, if you write to a node that's actually failed, but not yet detected
by Cassandra, the write will also fail.

Overall, I think eventual consistency is fine. However, eventual
consistency probably shouldn't be equated to updates taking forever to show
up. Some sort of guarantee on how outdated a piece of data is will likely
be useful to many applications.

>
> -Jonathan
Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message