[ https://issues.apache.org/jira/browse/CASSANDRA-19120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17791820#comment-17791820 ]
Stefan Miklosovic commented on CASSANDRA-19120: ----------------------------------------------- I see ... well, I dont know ... something like this? https://github.com/apache/cassandra/pull/2953/commits/06069fac4f73882115c325722c2cc7bb9b779080 I dont see anything wrong with that special case but we are changing the behavior here and we should be careful. Let's test this idea first and then we can invite another committer to evaluate it as well. > 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 > > 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