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

Jonathan Ellis commented on CASSANDRA-4705:
-------------------------------------------

Okay, let's leave UpdateSampleLatencies alone (although as style I'd prefer to 
inline it as an anyonymous Runnable).

Thinking more about the core functionality:

- a RetryType of "one pre-emptive redundant data read" would be a useful 
alternative to ALL.  (If supporting both makes things more complex, I would 
vote for just supporting the single extra read.)  E.g., for a CL.ONE read it 
would perform two data reads; for CL.QUORUM it would perform two data reads and 
a digest read.  Put another way, it would do the same exta data read 
Xpercentile would, but it would do it ahead of the threshold timeout.
- ISTM we should continue to use RDR for normal (non-RR) SR reads, and just 
accept the first data reply that comes back without comparing it to others.  
This makes the most sense to me semantically, and keeps CL.ONE reads 
lightweight.
- I think it's incorrect (again, in the non-RR case) to perform a data read 
against the same host we sent a digest read to.  Consider CL.QUORUM: I send a 
data read to replica X and a digest to replica Y.  X is slow to respond.  Doing 
a data read to Y won't help, since I need both to meet my CL.  I have to do my 
SR read to replica Z, if one exists and is alive.
- We should probably extend this to doing extra digest reads for CL > ONE, when 
we get the data read back quickly but the digest read is slow.
- SR + RR is the tricky part... this is where SR could result in data and 
digests from the same host.  So ideally, we want the ability to compare 
(potentially) multiple data reads, *and* multiple digests, *and* track the 
source for CL purposes, which neither RDR nor RRR is equipped to do.  Perhaps 
we should just force all reads to data reads for SR + RR [or even for all RR 
reads], to simplify this.

Finally,
- millis may be too coarse a grain here, especially for Custom settings.  
Currently an in-memory read will typically be under 2ms and it's quite possible 
we can get that down to 1 if we can purge some of the latency between stages.  
Might as well use micros since Timer gives it to us for free, right?
                
> Speculative execution for CL_ONE
> --------------------------------
>
>                 Key: CASSANDRA-4705
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4705
>             Project: Cassandra
>          Issue Type: Improvement
>    Affects Versions: 1.2.0
>            Reporter: Vijay
>            Assignee: Vijay
>            Priority: Minor
>         Attachments: 0001-CASSANDRA-4705.patch, 0001-CASSANDRA-4705-v2.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

Reply via email to