You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Varun Barala (JIRA)" <ji...@apache.org> on 2018/12/16 17:05:00 UTC

[jira] [Comment Edited] (CASSANDRA-14702) Cassandra Write failed even when the required nodes to Ack(consistency) are up.

    [ https://issues.apache.org/jira/browse/CASSANDRA-14702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16722455#comment-16722455 ] 

Varun Barala edited comment on CASSANDRA-14702 at 12/16/18 5:04 PM:
--------------------------------------------------------------------

> We get the writetimeout exception from cassandra even when 2 nodes are up
In 5 node cluster with RF 3, you can not expect that every query will get executed successfully!

Because:-
Let's say you have 5 nodes with V Node 1 in each:-
Ring will look like:-
{code:java}
-9223372036854775808   1
-5534023222112865485   2
-1844674407370955162   3
1844674407370955161    4
5534023222112865484    5{code}
Now let's say you have keyspace:-
{code:java}
CREATE KEYSPACE ks1
WITH durable_writes = true
AND replication = {
    'class' : 'SimpleStrategy',
    'replication_factor' : 3
};
{code}
table structure:-
{code:java}
CREATE TABLE ks1.table1 (
    id boolean,
    pk1 boolean,
    pk2 boolean,
    ck1 int,
    PRIMARY KEY ((id,pk1,pk2))
);{code}
Let's insert two statements:-
{code:java}
insert into ks1.table1 (id, pk1, pk2, ck1) VALUES (true, true, true, 1);

insert into ks1.table1 (id, pk1, pk2, ck1) VALUES (false, true, true, 1);{code}
Let's see ring token for the above partitions:-
{code:java}
select token(id,pk1,pk2) from ks1.table1;{code}
It'll return result:-
{code:java}
-3439815377359905503 this belongs to node2 and should have replica node3 and node4
 6885159420904076627 this belongs to node5 and should have replica node1 and node2
{code}
Let's say you are updating above two partitions using batch statement and node2 and node5 are only up in the cluster.
{code:java}
As you can see query on node5 will be successful because node2 and node5 are satisfying quorum but query for node2 will fail since all the replicas (node3, node4) are down.{code}
Exception with only Node2 and Node5 up:-
{code:java}
Caused by: com.datastax.driver.core.exceptions.UnavailableException: Not enough replicas available for query at consistency LOCAL_QUORUM (2 required but only 1 alive)
    at com.datastax.driver.core.exceptions.UnavailableException.copy(UnavailableException.java:128)
    at com.datastax.driver.core.Responses$Error.asException(Responses.java:114)
    at com.datastax.driver.core.RequestHandler$SpeculativeExecution.onSet(RequestHandler.java:504)
    at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:1070)
    at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:993)
    at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105){code}
Then I tried to produce write time out scenario by shutting down nodes during query execution:-
{code:java}
Caused by: com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout during write query at consistency LOCAL_QUORUM (2 replica were required but only 1 acknowledged the write)
    at com.datastax.driver.core.exceptions.WriteTimeoutException.copy(WriteTimeoutException.java:100)
    at com.datastax.driver.core.Responses$Error.asException(Responses.java:122)
    at com.datastax.driver.core.RequestHandler$SpeculativeExecution.onSet(RequestHandler.java:504)
    at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:1070)
    at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:993)
    at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105){code}
*cassamdra version was 3.11.2*

[~rohitsngh27] I doubt you were having RF as 5. Can you check above scenarios? Let me know If I'm missing anything.

 

 


was (Author: varuna):
> We get the writetimeout exception from cassandra even when 2 nodes are up
In 5 node cluster with RF 3, you can not accept that every query will get executed successfully!

Because:-
Let's say you have 5 nodes with V Node 1 in each:-
Ring will look like:-
{code:java}

-9223372036854775808   1
-5534023222112865485   2
-1844674407370955162   3
1844674407370955161    4
5534023222112865484    5{code}

Now let's say you have keyspace:-
{code:java}
CREATE KEYSPACE ks1
WITH durable_writes = true
AND replication = {
    'class' : 'SimpleStrategy',
    'replication_factor' : 3
};
{code}

table structure:-
{code:java}
CREATE TABLE ks1.table1 (
    id boolean,
    pk1 boolean,
    pk2 boolean,
    ck1 int,
    PRIMARY KEY ((id,pk1,pk2))
);{code}

Let's insert two statements:-
{code:java}
insert into ks1.table1 (id, pk1, pk2, ck1) VALUES (true, true, true, 1);

insert into ks1.table1 (id, pk1, pk2, ck1) VALUES (false, true, true, 1);{code}

Let's see ring token for the above partitions:-
{code:java}
select token(id,pk1,pk2) from ks1.table1;{code}

It'll return result:-
{code:java}
-3439815377359905503 this belongs to node2 and should have replica node3 and node4
 6885159420904076627 this belongs to node5 and should have replica node1 and node2
{code}

Let's say you are updating above two partitions using batch statement and node2 and node5 are only up in the cluster.
{code:java}
As you can see query on node5 will be successful because node2 and node5 are satisfying quorum but query for node2 will fail since all the replicas (node3, node4) are down.{code}

Exception with only Node2 and Node5 up:-
{code:java}
Caused by: com.datastax.driver.core.exceptions.UnavailableException: Not enough replicas available for query at consistency LOCAL_QUORUM (2 required but only 1 alive)
    at com.datastax.driver.core.exceptions.UnavailableException.copy(UnavailableException.java:128)
    at com.datastax.driver.core.Responses$Error.asException(Responses.java:114)
    at com.datastax.driver.core.RequestHandler$SpeculativeExecution.onSet(RequestHandler.java:504)
    at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:1070)
    at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:993)
    at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105){code}

Then I tried to produce write time out scenario by shutting down nodes during query execution:-
{code:java}
Caused by: com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout during write query at consistency LOCAL_QUORUM (2 replica were required but only 1 acknowledged the write)
    at com.datastax.driver.core.exceptions.WriteTimeoutException.copy(WriteTimeoutException.java:100)
    at com.datastax.driver.core.Responses$Error.asException(Responses.java:122)
    at com.datastax.driver.core.RequestHandler$SpeculativeExecution.onSet(RequestHandler.java:504)
    at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:1070)
    at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:993)
    at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105){code}

*cassamdra version was 3.11.2*

[~rohitsngh27] I doubt you were having RF as 5. Can you check above scenarios? Let me know If I'm missing anything.

 

 

> Cassandra Write failed even when the required nodes to Ack(consistency) are up.
> -------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-14702
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14702
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Coordination
>            Reporter: Rohit Singh
>            Priority: Blocker
>
> Hi,
> We have following configuration in our project for cassandra. 
> Total nodes in Cluster-5
> Replication Factor- 3
> Consistency- LOCAL_QUORUM
> We get the writetimeout exception from cassandra even when 2 nodes are up and why does stack trace says that 3 replica were required when consistency is 2?
> Below is the exception we got:-
> com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout during write query at consistency LOCAL_QUORUM (3 replica were required but only 2 acknowledged the write)
>  at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:59)
>  at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:37)
>  at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:289)
>  at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:269)
>  at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org