You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Li Zou (JIRA)" <ji...@apache.org> on 2013/09/12 00:19:53 UTC

[jira] [Commented] (CASSANDRA-4705) Speculative execution for reads / eager retries

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

Li Zou commented on CASSANDRA-4705:
-----------------------------------

Hello [~vijay2win@yahoo.com] and [~jbellis],

My company is adopting the Cassandra db technology and we are very much interested in this new feature of "Speculative Execution for Reads", as it could potentially help reduce the overall outage window to the sub-second range upon the failure of any one of Cassandra nodes in a data center.

I have recently tested the "Speculative Execution for Reads" using Cassandra 2.0.0-rc2 and I have not seen the expected results. In my tests, I have already excluded the possible effects from the client (connection pool) side. Not sure what is still missed out in my tests.

Here is my setup for the tests. One data center of four Cassandra nodes is configured on machine A, B, C and D with each physical machine has one Cassandra node running on it. My testing app (Cassandra client) is running on machine A and it is configured to connect to Cassandra nodes A, B, and C only. In other words, my testing app will never connect to Cassandra node D.

* Replication Factor (RF) is set to 3
* Client requested Consistency Level (CL) is set to CL_TWO

For Cassandra 1.2.4, upon the failure of Cassandra node D (via "kill -9 <pid>" or "kill -s SIGKILL <pid>"), there will be an outage window for around 20 seconds with zero transactions. The outage ends as soon as the gossip protocol detects the failure of Cassandra node D and marks it down.

For Cassandra 2.0.0-rc2, database tables are configured with "speculative_retry" proper values (such as 'ALWAYS', '10 ms', '100 ms', '80 percentile', 'NONE'). No expected results are observed in the tests. The testing results are quite similar to those observed for Cassandra 1.2.4. That is, upon the failure of Cassandra node D, there will be an outage window for around 20 seconds with zero transactions. The outage ends as soon as the gossip protocol detects the failure of Cassandra node D and marks it down. The tested values for _speculative_retry_ against database table are listed below:

* ALWAYS
* 10 ms
* 100 ms
* 80 percentile
* NONE

In my tests, I also checked the JMX stats of Cassandra node A for the speculative_retry for each table. What I observed really surprised me. For an instance, the speculative_retry (for each table) is set to '20 ms'. During normal operations with all of four Cassandra nodes up, the "SpeculativeRetry" count went up occasionally. However, during the 20-second outage window, the "SpeculativeRetry" count somehow never went up for three tests in a row. On the contrary, I would expect to see lots of speculative retries were executed (say on Cassandra node A) during the 20-second outage window.

Some notes for the "kill" signals. I tested if the Cassandra node D is killed using SIGTERM (for Cassandra 1.2.4) and SIGINT / SIGTERM (for Cassandra 2.0.0-rc2), there will be no observed outage in second range, as the Cassandra node D does the orderly shutdown and the gossip announces the node "down". From my testing app point of view, everything is going very well.

But if the Cassandra node D is killed using SIGKILL, there will be an outage with zero transactions. I guess this might be a TCP socket related issue. I often observed TCP socket "CLOSE_WAIT" on the passive close side (i.e. Cassandra node A, B, and C) and TCP socket TIME_WAIT on the active close side (Cassandra node D).

                
> Speculative execution for reads / eager retries
> -----------------------------------------------
>
>                 Key: CASSANDRA-4705
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4705
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Vijay
>            Assignee: Vijay
>             Fix For: 2.0 beta 1
>
>         Attachments: 0001-CASSANDRA-4705.patch, 0001-CASSANDRA-4705-v2.patch, 0001-CASSANDRA-4705-v3.patch, 0001-Refactor-to-introduce-AbstractReadExecutor.patch, 0002-Add-Speculative-execution-for-Row-Reads.patch
>
>
> When read_repair is not 1.0, we send the request to one node for some of the requests. When a node goes down or when a node is too busy the client has to wait for the timeout before it can retry. 
> It would be nice to watch for latency and execute an additional request to a different node, if the response is not received within average/99% of the response times recorded in the past.
> CASSANDRA-2540 might be able to solve the variance when read_repair is set to 1.0
> 1) May be we need to use metrics-core to record various Percentiles
> 2) Modify ReadCallback.get to execute additional request speculatively.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira