incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Edward Capriolo <edlinuxg...@gmail.com>
Subject Re: New Chain for : Does Cassandra use vector clocks
Date Thu, 24 Feb 2011 20:31:05 GMT
On Thu, Feb 24, 2011 at 3:03 PM, A J <s5alye@gmail.com> wrote:
> yes, that is difficult to digest and one has to be sure if the use
> case can afford it.
>
> Some other NOSQL databases deals with it differently (though I don't
> think any of them use atomic 2-phase commit). MongoDB for example will
> ask you to read from the node you wrote first (primary node) unless
> you are ok with eventual consistency. If the write did not make to
> majority of other nodes, it will be rolled-back from the original
> primary when it comes up again as a secondary.
> In some cases, you still could server either new value (that was
> returned as failed) or the old one. But it is different from Cassandra
> in the sense that Cassandra will never rollback.
>
>
>
> On Thu, Feb 24, 2011 at 2:47 PM, Anthony John <chirayithaj@gmail.com> wrote:
>> The leap of faith here is that an error does not mean a clean backing out to
>> prior state - as we are used to with databases. It means that the operation
>> in error could have gone through partially
>>
>> Again, this is not an absolutely unfamiliar territory and can be dealt with.
>> -JA
>> On Thu, Feb 24, 2011 at 1:16 PM, A J <s5alye@gmail.com> wrote:
>>>
>>> >>but could be broken in case of a failed write<<
>>> You can think of a scenario where R + W >N still leads to
>>> inconsistency even for successful writes. Say you keep W=1 and R=N .
>>> Lets say the one node where a write happened with success goes down
>>> before it made to the other N-1 nodes. Lets say it goes down for good
>>> and is unrecoverable. The only option is to build a new node from
>>> scratch from other active nodes. This will lead to a write that was
>>> lost and you will end up serving stale copy of it.
>>>
>>> It is better to talk in terms of use cases and if cassandra will be a
>>> fit for it. Otherwise unless you have W=R=N and fsync before each
>>> write commit, there will be scope for inconsistency.
>>>
>>>
>>> On Thu, Feb 24, 2011 at 1:25 PM, Anthony John <chirayithaj@gmail.com>
>>> wrote:
>>> > I see the point - apologies for putting everyone through this!
>>> > It was just militating against my mental model.
>>> > In summary, here is my take away - simple stuff but - IMO - important to
>>> > conclude this thread (I hope):-
>>> > 1. I was splitting hair over a failed ( partial ) Q Write. Such an event
>>> > should be immediately followed by the same write going to a connection
>>> > on to
>>> > another node ( potentially using connection caches of client
>>> > implementations
>>> > ) or a Read at CL of All. Because a write could have partially gone
>>> > through.
>>> > 2. Timestamps are used in determining the latest version ( correcting
>>> > the
>>> > false impression I was propagating)
>>> > Finally, wrt "W + R > N for Q CL statement" holds, but could be broken
>>> > in
>>> > case of a failed write as it is unsure whether the new value got written
>>> > on
>>> >  any server or not. Is that a fair characterization ?
>>> > Bottom line - unlike traditional DBMS, errors do not ensure automatic
>>> > cleanup and revert back, app code has to follow up if  immediate - and
>>> > not
>>> > eventual -  consistency is desired. I made that leap in almost all cases
>>> > - I
>>> > think - but the case of a failed write.
>>> > My bad and I can live with this!
>>> > Regards,
>>> > -JA
>>> >
>>> > On Thu, Feb 24, 2011 at 11:50 AM, Sylvain Lebresne
>>> > <sylvain@datastax.com>
>>> > wrote:
>>> >>
>>> >> On Thu, Feb 24, 2011 at 6:33 PM, Anthony John <chirayithaj@gmail.com>
>>> >> wrote:
>>> >>>
>>> >>> Completely understand!
>>> >>> All that I am quibbling over is whether a CL of quorum guarantees
>>> >>> consistency or not. That is what the documentation says - right.
IF
>>> >>> for a CL
>>> >>> of Q read - it depends on which node returns read first to determine
>>> >>> the
>>> >>> actual returned result or other more convoluted conditions , then
a
>>> >>> Quorum
>>> >>> read/write is not consistent, by any definition.
>>> >>
>>> >> But that's the point. The definition of consistency we are talking
>>> >> about
>>> >> has no meaning if you consider only a quorum read. The definition
>>> >> (which is
>>> >> the de facto definition of consistency in 'eventually consistent') make
>>> >> sense if we talk about a write followed by a read. And it is
>>> >> considering succeeding write followed by succeeding read.
>>> >> And that is the statement the wiki is making.
>>> >> Honestly, we could debate forever on the definition of consistency and
>>> >> whatnot. Cassandra guaranties that if you do a (succeeding) write on
W
>>> >> replica and then a (succeeding) read on R replica and if R+W>N, then
it
>>> >> is
>>> >> guaranteed that the read will see the preceding write. And this is what
>>> >> is
>>> >> called consistency in the context of eventual consistency (which is
not
>>> >> the
>>> >> context of ACID).
>>> >> If this is not the definition of consistency you had in mind then by
>>> >> all
>>> >> mean, Cassandra probably don't guarantee this definition. But given
>>> >> that the
>>> >> paragraph preceding what you pasted state clearly we are not talking
>>> >> about
>>> >> ACID consistency, but eventual consistency, I don't think the wiki is
>>> >> making
>>> >> any unfair statement.
>>> >> That being said, the wiki may not be always as clear as it could. But
>>> >> it's
>>> >> an editable wiki :)
>>> >> --
>>> >> Sylvain
>>> >>
>>> >>>
>>> >>> I can still use Cassandra, and will use it, luv it!!! But let us
not
>>> >>> make
>>> >>> this statement on the Wiki architecture section:-
>>> >>> -------------------------------------------------------------
>>> >>>
>>> >>> More specifically: R=read replica count W=write replica
>>> >>> count N=replication factor Q=QUORUM (Q = N / 2 + 1)
>>> >>>
>>> >>> If W + R > N, you will have consistency
>>> >>>
>>> >>> W=1, R=N
>>> >>> W=N, R=1
>>> >>> W=Q, R=Q where Q = N / 2 + 1
>>> >>>
>>> >>> Cassandra provides consistency when R + W > N (read replica count
>>> >>> + write
>>> >>> replica count > replication factor).
>>> >>>
>>> >>> ----------------------------------------------------
>>> >>>
>>> >>> .
>>> >>>
>>> >>> On Thu, Feb 24, 2011 at 11:22 AM, Sylvain Lebresne
>>> >>> <sylvain@datastax.com>
>>> >>> wrote:
>>> >>>>
>>> >>>> On Thu, Feb 24, 2011 at 6:01 PM, Anthony John <chirayithaj@gmail.com>
>>> >>>> wrote:
>>> >>>>>
>>> >>>>> If you are correct and you are probably closer to the code
- then CL
>>> >>>>> of
>>> >>>>> Quorum does not guarantee a consistency.
>>> >>>>
>>> >>>> If the operation succeed, it does (for some definition of consistency
>>> >>>> which is, following reads at Quorum will be guaranteed to see
the new
>>> >>>> value
>>> >>>> of a update at quorum). If it fails, then no, it does not guarantee
>>> >>>> consistency.
>>> >>>> It is important to note that the word consistency has multiple
>>> >>>> meaning.
>>> >>>> In particular, when we are talking of consistency in Cassandra,
we
>>> >>>> are not
>>> >>>> talking of the same definition as the C in ACID
>>> >>>>
>>> >>>> (see: http://www.allthingsdistributed.com/2007/12/eventually_consistent.html)
>>> >>>>>
>>> >>>>> On Thu, Feb 24, 2011 at 10:54 AM, Sylvain Lebresne
>>> >>>>> <sylvain@datastax.com> wrote:
>>> >>>>>>
>>> >>>>>> On Thu, Feb 24, 2011 at 5:34 PM, Anthony John
>>> >>>>>> <chirayithaj@gmail.com>
>>> >>>>>> wrote:
>>> >>>>>>>>
>>> >>>>>>>> >>Time stamps are not used for conflict
resolution - unless is is
>>> >>>>>>>> >> part of the application logic!!!
>>> >>>>>>>
>>> >>>>>>> >>What is you definition of conflict resolution
? Because if you
>>> >>>>>>> >> update twice the same column (which
>>> >>>>>>> >>I'll call a conflict), then the timestamps
are used to decide
>>> >>>>>>> >> which
>>> >>>>>>> >> update wins (which I'll call a resolution).
>>> >>>>>>> I understand what you are saying, and yes semantics
is very
>>> >>>>>>> important
>>> >>>>>>> here. And yes we are responding to the immediate
questions without
>>> >>>>>>> covering
>>> >>>>>>> all questions in the thread.
>>> >>>>>>> The point being made here is that the timestamp
of the column is
>>> >>>>>>> not
>>> >>>>>>> used by Cassandra to figure out what data to return.
>>> >>>>>>
>>> >>>>>> Not quite true.
>>> >>>>>>>
>>> >>>>>>> E.g. - Quorum is 2 nodes - and RF of 3 over N1/2/3
>>> >>>>>>> A Quorum  Write comes and add/updates the time
stamp (TS2) of a
>>> >>>>>>> particular data element. It succeeds on N1 - fails
on N2/3. So the
>>> >>>>>>> write is
>>> >>>>>>> returned as failed - right ?
>>> >>>>>>> Now Quorum read comes in for exactly the same piece
of data that
>>> >>>>>>> the
>>> >>>>>>> write failed for.
>>> >>>>>>> So N1 has TS2 but both N2/3 have the old TS (say
TS1)
>>> >>>>>>> And the read succeeds - Will it return TS1 or TS2.
>>> >>>>>>> I submit it will return TS1 - the old TS.
>>> >>>>>>
>>> >>>>>> It all depends on which (first 2) nodes respond to the
read (since
>>> >>>>>> RF=3, that can any two of N1/N2/N3). If N1 is part of
the two that
>>> >>>>>> makes the
>>> >>>>>> quorum, then TS2 will be returned, because cassandra
will compare
>>> >>>>>> the
>>> >>>>>> timestamp and decide what to return based on this. If
N2/N3
>>> >>>>>> responds
>>> >>>>>> however, both timestamp will be TS1 and so, after timestamp
>>> >>>>>> resolution, it
>>> >>>>>> will stil be TS1 that will be returned.
>>> >>>>>> So yes timestamp is used for conflict resolution.
>>> >>>>>> In your example, you could get TS1 back because a failed
write can
>>> >>>>>> let
>>> >>>>>> you cluster in an inconsistent state. You'd have to
retry the
>>> >>>>>> quorum and
>>> >>>>>> only when it succeeds can you be guaranteed that quorum
read will
>>> >>>>>> always
>>> >>>>>> return TS2.
>>> >>>>>> This is because when a write fails, Cassandra doesn't
guarantee
>>> >>>>>> that
>>> >>>>>> the write did not made it in (there is no revert).
>>> >>>>>>
>>> >>>>>>>
>>> >>>>>>> Are we on the same page with this interpretation
?
>>> >>>>>>> Regards,
>>> >>>>>>> -JA
>>> >>>>>>> On Thu, Feb 24, 2011 at 10:12 AM, Sylvain Lebresne
>>> >>>>>>> <sylvain@datastax.com> wrote:
>>> >>>>>>>>
>>> >>>>>>>> On Thu, Feb 24, 2011 at 4:52 PM, Anthony John
>>> >>>>>>>> <chirayithaj@gmail.com> wrote:
>>> >>>>>>>>>
>>> >>>>>>>>> Sylvan,
>>> >>>>>>>>> Time stamps are not used for conflict resolution
- unless is is
>>> >>>>>>>>> part of the application logic!!!
>>> >>>>>>>>
>>> >>>>>>>> What is you definition of conflict resolution
? Because if you
>>> >>>>>>>> update twice the same column (which
>>> >>>>>>>> I'll call a conflict), then the timestamps are
used to decide
>>> >>>>>>>> which
>>> >>>>>>>> update wins (which I'll call a resolution).
>>> >>>>>>>>
>>> >>>>>>>>>
>>> >>>>>>>>> You can have "lost updates" w/Cassandra.
You need to to use 3rd
>>> >>>>>>>>> products - cages for e.g. - to get ACID
type consistency.
>>> >>>>>>>>
>>> >>>>>>>> Then again, you'll have to define what you are
calling "lost
>>> >>>>>>>> updates". Provided you use a reasonable consistency
level,
>>> >>>>>>>> Cassandra
>>> >>>>>>>> provides fairly strong durability guarantee,
so for some
>>> >>>>>>>> definition you
>>> >>>>>>>> don't "lose updates".
>>> >>>>>>>> That being said, I never pretended that Cassandra
provided any
>>> >>>>>>>> ACID
>>> >>>>>>>> guarantee. ACID relates to transaction, which
Cassandra doesn't
>>> >>>>>>>> support. If
>>> >>>>>>>> we're talking about the guarantees of transaction,
then by all
>>> >>>>>>>> means,
>>> >>>>>>>> cassandra won't provide it. And yes you can
use cages or the like
>>> >>>>>>>> to get
>>> >>>>>>>> transaction. But that was not the point of the
thread, was it ?
>>> >>>>>>>> The thread
>>> >>>>>>>> is about vector clocks, and that has nothing
to do with
>>> >>>>>>>> transaction (vector
>>> >>>>>>>> clocks certainly don't give you transactions).
>>> >>>>>>>> Sorry if I wasn't clear in my mail, but I was
only responding to
>>> >>>>>>>> why
>>> >>>>>>>> so far I don't think vector clocks would really
provide much for
>>> >>>>>>>> Cassandra.
>>> >>>>>>>> --
>>> >>>>>>>> Sylvain
>>> >>>>>>>>
>>> >>>>>>>>>
>>> >>>>>>>>> -JA
>>> >>>>>>>>>
>>> >>>>>>>>> On Thu, Feb 24, 2011 at 7:41 AM, Sylvain
Lebresne
>>> >>>>>>>>> <sylvain@datastax.com> wrote:
>>> >>>>>>>>>>
>>> >>>>>>>>>> On Thu, Feb 24, 2011 at 3:22 AM, Anthony
John
>>> >>>>>>>>>> <chirayithaj@gmail.com> wrote:
>>> >>>>>>>>>>>
>>> >>>>>>>>>>> Apologies : For some reason my response
on the original mail
>>> >>>>>>>>>>> keeps bouncing back, thus this new
one!
>>> >>>>>>>>>>>
>>> >>>>>>>>>>> > From the other hand, the same
article says:
>>> >>>>>>>>>>> > "For conditional writes to
work, the condition must be
>>> >>>>>>>>>>> > evaluated at all update
>>> >>>>>>>>>>> > sites before the write can
be allowed to succeed."
>>> >>>>>>>>>>> >
>>> >>>>>>>>>>> > This means, that when doing
such an update CL=ALL must be
>>> >>>>>>>>>>> > used
>>> >>>>>>>>>>>
>>> >>>>>>>>>>> Sorry, but I am confused by that
entire thread!
>>> >>>>>>>>>>> Questions:-
>>> >>>>>>>>>>> 1. Does Cassandra implement any
kind of data locking - at any
>>> >>>>>>>>>>> granularity whether it be row/colF/Col
?
>>> >>>>>>>>>>
>>> >>>>>>>>>> No locking, no.
>>> >>>>>>>>>>
>>> >>>>>>>>>>>
>>> >>>>>>>>>>> 2. If the answer to 1 above is NO!
- how does CL ALL prevent
>>> >>>>>>>>>>> conflicts. Concurrent updates on
exactly the same piece of
>>> >>>>>>>>>>> data on different
>>> >>>>>>>>>>> nodes can still mess each other
up, right ?
>>> >>>>>>>>>>
>>> >>>>>>>>>> Not sure why you are taking CL.ALL specifically.
But in any CL,
>>> >>>>>>>>>> updating the same piece of data means
the same column value. In
>>> >>>>>>>>>> that case,
>>> >>>>>>>>>> the resolution rules are the following:
>>> >>>>>>>>>>   - If the updates have a different
timestamp, keep the one
>>> >>>>>>>>>> with
>>> >>>>>>>>>> the higher timestamp. That is, the more
recent of two updates
>>> >>>>>>>>>> win.
>>> >>>>>>>>>>   - It the timestamps are the same,
then it compares the values
>>> >>>>>>>>>> (byte comparison) and keep the highest
value. This is just to
>>> >>>>>>>>>> break ties in
>>> >>>>>>>>>> a consistent manner.
>>> >>>>>>>>>> So if you do two truly concurrent updates
(that is from two
>>> >>>>>>>>>> place
>>> >>>>>>>>>> at the same instant), then you'll end
with one of the update.
>>> >>>>>>>>>> This is the
>>> >>>>>>>>>> column level.
>>> >>>>>>>>>> However, if that simple conflict detection/resolution
mechanism
>>> >>>>>>>>>> is
>>> >>>>>>>>>> not good enough for some of your use
case and you need to keep
>>> >>>>>>>>>> two
>>> >>>>>>>>>> concurrent updates, it is easy enough.
Just make sure that the
>>> >>>>>>>>>> update don't
>>> >>>>>>>>>> end up in the same column. This is easily
achieved by appending
>>> >>>>>>>>>> some unique
>>> >>>>>>>>>> identifier to the column name for instance.
And when reading,
>>> >>>>>>>>>> do a slice and
>>> >>>>>>>>>> reconcile whatever you get back with
whatever logic make sense.
>>> >>>>>>>>>> If you do
>>> >>>>>>>>>> that, congrats, you've roughly emulated
what vector clocks
>>> >>>>>>>>>> would do. Btw, no
>>> >>>>>>>>>> locking or anything needed.
>>> >>>>>>>>>> In my experience, for most things the
timestamp resolution is
>>> >>>>>>>>>> enough. If the same user update twice
it's profile picture on
>>> >>>>>>>>>> you web site
>>> >>>>>>>>>> at the same microsecond, it's usually
fine to end up with one
>>> >>>>>>>>>> of the two
>>> >>>>>>>>>> pictures. In the rare case where you
need something more
>>> >>>>>>>>>> specific, using the
>>> >>>>>>>>>> cassandra data model usually solves
the problem easily. The
>>> >>>>>>>>>> reason for not
>>> >>>>>>>>>> having vector clocks in Cassandra is
that so far, we haven't
>>> >>>>>>>>>> really found
>>> >>>>>>>>>> much example where it is no the case.
>>> >>>>>>>>>>
>>> >>>>>>>>>> --
>>> >>>>>>>>>> Sylvain
>>> >>>>>>>>>
>>> >>>>>>>>
>>> >>>>>>>
>>> >>>>>>
>>> >>>>>
>>> >>>>
>>> >>>
>>> >>
>>> >
>>> >
>>
>>
>


Just to make a note the "EVENTUAL" in eventual consistency could be a
time that is less then 1ms.

I have a program that demonstrates that "eventual" means if i write
data at the weakest level, and read it back from a random another node
as soon as possible. 99% I see the update. I can share the code if you
would like.

Remember http://en.wikipedia.org/wiki/Spacetime
...but there is no reference frame in which the two events can occur
at the same time...

As to MongoDB references ....Yes! most of the noSQL work differently.
They each approach CAP
http://www.julianbrowne.com/article/viewer/brewers-cap-theorem in a
different way.

Cassandra does not lock (it is no secret). But remember, you can not
have it all pick 2/3 from CAP.

Mime
View raw message