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

Runtian Liu commented on CASSANDRA-19120:
-----------------------------------------

[~smiklosovic] I think trunk(5.1) and 5.0 is basically same. Although 5.0 the 
int blockFor() function is returning "writePlan.writeQuorum();", looks like the 
function is never called. The latch is still initialized with the local 
variable blockFor in the constructor. This is same as the trunk version of 
adjustedBlockFor. The problem here for blocking read repair is that the cross 
DC node has not been contacted before(No read from the cross DC node). It's the 
[selector|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/locator/ReplicaPlans.java#L411]
 which is requiring 1 extra node to response when applying read repair mutation 
[[1]|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/locator/ReplicaPlans.java#L572].

This means for replication factor 3 in each DC, sending read request with 
local_quorum assuming no speculative retry is triggered. When we get digest 
mismatch, blocking read repair will start. For normal scenario, blocking read 
repair will only try to repair the replicas that have been contacted(read) 
before, also we want to satisfy the consistency blockFor. If blockFor is larger 
than the number of replicas contacted before, we will add more nodes to apply 
the read-repair-mutation. For normal cases, blockFor should be same as the 
number of nodes contacted before. However, if we are adding a node in the same 
DC, the blockFor will be blockFor + pending which requires one more node to be 
repaired. And this "one more node" can be any node that has not been contacted 
before. As mentioned, the while the latch is initialized with number of nodes 
that the read-repair-mutations have been sent to but only count down when 
getting response from same DC. The coordinator node will get timeout 100% if a 
cross DC node is selected to perform the read-repair-mutation.

> local consistencies may get timeout if blocking read repair is sending the 
> read repair mutation to other DC 
> ------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-19120
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19120
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Runtian Liu
>            Priority: Normal
>         Attachments: image-2023-11-29-15-26-08-056.png, signature.asc
>
>
> For a two DCs cluster setup. When a new node is being added to DC1, for 
> blocking read repair triggered by local_quorum in DC1, it will require to 
> send read repair mutation to an extra node(1)(2). The selector for read 
> repair may select *ANY* node that has not been contacted before(3) instead of 
> selecting the DC1 nodes. If a node from DC2 is selected, this will cause 100% 
> timeout because of the bug described below:
> When we initialized the latch(4) for blocking read repair, the shouldBlockOn 
> function will only return true for local nodes(5), the blockFor value will be 
> reduced if a local node doesn't require repair(6). The blockFor is same as 
> the number of read repair mutation sent out. But when the coordinator node 
> receives the response from the target nodes, the latch only count down for 
> nodes in same DC(7). The latch will wait till timeout and the read request 
> will timeout.
> This can be reproduced if you have a constant load on a 3 + 3 cluster when 
> adding a node. If you have someway to trigger blocking read repair(maybe by 
> adding load using stress tool). If you use local_quorum consistency with a 
> constant read after write load in the same DC that you are adding node. You 
> will see read timeout issue from time to time because of the bug described 
> above
>  
> I think for read repair when selecting the extra node to do repair, we should 
> prefer local nodes than the nodes from other region. Also, we need to fix the 
> latch part so even if we send mutation to the nodes in other DC, we don't get 
> a timeout.
> (1)[https://github.com/apache/cassandra/blob/cassandra-4.0.11/src/java/org/apache/cassandra/locator/ReplicaPlans.java#L455]
> (2)[https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/ConsistencyLevel.java#L183]
> (3)[https://github.com/apache/cassandra/blob/cassandra-4.0.11/src/java/org/apache/cassandra/locator/ReplicaPlans.java#L458]
> (4)[https://github.com/apache/cassandra/blob/cassandra-4.0.11/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java#L96]
> (5)[https://github.com/apache/cassandra/blob/cassandra-4.0.11/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java#L71]
> (6)[https://github.com/apache/cassandra/blob/cassandra-4.0.11/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java#L88]
> (7)[https://github.com/apache/cassandra/blob/cassandra-4.0.11/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java#L113]
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to