You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Cameron Zemek (JIRA)" <ji...@apache.org> on 2018/09/11 05:50:00 UTC

[jira] [Commented] (CASSANDRA-14715) Read repairs can result in bogus timeout errors to the client

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

Cameron Zemek commented on CASSANDRA-14715:
-------------------------------------------

I should also point out this means that the timeouts don't get captured in the read timeout metric either due to the timeout occuring on the close for the PartitionIterator returned by StorageProxy:read where the timeouts are caught (see readRegular)

> Read repairs can result in bogus timeout errors to the client
> -------------------------------------------------------------
>
>                 Key: CASSANDRA-14715
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14715
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local Write-Read Paths
>            Reporter: Cameron Zemek
>            Priority: Minor
>
> In RepairMergeListener:close() it does the following:
>  
> {code:java}
> try
> {
>     FBUtilities.waitOnFutures(repairResults, DatabaseDescriptor.getWriteRpcTimeout());
> }
> catch (TimeoutException ex)
> {
>     // We got all responses, but timed out while repairing
>     int blockFor = consistency.blockFor(keyspace);
>     if (Tracing.isTracing())
>         Tracing.trace("Timed out while read-repairing after receiving all {} data and digest responses", blockFor);
>     else
>         logger.debug("Timeout while read-repairing after receiving all {} data and digest responses", blockFor);
>     throw new ReadTimeoutException(consistency, blockFor-1, blockFor, true);
> }
> {code}
> This propagates up and gets sent to the client and we have customers get confused cause they see timeouts for CL ALL requiring ALL replicas even though they have read_repair_chance = 0 and using a LOCAL_* CL.
> At minimum I suggest instead of using the consistency level of DataResolver (which is always ALL with read repairs) for the timeout it instead use repairResults.size(). That is blockFor = repairResults.size() . But saying it received _blockFor - 1_ is bogus still. Fixing that would require more changes. I was thinking maybe like so:
>  
> {code:java}
> public static void waitOnFutures(List<AsyncOneResponse> results, long ms, MutableInt counter) throws TimeoutException
> {
>     for (AsyncOneResponse result : results)
>     {
>         result.get(ms, TimeUnit.MILLISECONDS);
>         counter.increment();
>     }
> }
> {code}
>  
>  
>  
> Likewise in SinglePartitionReadLifecycle:maybeAwaitFullDataRead() it says _blockFor - 1_ for how many were received, which is also bogus.
>  
> Steps used to reproduce was modify RepairMergeListener:close() to always throw timeout exception.  With schema:
> {noformat}
> CREATE KEYSPACE weather WITH replication = {'class': 'NetworkTopologyStrategy', 'dc1': '3', 'dc2': '3'}  AND durable_writes = true;
> CREATE TABLE weather.city (
> cityid int PRIMARY KEY,
> name text
> ) WITH bloom_filter_fp_chance = 0.01
> AND dclocal_read_repair_chance = 0.0
> AND read_repair_chance = 0.0
> AND speculative_retry = 'NONE';
> {noformat}
> Then using the following steps:
>  # ccm node1 cqlsh
>  # INSERT INTO weather.city(cityid, name) VALUES (1, 'Canberra');
>  # exit;
>  # ccm node1 flush
>  # ccm node1 stop
>  # rm -rf ~/.ccm/test_repair/node1/data0/weather/city-ff2fade0b18d11e8b1cd097acbab1e3d/mc-1-big-* # remove the sstable with the insert
>  # ccm node1 start
>  # ccm node1 cqlsh
>  # CONSISTENCY LOCAL_QUORUM;
>  # select * from weather.city where cityid = 1;
> You get result of:
> {noformat}
> ReadTimeout: Error from server: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message="Operation timed out - received only 5 responses." info={'received_responses': 5, 'required_responses': 6, 'consistency': 'ALL'}{noformat}
> But was expecting:
> {noformat}
> ReadTimeout: Error from server: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message="Operation timed out - received only 1 responses." info={'received_responses': 1, 'required_responses': 2, 'consistency': 'LOCAL_QUORUM'}{noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org