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 4 active replicas to write to, one of these will store a
hint and update the downed node when it comes back.

RF=2, Nodes=3

If you READ/WRITE with CL.QUORUM you need 2 live nodes.  If one of these 2
are down you can't meet the QUORUM level so the write will fail.

In your scenario your best bet is to update to RF=3, then any two nodes will
accept QUORUM

Sorry for the confusion,

-Jake

On Sun, Nov 28, 2010 at 12:26 PM, David Boxenhorn <da...@lookin2.com> wrote:

> 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 <jak...@gmail.com> wrote:
>
>> If you read/write data with quorum then you can safely take a node down in
>> this scenario.  Subsequent writes will use hinted handoff to be passed to
>> the node when it comes back up.
>>
>> More info is here: http://wiki.apache.org/cassandra/HintedHandoff
>>
>> Does that answer your question?
>>
>> -Jake
>>
>>
>> On Sun, Nov 28, 2010 at 9:42 AM, Ran Tavory <ran...@gmail.com> wrote:
>>
>>> 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, did you disable HH in your storage-config?
>>> <HintedHandoffEnabled>false</HintedHandoffEnabled>)
>>>
>>>
>>> On Sun, Nov 28, 2010 at 4:32 PM, David Boxenhorn <da...@lookin2.com>wrote:
>>>
>>>> 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, or does it have to
>>>> be the particular 2 nodes that store the data? If it is the particular 2
>>>> nodes that store the data, that means that I can't even take down one node,
>>>> since it will be the mandatory 2nd node for 1/3 of my data...
>>>>
>>>
>>>
>>>
>>> --
>>> /Ran
>>>
>>>
>>
>

Reply via email to