incubator-cassandra-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jonathan Ellis <jbel...@gmail.com>
Subject Re: handling deletes
Date Wed, 01 Apr 2009 17:50:37 GMT
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.)

-Jonathan

Mime
View raw message