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

Blake Eggleston commented on CASSANDRA-19120:
---------------------------------------------

Thanks for your time on this Runtian and Stefan. Feedback is below, 

So after the removal of read_repair_chance/dclocal_read_repair_chance in 
CASSANDRA-13910, there should never be a case where a DC local read or read 
repairs are talking to remote DCs.

First, replicas from remote dcs should not be making it into the contact list. 
If we weren’t correctly disregarding their acks in BlockingReadRepair, we’d 
have a violation of monotonic read guarantees. So I’d suggest that we filter 
remote DC replicas out when constructing the initial replica plan for the read 
repair

Second, if we don’t have to worry about remote DCs making it into 
BlockingPartitionRepair, we can remove the shouldBlockOn logic from that class, 
though we should add something to the effect of 
`Preconditions.checkState(!consistencyLevel.isDatacenterLocal() ||  
InOurDcTester.{_}endpoints{_}().contains(participant))` in the ctor as a sanity 
check. This would also mean the removal of the blockFor decrementing code you 
added.

Third, in some situations (ie: when there is a pending node), the 
pendingRepairMap will always have fewer items than blockFor, which will mean 
the repair will always speculate. To avoid this, I’d proactively add repairs to 
the pendingRepair map when this is the case, though it will mean a little 
reworking of maybeSendAdditionalWrites.

Let me know what you think. Also, [~samt] does all that seem reasonable to you?

> 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
>          Components: Consistency/Repair
>            Reporter: Runtian Liu
>            Assignee: Runtian Liu
>            Priority: Normal
>             Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
>         Attachments: image-2023-11-29-15-26-08-056.png, signature.asc
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> 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