Return-Path: Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: (qmail 99015 invoked from network); 24 Feb 2011 19:17:14 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 24 Feb 2011 19:17:14 -0000 Received: (qmail 94672 invoked by uid 500); 24 Feb 2011 19:17:11 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 94557 invoked by uid 500); 24 Feb 2011 19:17:09 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 94541 invoked by uid 99); 24 Feb 2011 19:17:08 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 24 Feb 2011 19:17:08 +0000 X-ASF-Spam-Status: No, hits=-0.7 required=5.0 tests=FREEMAIL_FROM,RCVD_IN_DNSWL_LOW,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of s5alye@gmail.com designates 209.85.214.172 as permitted sender) Received: from [209.85.214.172] (HELO mail-iw0-f172.google.com) (209.85.214.172) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 24 Feb 2011 19:16:57 +0000 Received: by iwl42 with SMTP id 42so548878iwl.31 for ; Thu, 24 Feb 2011 11:16:36 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=domainkey-signature:mime-version:in-reply-to:references:date :message-id:subject:from:to:cc:content-type :content-transfer-encoding; bh=0jHxPNBk/b6iT9/WV+7GI10fJDXk2OrT/nZoXiZh+DE=; b=wkirmcQLd0R0BnG0sQaBbnJeTB//up/orG7oJhT+vFBR1sTHc0OKecK6a54YyDyWU7 L/Wxbe0DZh78Lh0EHHfE00qD/imqaLEqg8IgbU/wnrjfwnAR/07skROmYBAMKsPk4/uF b5opjjPCMOg4mjiv05NAHosGZCXf/tsLP2AWc= DomainKey-Signature: a=rsa-sha1; c=nofws; d=gmail.com; s=gamma; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :cc:content-type:content-transfer-encoding; b=BGdLwhf77JCzcFtqk3ENvMZL19GipLELQdDgH7qXudy41BaIhSksoOmnvdOIlNL+Ss 7CHA4FccPIMwzzMaHOgfAGmyWrNnIAJbrawTujI3TqXqrzCs6Ue9JW8p9XgJuu/qk22d NavbTJ+Ma+PuVrW0YOROlRbJm4Da5cxD/Xa6I= MIME-Version: 1.0 Received: by 10.231.36.198 with SMTP id u6mr1987386ibd.100.1298574995888; Thu, 24 Feb 2011 11:16:35 -0800 (PST) Received: by 10.231.12.71 with HTTP; Thu, 24 Feb 2011 11:16:35 -0800 (PST) In-Reply-To: References: Date: Thu, 24 Feb 2011 14:16:35 -0500 Message-ID: Subject: Re: New Chain for : Does Cassandra use vector clocks From: A J To: user@cassandra.apache.org Cc: Anthony John , Sylvain Lebresne Content-Type: text/plain; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable X-Virus-Checked: Checked by ClamAV on apache.org >>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=3D1 and R=3DN . 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=3DR=3DN and fsync before each write commit, there will be scope for inconsistency. On Thu, Feb 24, 2011 at 1:25 PM, Anthony John 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 implementati= ons > ) or a Read at CL of All. Because a write could have partially gone throu= gh. > 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 > =A0any server or not. Is that a fair=A0characterization=A0? > Bottom line - unlike traditional DBMS, errors do not ensure automatic > cleanup and revert back, app code has to follow up if =A0immediate - and = not > eventual - =A0consistency is desired. I made that leap in almost all case= s - I > think - but the case of a failed write. > My bad and=A0I can live with this! > Regards, > -JA > > On Thu, Feb 24, 2011 at 11:50 AM, Sylvain Lebresne > wrote: >> >> On Thu, Feb 24, 2011 at 6:33 PM, Anthony John >> 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 th= e >>> actual returned result or other more convoluted conditions , then a Quo= rum >>> 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=A0succeeding=A0write 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 abo= ut >> ACID consistency, but eventual consistency, I don't think the wiki is ma= king >> 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 ma= ke >>> this statement on the Wiki architecture section:- >>> ------------------------------------------------------------- >>> >>> More specifically:=A0R=3Dread replica count=A0W=3Dwrite replica >>> count=A0N=3Dreplication factor=A0Q=3DQUORUM=A0(Q =3D N / 2 + 1) >>> >>> If W + R > N, you will have consistency >>> >>> W=3D1, R=3DN >>> W=3DN, R=3D1 >>> W=3DQ, R=3DQ where Q =3D N / 2 + 1 >>> >>> Cassandra provides consistency when R + W > N (read replica count +=A0w= rite >>> replica count > replication factor). >>> >>> ---------------------------------------------------- >>> >>> . >>> >>> On Thu, Feb 24, 2011 at 11:22 AM, Sylvain Lebresne >>> wrote: >>>> >>>> On Thu, Feb 24, 2011 at 6:01 PM, Anthony John >>>> 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:=A0http://www.allthingsdistributed.com/2007/12/eventually_consist= ent.html) >>>>> >>>>> On Thu, Feb 24, 2011 at 10:54 AM, Sylvain Lebresne >>>>> wrote: >>>>>> >>>>>> On Thu, Feb 24, 2011 at 5:34 PM, Anthony John >>>>>> 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 whi= ch >>>>>>> >> update wins (which I'll call a resolution). >>>>>>> I understand what you are saying, and yes semantics is very importa= nt >>>>>>> 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 no= t >>>>>>> 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 =A0Write 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 th= e >>>>>>> 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=3D3, 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 th= e >>>>>> timestamp and decide what to return based on this. If N2/N3 responds >>>>>> however, both timestamp will be TS1 and so, after timestamp resoluti= on, 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 l= et >>>>>> 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 al= ways >>>>>> 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 >>>>>>> wrote: >>>>>>>> >>>>>>>> On Thu, Feb 24, 2011 at 4:52 PM, Anthony John >>>>>>>> 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 whic= h >>>>>>>> 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, Cassand= ra >>>>>>>> provides fairly strong durability guarantee, so for some definitio= n you >>>>>>>> don't "lose updates". >>>>>>>> That being said,=A0I never pretended that Cassandra provided any A= CID >>>>>>>> guarantee. ACID relates to transaction, which Cassandra doesn't su= pport. If >>>>>>>> we're talking about the guarantees of transaction, then by all mea= ns, >>>>>>>> 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 ? Th= e thread >>>>>>>> is about vector clocks, and that has nothing to do with transactio= n (vector >>>>>>>> clocks certainly don't give you transactions). >>>>>>>> Sorry if I wasn't clear in my mail, but I was only responding to w= hy >>>>>>>> so far I don't think vector clocks would really provide much for C= assandra. >>>>>>>> -- >>>>>>>> Sylvain >>>>>>>> >>>>>>>>> >>>>>>>>> -JA >>>>>>>>> >>>>>>>>> On Thu, Feb 24, 2011 at 7:41 AM, Sylvain Lebresne >>>>>>>>> wrote: >>>>>>>>>> >>>>>>>>>> On Thu, Feb 24, 2011 at 3:22 AM, Anthony John >>>>>>>>>> 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=3DALL must be u= sed >>>>>>>>>>> >>>>>>>>>>> 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: >>>>>>>>>> =A0=A0- If the updates have a different timestamp, keep the one = with >>>>>>>>>> the higher timestamp. That is, the more recent of two updates wi= n. >>>>>>>>>> =A0=A0- It the timestamps are the same, then it compares the val= ues >>>>>>>>>> (byte comparison) and keep the highest value. This is just to br= eak ties in >>>>>>>>>> a consistent manner. >>>>>>>>>> So if you do two truly concurrent updates (that is from two plac= e >>>>>>>>>> at the same instant), then you'll end with one of the update. Th= is 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 t= wo >>>>>>>>>> concurrent updates, it is easy enough. Just make sure that the u= pdate 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 yo= u 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 specifi= c, using the >>>>>>>>>> cassandra data model usually solves the problem easily. The reas= on for not >>>>>>>>>> having vector clocks in Cassandra is that so far, we haven't rea= lly found >>>>>>>>>> much example where it is no the case. >>>>>>>>>> >>>>>>>>>> -- >>>>>>>>>> Sylvain >>>>>>>>> >>>>>>>> >>>>>>> >>>>>> >>>>> >>>> >>> >> > >