[ https://issues.apache.org/jira/browse/CASSANDRA-20291?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Manish Khandelwal reassigned CASSANDRA-20291: --------------------------------------------- Assignee: Manish Khandelwal > Batch queries timeout when private IP of a node fails in multi-dc Cassandra > setup > --------------------------------------------------------------------------------- > > Key: CASSANDRA-20291 > URL: https://issues.apache.org/jira/browse/CASSANDRA-20291 > Project: Apache Cassandra > Issue Type: Bug > Components: Consistency/Batch Log > Reporter: Manish Khandelwal > Assignee: Manish Khandelwal > Priority: Normal > Time Spent: 10m > Remaining Estimate: 0h > > I have a Cassandra 4.1.4 cluster with two data centers, once cluster with 3 > nodes and other cluster with 2 nodes. The configuration is: listen_address = > private IP, broadcast_address = public IP, listen_on_broadcast_address = > true, prefer_local = true > {code:java} > Status=Up/Down > |/ State=Normal/Leaving/Joining/Moving > -- Address Load Tokens Owns Host ID > Rack > UN X.Y.Z.Node1 2.4 GiB 256 ? 766ae19b-3c87-47f1-8b85-23085b69b202 > RAC1 > UN X.Y.Z.Node3 2.51 GiB 256 ? > 2cbdcb3d-5344-46f3-9b9e-283888d92a4b RAC1 > UN X.Y.Z.Node2 2.47 GiB 256 ? 30756549-255d-4ec2-b15b-071c3983edc0 > RAC1 > > Datacenter: DC2 > =============== > Status=Up/Down > |/ State=Normal/Leaving/Joining/Moving > -- Address Load Tokens Owns Host ID > Rack > UN X.Y.Z.Node4 2.5 GiB 256 ? > 38c84625-fbd0-403f-9862-48d3e8d30400 RAC1 > UN X.Y.Z.Node5 2.54 GiB 256 ? > 9af001f9-4226-4958-9cc9-efdc52ca9e1d RAC1{code} > *Issue Observed:* > * We execute a multi-partition batch query with LOCAL_QUORUM consistency, > and it succeeds. > * We bring down the private IP of one node in the DC where queries are > executed. > * Batch queries start timing out, but simple INSERT and SELECT queries work > fine. > * Stopping /Restarting the affected node (node where private IP was bought > down) resolves the issue, and batch queries succeed again. > *Analysis So far* > The node is not marked down in nodetool status but is unreachable in nodetool > describecluster. From the Cassandra source code, it looks like the > coordinator picks two other nodes for writing batch logs. The failed node > (private IP down) gets selected for batch logs, but it never responds, > causing the timeout. This is evident from the TRACE logs as well > For example in the above topology, for DC1, we bought down the private > interface of node with IP X.Y.Z.Node1, and fired a query on node with IP > X.Y.Z.Node3. following were the logs at the coordinator that is X.Y.Z.Node3 > {code:java} > TRACE [Native-Transport-Requests-1] 2025-02-05 06:59:33,534 > StorageProxy.java:1280 - Sending batchlog store request > 5f4624d0-e386-11ef-9bab-e9a6cd1b4f50 to > Full(/X.Y.Z.Node2:9700,(-9223372036854775808,-9223372036854775808]) for 3 > mutations > TRACE [Native-Transport-Requests-1] 2025-02-05 06:59:33,534 > StorageProxy.java:1280 - Sending batchlog store request > 5f4624d0-e386-11ef-9bab-e9a6cd1b4f50 to > Full(/X.Y.Z.Node1:9700,(-9223372036854775808,-9223372036854775808]) for 3 > mutations > WARN [Messaging-EventLoop-3-11] 2025-02-05 06:59:33,608 > NoSpamLogger.java:108 - > host3/X.Y.Z.Node3:9700>host1/X.Y.Z.Node1:9700-SMALL_MESSAGES[no-channel] > dropping message of type BATCH_STORE_REQ whose timeout expired before > reaching the network{code} > In the client logs we get > {code:java} > Caused by: com.datastax.driver.core.exceptions.WriteTimeoutException: > Cassandra timeout during BATCH_LOG write query at consistency TWO (2 replica > were required but only 1 acknowledged the write){code} > If we restart or stop the node (X.Y.Z.Node1) where private interface was > down, Nodetool status of other nodes is updated and they register it as down. > Now if we fire the batch query, only one node gets selected for batchlog > which is available and hint is saved for node with downed private interface. > Evident from the logs below > {code:java} > TRACE [Native-Transport-Requests-1] 2025-02-05 07:38:23,863 > StorageProxy.java:1280 - Sending batchlog store request > cc424870-e38b-11ef-9bab-e9a6cd1b4f50 to > Full(/X.Y.Z.Node2:9700,(-9223372036854775808,-9223372036854775808]) for 3 > mutations > TRACE [MutationStage-2] 2025-02-05 07:38:23,867 StorageProxy.java:2730 - > Adding hints for [/X.Y.Z.Node1:9700]{code} > > I think the issue is that in multi DC cluster with both public and private > interface enabled, if private interface of node goes down and batch queries > are fired. Failure detector mechanism does not update the status of the node > as unreachable and that node appears live. During batch query execution when > replicas are filtered for batchlog, the node with failed private interface > gets selected but request is never sent to it. This request results in > WriteTimeoutExeption with writetype "BATCH_LOG" > > -- 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