For the vast majority of my data usage eventual consistency is fine (i.e.
CL=ONE) but I have a small amount of critical data for which I read and
write using CL=QUORUM.
If I have a cluster with 3 nodes and RF=2, and CL=QUORUM does that mean that
a value can be read from or written to any 2 nodes,
to me it makes sense that if hinted handoff is off then cassandra cannot
satisfy 2 out of every 3rd writes writes when one of the nodes is down since
this node is the designated node of 2/3 writes.
But I don't remember reading this somewhere. Does hinted handoff affect
David's situation?
(David,
Actually you can't. As explained in the wiki page linked:
A hinted write does NOT count towards ConsistencyLevel requirements
of ONE, QUORUM, or ALL
For CL.QUORUM, you do need QUORUM *replicas* to be alive to answer
the query. At RF=2, QUORUM=2 so no, you cannot take down any node
down or (some)
Thank you, Jake. It does... except that in another context you told me:
Hints only happen when a node is unavailable and you are writing with CL.ANY
If you never write with CL.ANY then you can turn off hinted handoff.
How do I reconcile this?
On Sun, Nov 28, 2010 at 7:11 PM, Jake Luciani
On Sun, Nov 28, 2010 at 6:26 PM, David Boxenhorn da...@lookin2.com wrote:
Hints only happen when a node is unavailable and you are writing with CL.ANY
If you never write with CL.ANY then you can turn off hinted handoff.
Hints happens at any consistency level. But they never helps to
reach the
I was wrong on this scenario and I'll explain where I was incorrect.
Hints are stored for a downed node but they don't count towards meeting a
consistency level.
Let's take 2 scenarios:
RF=6, Nodes=10
If you READ/WRITE with CL.QUORUM you will need 4 alive nodes if one is down
it will still have
OK. To sum up: RF=2 and QUORUM are incompatible (if you want to be able to
take a node down).
Right?
On Sun, Nov 28, 2010 at 7:59 PM, Jake Luciani jak...@gmail.com wrote:
I was wrong on this scenario and I'll explain where I was incorrect.
Hints are stored for a downed node but they don't
Right.
On Sun, Nov 28, 2010 at 1:03 PM, David Boxenhorn da...@lookin2.com wrote:
OK. To sum up: RF=2 and QUORUM are incompatible (if you want to be able to
take a node down).
Right?
On Sun, Nov 28, 2010 at 7:59 PM, Jake Luciani jak...@gmail.com wrote:
I was wrong on this scenario and
Put another way, for RF=2, QUORUM is equivalent to ALL.
On Sun, Nov 28, 2010 at 12:03 PM, David Boxenhorn da...@lookin2.com wrote:
OK. To sum up: RF=2 and QUORUM are incompatible (if you want to be able to
take a node down).
Right?
--
Jonathan Ellis
Project Chair, Apache Cassandra