Uploaded image for project: 'Cassandra'
  1. Cassandra
  2. CASSANDRA-10041

"timeout during write query at consistency ONE" when updating counter at consistency QUORUM and 2 of 3 nodes alive

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Not A Problem
    • Fix Version/s: 2.1.x
    • Component/s: None
    • Labels:
      None
    • Environment:

      centos 6.6 server, java version "1.8.0_45", cassandra 2.1.8, 3 machines, keyspace with replication factor 3

      Description

      Test scenario is: kill -9 one node, wait 60 seconds, start it back, wait till it becomes available, wait 120 seconds (during that time all 3 nodes are up), repeat with the next node. Application reads from one table and updates counters in another table with consistency QUORUM. When one node out of 3 is killed application logs this exception for several seconds:

      Caused by: com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout during write query at consistency ONE (1 replica were required but only 0 acknowledged the write)
              at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:57) ~[com.datastax.cassandra.cassandra-driver-core-2.1.6.jar:na]
              at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:37) ~[com.datastax.cassandra.cassandra-driver-core-2.1.6.jar:na]
              at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:204) ~[com.datastax.cassandra.cassandra-driver-core-2.1.6.jar:na]
              at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:195) ~[com.datastax.cassandra.cassandra-driver-core-2.1.6.jar:na]
              at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:89) [io.netty.netty-codec-4.0.27.Final.jar:4.0.27.Final]
              ... 13 common frames omitted
      

        Activity

        Hide
        re_weavers Rob Emery added a comment -

        Interesting; from our perspective the nodes that intermittently throw this exception are not actually down. They are delivering other workloads without any apparent problem. We just get a few of these per day and have been unable to track down any cause (there does not appear to be any IO pressure or GC pauses etc); therefore for our situation all information, no matter how small, is vital for us to be able to diagnose the problem. Currently we have no way of tracking the point during the mutation that the issue is actually occurring.

        Show
        re_weavers Rob Emery added a comment - Interesting; from our perspective the nodes that intermittently throw this exception are not actually down. They are delivering other workloads without any apparent problem. We just get a few of these per day and have been unable to track down any cause (there does not appear to be any IO pressure or GC pauses etc); therefore for our situation all information, no matter how small, is vital for us to be able to diagnose the problem. Currently we have no way of tracking the point during the mutation that the issue is actually occurring.
        Hide
        slebresne Sylvain Lebresne added a comment -

        In this case, it is not possible to identify in which phase the counter mutation failed.

        That's right, you can't (identify in which phase the counter mutation failed). But given how counters currently work we can't send you that information: the timeout is sent by the coordinator which only get acks once everything is finished, so if it doesn't get acks, it doesn't know which phase we're in. We'd need to change the protocol used internally as suggested a long time ago in CASSANDRA-3199, but we've so far decided that the ROI for that wasn't good enough (mostly due to the huge headache that making this change while maintaining backward compatibility/rolling upgrade would be). Note in particular that even doing that wouldn't avoid the timeout, it would just make a tiny bit more info available to the coordinator when it happens but that info might not even help being sure whether the counter update has been persisted or not.

        Overall, closing that issue as not a problem. Yes, whenever a node dies some counter inserts can timeout during the windows it takes for the failure detector to mark that node dead and this even if you have in theory enough nodes alive to fulfill the CL requirements. And yes, that's sad. But it's unfortunately a intrinsic limitation of the counter design for which we don't have a solution.

        Or to put it another way, this is working as designed, which doesn't mean we disagree that this is a weakness of said design.

        Show
        slebresne Sylvain Lebresne added a comment - In this case, it is not possible to identify in which phase the counter mutation failed. That's right, you can't (identify in which phase the counter mutation failed). But given how counters currently work we can't send you that information: the timeout is sent by the coordinator which only get acks once everything is finished, so if it doesn't get acks, it doesn't know which phase we're in. We'd need to change the protocol used internally as suggested a long time ago in CASSANDRA-3199 , but we've so far decided that the ROI for that wasn't good enough (mostly due to the huge headache that making this change while maintaining backward compatibility/rolling upgrade would be). Note in particular that even doing that wouldn't avoid the timeout, it would just make a tiny bit more info available to the coordinator when it happens but that info might not even help being sure whether the counter update has been persisted or not. Overall, closing that issue as not a problem. Yes, whenever a node dies some counter inserts can timeout during the windows it takes for the failure detector to mark that node dead and this even if you have in theory enough nodes alive to fulfill the CL requirements. And yes, that's sad. But it's unfortunately a intrinsic limitation of the counter design for which we don't have a solution. Or to put it another way, this is working as designed, which doesn't mean we disagree that this is a weakness of said design.
        Hide
        jorgebg Jorge Bay added a comment -

        The difference with CASSANDRA-9620 is that this one can not be identified in the retry policy.

        Batch log writes that timeout contain that information in the error writeType, which allows retry policies / any clients to identify it and retry accordingly.

        In this case, it is not possible to identify in which phase the counter mutation failed.

        Show
        jorgebg Jorge Bay added a comment - The difference with CASSANDRA-9620 is that this one can not be identified in the retry policy. Batch log writes that timeout contain that information in the error writeType , which allows retry policies / any clients to identify it and retry accordingly. In this case, it is not possible to identify in which phase the counter mutation failed.
        Hide
        jkni Joel Knighton added a comment -

        I should have probably included more detail in my initial post.

        Any node in a Cassandra cluster can act as coordinator, not only replicas. I assume what you mean is that you are using token-awareness in the driver in an attempt to route your query to a coordinator that is also a replica.

        Even in situations where the coordinator is a replica, the coordinator may choose a leader other than itself for the counter mutation. You should not interpret these timeouts as an indication that you are not routing your queries from the driver optimally.

        Show
        jkni Joel Knighton added a comment - I should have probably included more detail in my initial post. Any node in a Cassandra cluster can act as coordinator, not only replicas. I assume what you mean is that you are using token-awareness in the driver in an attempt to route your query to a coordinator that is also a replica. Even in situations where the coordinator is a replica, the coordinator may choose a leader other than itself for the counter mutation. You should not interpret these timeouts as an indication that you are not routing your queries from the driver optimally.
        Hide
        re_weavers Rob Emery added a comment - - edited

        Hi Joel,

        Am I reading this correctly in that this implies that the UPDATE query isn't being sent to a replica? Which would imply that we have misconfigured the partition key within our queries?

        Thanks,

        Show
        re_weavers Rob Emery added a comment - - edited Hi Joel, Am I reading this correctly in that this implies that the UPDATE query isn't being sent to a replica? Which would imply that we have misconfigured the partition key within our queries? Thanks,
        Hide
        jkni Joel Knighton added a comment -

        I don't think this is cause for concern - as in CASSANDRA-9620, this is a result of the write path for the type of mutation.

        When writing to a counter, a replica is selected as the leader for the mutation. If the leader is not the coordinator, we send this mutation to the leader with CL.ONE. It is a timeout on this that you're seeing. Since there's no clear reason to handle timeouts on a leader write/coordinator write differently, these are both WriteType COUNTER (as opposed to the BATCH/BATCH_LOG disctinction).

        Show
        jkni Joel Knighton added a comment - I don't think this is cause for concern - as in CASSANDRA-9620 , this is a result of the write path for the type of mutation. When writing to a counter, a replica is selected as the leader for the mutation. If the leader is not the coordinator, we send this mutation to the leader with CL.ONE. It is a timeout on this that you're seeing. Since there's no clear reason to handle timeouts on a leader write/coordinator write differently, these are both WriteType COUNTER (as opposed to the BATCH/BATCH_LOG disctinction).
        Hide
        jorgebg Jorge Bay added a comment -

        I was able to reproduce it with 3.0.x:

        ccm create test -n 3 -b -s -v 3.0.4
        ccm node1 cqlsh
        
        CREATE KEYSPACE ks1 WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};
        USE ks1;
        CREATE TABLE tbl1 (id int PRIMARY KEY, c counter);
        CONSISTENCY QUORUM
        UPDATE tbl1 SET c = c + 1 WHERE id = 1;
        exit
        
        ccm node2 pause
        ccm node1 cqlsh
        
        USE ks1
        CONSISTENCY QUORUM
        UPDATE tbl1 SET c = c + 1 WHERE id = 1;
        
        Show
        jorgebg Jorge Bay added a comment - I was able to reproduce it with 3.0.x: ccm create test -n 3 -b -s -v 3.0.4 ccm node1 cqlsh CREATE KEYSPACE ks1 WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3}; USE ks1; CREATE TABLE tbl1 (id int PRIMARY KEY, c counter); CONSISTENCY QUORUM UPDATE tbl1 SET c = c + 1 WHERE id = 1; exit ccm node2 pause ccm node1 cqlsh USE ks1 CONSISTENCY QUORUM UPDATE tbl1 SET c = c + 1 WHERE id = 1;
        Hide
        re_weavers Rob Emery added a comment -

        We believe that we are experiencing this issue also; we have raised the issue with DataStax at the moment because we believed it may be a driver issue; however if this is reproducable in other places it's possible that it's actually an issue within Cassandra. We are running on Cassanadra 2.0.17

        The issue we raised: https://datastax-oss.atlassian.net/browse/CSHARP-438

        {
            "ConsistencyLevel": 1,
            "Message": "Cassandra timeout during write query at consistency ONE (0 replica(s) acknowledged the write over 1 required)",
            "RequiredAcknowledgements": 1,
            "ReceivedAcknowledgements": 0,
            "WriteType": "COUNTER"
        }
        
        Show
        re_weavers Rob Emery added a comment - We believe that we are experiencing this issue also; we have raised the issue with DataStax at the moment because we believed it may be a driver issue; however if this is reproducable in other places it's possible that it's actually an issue within Cassandra. We are running on Cassanadra 2.0.17 The issue we raised: https://datastax-oss.atlassian.net/browse/CSHARP-438 { "ConsistencyLevel" : 1, "Message" : "Cassandra timeout during write query at consistency ONE (0 replica(s) acknowledged the write over 1 required)" , "RequiredAcknowledgements" : 1, "ReceivedAcknowledgements" : 0, "WriteType" : "COUNTER" }
        Hide
        sylvestor88 Sylvestor George added a comment - - edited

        I am able to reproduce the exception as per the given specification.

        Caused by: com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout during write query at consistency ONE (1 replica were required but only 0 acknowledged the write)
        	at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:54) ~[cassandra-driver-core-2.0.9.2.jar:na]
        	at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:34) ~[cassandra-driver-core-2.0.9.2.jar:na]
        	at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:182) ~[cassandra-driver-core-2.0.9.2.jar:na]
        	at com.datastax.shaded.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:66) ~[cassandra-driver-core-2.0.9.2.jar:na]
        	... 21 common frames omitted
        
        Show
        sylvestor88 Sylvestor George added a comment - - edited I am able to reproduce the exception as per the given specification. Caused by: com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout during write query at consistency ONE (1 replica were required but only 0 acknowledged the write) at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:54) ~[cassandra-driver-core-2.0.9.2.jar:na] at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:34) ~[cassandra-driver-core-2.0.9.2.jar:na] at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:182) ~[cassandra-driver-core-2.0.9.2.jar:na] at com.datastax.shaded.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:66) ~[cassandra-driver-core-2.0.9.2.jar:na] ... 21 common frames omitted

          People

          • Assignee:
            Unassigned
            Reporter:
            mabrek Anton Lebedevich
          • Votes:
            2 Vote for this issue
            Watchers:
            8 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development