Re: QUORUM writes, QUORUM reads -- and eventual consistency

2012-08-27 Thread aaron morton
  Doesn't this mean that the read does not reflect the most recent write?
Yes. 
A write that fails is not a write. 

 If it were to have read the newer data from the 1 node and then afterwards 
 read the old data from the other 2 then there is a consistency problem, but 
 in the example you give the second reader seems to still have a consistent 
 view.
In the scenario of a TimedOutException for a write that is entirely possible. 
The write is not considered to be successful at the CL requested. So R + W  N 
does not hold for that datum. 

When in doubt, ask Werner…

when R + W  N we have strong consistency…
Strong consistency. After the update completes, any subsequent access (by A, 
B, or C) will return the updated value.

when R + W = N we have weak / eventual consistency…
*Eventual consistency. This is a specific form of weak consistency; the 
storage system guarantees that if no new updates are made to the object, 
eventually *all* accesses will return the last updated value.

http://queue.acm.org/detail.cfm?id=1466448
(emphasis added)

In C* this may mean HH or RR or repair or standard CL checks kicking in to make 
the second read return the correct consistent value. 

 Isn't it cheaper to retry the mutation on _any exception_ than to have a 
 transaction in place for the majority of non failing writes?
Yes (with the counter exception). 

if you get an UnavailableException it's from the point of view of the 
coordinator. it may be the case that the coordinator is isolated and all the 
other nodes are UP and happy. 

Hope that helps. 

-
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com

On 26/08/2012, at 5:03 AM, Guillermo Winkler gwink...@inconcertcc.com wrote:

 Isn't it cheaper to retry the mutation on _any exception_ than to have a 
 transaction in place for the majority of non failing writes?
 
 The special case to be considered is obviously counters which are not 
 idempotent
 
 https://issues.apache.org/jira/browse/CASSANDRA-2495 
 
 
 
 On Sat, Aug 25, 2012 at 4:38 AM, Russell Haering russellhaer...@gmail.com 
 wrote:
 The issue is that it is possible for a quorum write to return an
 error, but for the result of the write to still be reflected in the
 view seen by the client. There is really no performant way around this
 (although reading at ALL can make it much less frequent). Guaranteeing
 complete success or failure would (barring a creative solution I'm
 unaware of) require a transactional commit of some sort across the
 replica nodes for the key being written to. The performance tradeoff
 might be desirable under some circumstances, but if this is a
 requirement you should probably look at other databases.
 
 Some good rules to play by (someone correct me if these aren't 100% true):
 
 1. For writes to a single key, an UnavailableException means the write
 failed totally (clients will never see the data you wrote)
 2. For writes to a single key, a TimedOutException means you cannot
 know whether the write succeeded or failed
 3. For writes to multiple keys, either an UnavailableException or a
 TimedOutException means you cannot know whether the write succeeded or
 failed.
 
 -Russell
 
 On Sat, Aug 25, 2012 at 12:17 AM, Guillermo Winkler
 gwink...@inconcertcc.com wrote:
  Hi Philip,
 
  From http://wiki.apache.org/cassandra/ArchitectureOverview
 
  Quorum write: blocks until quorum is reached
 
  By my understanding if you _did_ a quorum write it means it successfully
  completed.
 
  Guille
 
 
  I *think* we're saying the same thing here. The addition of the word
  successful (or something more suitable) would make the documentation more
  precise, not less.
 



Re: QUORUM writes, QUORUM reads -- and eventual consistency

2012-08-27 Thread Philip O'Toole
Cool - thanks to all for the replies. I believe I have what I need now. 

Philip

On Aug 25, 2012, at 12:17 AM, Guillermo Winkler gwink...@inconcertcc.com 
wrote:

 Hi Philip, 
 
 From http://wiki.apache.org/cassandra/ArchitectureOverview
 
 Quorum write: blocks until quorum is reached
 
 By my understanding if you _did_ a quorum write it means it successfully 
 completed.
 
 Guille
 
 
 I *think* we're saying the same thing here. The addition of the word 
 successful (or something more suitable) would make the documentation more 
 precise, not less.


Re: QUORUM writes, QUORUM reads -- and eventual consistency

2012-08-25 Thread Philip O'Toole
Hi Derek -- thanks. More inline.

On Fri, Aug 24, 2012 at 11:52:49PM -0600, Derek Williams wrote:
 On Fri, Aug 24, 2012 at 10:55 PM, Philip O'Toole phi...@loggly.com wrote:
 
  But consider this. Say I have a replication factor of 3. I request a
  QUORUM write, and it fails because the write only reaches 1 node. Perhaps
  there is a temporary partition in my cluster. Now, asynchronously, a
  different reader performs a QUORUM read of the same cluster and just before
  it issues the read, the partition is resolved. The quorum read is satisfied
  by the two nodes that have *not* received the latest write (yet). Doesn't
  this mean that the read does not reflect the most recent write? I realise
  this is very unlikely to happen in practise, but I want to be sure I
  understand all this.
 
 
 Others might disagree, but as long as the view from the second reader
 remains consistent then I see no problem. If it were to have read the newer
 data from the 1 node and then afterwards read the old data from the other 2
 then there is a consistency problem, but in the example you give the second
 reader seems to still have a consistent view. Trying to guarantee that all
 clients will have the same view at all times is working against Cassandra's
 strengths.

I can agree with this interpretation, and that it is a reasonable 
interpretation of consistency.

 
 Where quorum reads and writes are most important is when consistency is
 required from the point of view of a single client.

A single client is exactly what I am thinking about.

 
 This is besides the point that the documentation states that the sum of the
 nodes written to and read from needs to be greater then the replication
 factor for the statement to be true. In your example only 1 node was
 written to, when 2 were required to guarantee consistency. The intent to do
 a quorum write is not the same as actually doing one.

I *think* we're saying the same thing here. The addition of the word 
successful (or something more suitable) would make the documentation more 
precise, not less.

 
 -- 
 Derek Williams

-- 
Philip O'Toole
Senior Developer
Loggly, Inc.
San Francisco, CA


Re: QUORUM writes, QUORUM reads -- and eventual consistency

2012-08-25 Thread Guillermo Winkler
Hi Philip,

From http://wiki.apache.org/cassandra/ArchitectureOverview

*Quorum write*: blocks until quorum is reached

By my understanding if you _did_ a quorum write it means it successfully
completed.

Guille


I *think* we're saying the same thing here. The addition of the word
 successful (or something more suitable) would make the documentation more
 precise, not less.



Re: QUORUM writes, QUORUM reads -- and eventual consistency

2012-08-25 Thread Russell Haering
The issue is that it is possible for a quorum write to return an
error, but for the result of the write to still be reflected in the
view seen by the client. There is really no performant way around this
(although reading at ALL can make it much less frequent). Guaranteeing
complete success or failure would (barring a creative solution I'm
unaware of) require a transactional commit of some sort across the
replica nodes for the key being written to. The performance tradeoff
might be desirable under some circumstances, but if this is a
requirement you should probably look at other databases.

Some good rules to play by (someone correct me if these aren't 100% true):

1. For writes to a single key, an UnavailableException means the write
failed totally (clients will never see the data you wrote)
2. For writes to a single key, a TimedOutException means you cannot
know whether the write succeeded or failed
3. For writes to multiple keys, either an UnavailableException or a
TimedOutException means you cannot know whether the write succeeded or
failed.

-Russell

On Sat, Aug 25, 2012 at 12:17 AM, Guillermo Winkler
gwink...@inconcertcc.com wrote:
 Hi Philip,

 From http://wiki.apache.org/cassandra/ArchitectureOverview

 Quorum write: blocks until quorum is reached

 By my understanding if you _did_ a quorum write it means it successfully
 completed.

 Guille


 I *think* we're saying the same thing here. The addition of the word
 successful (or something more suitable) would make the documentation more
 precise, not less.


Re: QUORUM writes, QUORUM reads -- and eventual consistency

2012-08-25 Thread Guillermo Winkler
Isn't it cheaper to retry the mutation on _any exception_ than to have a
transaction in place for the majority of non failing writes?

The special case to be considered is obviously counters which are not
idempotent

https://issues.apache.org/jira/browse/CASSANDRA-2495



On Sat, Aug 25, 2012 at 4:38 AM, Russell Haering
russellhaer...@gmail.comwrote:

 The issue is that it is possible for a quorum write to return an
 error, but for the result of the write to still be reflected in the
 view seen by the client. There is really no performant way around this
 (although reading at ALL can make it much less frequent). Guaranteeing
 complete success or failure would (barring a creative solution I'm
 unaware of) require a transactional commit of some sort across the
 replica nodes for the key being written to. The performance tradeoff
 might be desirable under some circumstances, but if this is a
 requirement you should probably look at other databases.

 Some good rules to play by (someone correct me if these aren't 100% true):

 1. For writes to a single key, an UnavailableException means the write
 failed totally (clients will never see the data you wrote)
 2. For writes to a single key, a TimedOutException means you cannot
 know whether the write succeeded or failed
 3. For writes to multiple keys, either an UnavailableException or a
 TimedOutException means you cannot know whether the write succeeded or
 failed.

 -Russell

 On Sat, Aug 25, 2012 at 12:17 AM, Guillermo Winkler
 gwink...@inconcertcc.com wrote:
  Hi Philip,
 
  From http://wiki.apache.org/cassandra/ArchitectureOverview
 
  Quorum write: blocks until quorum is reached
 
  By my understanding if you _did_ a quorum write it means it successfully
  completed.
 
  Guille
 
 
  I *think* we're saying the same thing here. The addition of the word
  successful (or something more suitable) would make the documentation
 more
  precise, not less.



QUORUM writes, QUORUM reads -- and eventual consistency

2012-08-24 Thread Philip O'Toole
Hello -- perhaps someone could provide me some clarification about this.

From:

http://www.datastax.com/docs/1.1/dml/data_consistency#data-consistency

If consistency is top priority, you can ensure that a read will always reflect 
the most recent write by using the following formula:

(nodes_written + nodes_read)  replication_factor

But consider this. Say I have a replication factor of 3. I request a QUORUM 
write, and it fails because the write only reaches 1 node. Perhaps there is a 
temporary partition in my cluster. Now, asynchronously, a different reader 
performs a QUORUM read of the same cluster and just before it issues the read, 
the partition is resolved. The quorum read is satisfied by the two nodes that 
have *not* received the latest write (yet). Doesn't this mean that the read 
does not reflect the most recent write? I realise this is very unlikely to 
happen in practise, but I want to be sure I understand all this.

Perhaps the documentation would be more correct if the statement read as 
...reflect the most recent SUCCESSFUL write...?

Thanks,

Philip

-- 
Philip O'Toole
Senior Developer
Loggly, Inc.
San Francisco, CA


Re: QUORUM writes, QUORUM reads -- and eventual consistency

2012-08-24 Thread Derek Williams
On Fri, Aug 24, 2012 at 10:55 PM, Philip O'Toole phi...@loggly.com wrote:

 But consider this. Say I have a replication factor of 3. I request a
 QUORUM write, and it fails because the write only reaches 1 node. Perhaps
 there is a temporary partition in my cluster. Now, asynchronously, a
 different reader performs a QUORUM read of the same cluster and just before
 it issues the read, the partition is resolved. The quorum read is satisfied
 by the two nodes that have *not* received the latest write (yet). Doesn't
 this mean that the read does not reflect the most recent write? I realise
 this is very unlikely to happen in practise, but I want to be sure I
 understand all this.


Others might disagree, but as long as the view from the second reader
remains consistent then I see no problem. If it were to have read the newer
data from the 1 node and then afterwards read the old data from the other 2
then there is a consistency problem, but in the example you give the second
reader seems to still have a consistent view. Trying to guarantee that all
clients will have the same view at all times is working against Cassandra's
strengths.

Where quorum reads and writes are most important is when consistency is
required from the point of view of a single client.

This is besides the point that the documentation states that the sum of the
nodes written to and read from needs to be greater then the replication
factor for the statement to be true. In your example only 1 node was
written to, when 2 were required to guarantee consistency. The intent to do
a quorum write is not the same as actually doing one.

-- 
Derek Williams