[ https://issues.apache.org/jira/browse/CASSANDRA-19120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17818479#comment-17818479 ]
Stefan Miklosovic commented on CASSANDRA-19120: ----------------------------------------------- [CASSANDRA-19120-4.0|https://github.com/instaclustr/cassandra/tree/CASSANDRA-19120-4.0] {noformat} java11_pre-commit_tests java11_separate_tests java8_pre-commit_tests ✓ j8_build 5m 37s ✓ j8_cqlsh-dtests-py2-no-vnodes 8m 7s ✓ j8_cqlsh-dtests-py2-with-vnodes 6m 56s ✓ j8_cqlsh_dtests_py3 8m 59s ✓ j8_cqlsh_dtests_py311 7m 35s ✓ j8_cqlsh_dtests_py311_vnode 7m 11s ✓ j8_cqlsh_dtests_py38 6m 32s ✓ j8_cqlsh_dtests_py38_vnode 6m 48s ✓ j8_cqlsh_dtests_py3_vnode 7m 29s ✓ j8_cqlshlib_tests 8m 15s ✓ j8_dtests 32m 15s ✓ j8_dtests_vnode 33m 48s ✓ j8_jvm_dtests 13m 25s ✓ j11_dtests_vnode 35m 2s ✓ j11_dtests 32m 17s ✓ j11_cqlsh_dtests_py3_vnode 5m 18s ✓ j11_cqlsh_dtests_py38_vnode 5m 24s ✓ j11_cqlsh_dtests_py38 5m 50s ✓ j11_cqlsh_dtests_py311_vnode 5m 27s ✓ j11_cqlsh_dtests_py311 5m 50s ✓ j11_cqlsh_dtests_py3 5m 25s ✓ j11_cqlsh-dtests-py2-with-vnodes 5m 25s ✓ j11_cqlsh-dtests-py2-no-vnodes 6m 43s ✕ j8_unit_tests 7m 46s org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace ✕ j8_utests_system_keyspace_directory 9m 38s org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace ✕ j11_unit_tests 7m 56s org.apache.cassandra.index.sasi.SASICQLTest testPagingWithClustering org.apache.cassandra.cql3.MemtableSizeTest testTruncationReleasesLogSpace java8_separate_tests {noformat} [java11_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/3892/workflows/26904ec4-ba86-4c8e-ba07-5da871b6c5a1] [java11_separate_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/3892/workflows/c35e2b54-b6f1-4283-a9f4-6e776517a721] [java8_pre-commit_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/3892/workflows/d7e43174-bc76-42da-93a9-e43eb65218ae] [java8_separate_tests|https://app.circleci.com/pipelines/github/instaclustr/cassandra/3892/workflows/1168aafe-d453-4b54-b688-5bf3d9d5624a] > 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