Ah. The lack of page cache hits after compaction makes sense. But I don't think 
the drastic effect it appears to have is expected. Do you have an idea of how 
much slower local reads get ?

If you are selecting coordinators based on token ranges the DS is not as much. 
It still has some utility as the Digest reads will be happening on other nodes 
and it should help with selecting them. 

Thanks for the extra info. 

Aaron

-----------------
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com

On 25/05/2012, at 1:24 AM, Viktor Jevdokimov wrote:

> All data is in the page cache. No repairs. Compactions not hitting disk for 
> read. CPU <50%. ParNew GC <100 ms in average.
>  
> After one compaction completes, new sstable is not in page cache, there may 
> be a disk usage spike before data is cached, so local reads gets slower for a 
> moment, comparing with other nodes. Redirecting almost all requests to other 
> nodes finally ends up with a huge latency spike almost on all nodes, 
> especially when ParNew GC may spike on one node (>200ms). We call it “cluster 
> hiccup”, when incoming and outgoing network traffic drops for a moment.
>  
> And such hiccups happens several times an hour, few seconds long. Playing 
> with badness threshold did not gave a lot better results, but turning DS off 
> completely fixed all problems with latencies, node spikes, cluster hiccups 
> and network traffic drops.
>  
> In our case, our client is selecting endpoints for a key by calculating a 
> token, so we always hit a replica.
>  
>  
> 
> 
> Best regards / Pagarbiai
> Viktor Jevdokimov
> Senior Developer
> 
> Email: viktor.jevdoki...@adform.com
> Phone: +370 5 212 3063, Fax +370 5 261 0453
> J. Jasinskio 16C, LT-01112 Vilnius, Lithuania
> Follow us on Twitter: @adforminsider
> What is Adform: watch this short video
> <signature-logo18be.png>
> 
> Disclaimer: The information contained in this message and attachments is 
> intended solely for the attention and use of the named addressee and may be 
> confidential. If you are not the intended recipient, you are reminded that 
> the information remains the property of the sender. You must not use, 
> disclose, distribute, copy, print or rely on this e-mail. If you have 
> received this message in error, please contact the sender immediately and 
> irrevocably delete this message and any copies.
> 
> From: aaron morton [mailto:aa...@thelastpickle.com] 
> Sent: Thursday, May 24, 2012 13:00
> To: user@cassandra.apache.org
> Subject: Re: Replication factor
>  
> Your experience is when using CL ONE the Dynamic Snitch is moving local reads 
> off to other nodes and this is causing spikes in read latency ? 
>  
> Did you notice what was happening on the node for the DS to think it was so 
> slow ? Was compaction or repair going on ? 
>  
> Have you played with the badness threshold 
> https://github.com/apache/cassandra/blob/trunk/conf/cassandra.yaml#L472 ? 
>  
> Cheers
>  
>  
> -----------------
> Aaron Morton
> Freelance Developer
> @aaronmorton
> http://www.thelastpickle.com
>  
> On 24/05/2012, at 5:28 PM, Viktor Jevdokimov wrote:
> 
> 
> Depends on use case. For ours we have another experience and statistics, when 
> turning dynamic snitch off makes overall latency and spikes much, much lower.
>  
>  
>  
> 
> Best regards / Pagarbiai
> Viktor Jevdokimov
> Senior Developer
>  
> Email: viktor.jevdoki...@adform.com
> Phone: +370 5 212 3063, Fax +370 5 261 0453
> J. Jasinskio 16C, LT-01112 Vilnius, Lithuania
> Follow us on Twitter: @adforminsider
> What is Adform: watch this short video
> <signature-logo29.png>
> 
> Disclaimer: The information contained in this message and attachments is 
> intended solely for the attention and use of the named addressee and may be 
> confidential. If you are not the intended recipient, you are reminded that 
> the information remains the property of the sender. You must not use, 
> disclose, distribute, copy, print or rely on this e-mail. If you have 
> received this message in error, please contact the sender immediately and 
> irrevocably delete this message and any copies.
>  
> From: Brandon Williams [mailto:dri...@gmail.com] 
> Sent: Thursday, May 24, 2012 02:35
> To: user@cassandra.apache.org
> Subject: Re: Replication factor
>  
> On Wed, May 23, 2012 at 5:51 AM, Viktor Jevdokimov 
> <viktor.jevdoki...@adform.com> wrote:
> > When RF == number of nodes, and you read at CL ONE you will always be 
> > reading locally.
> “always be reading locally” – only if Dynamic Snitch is “off”. With dynamic 
> snitch “on” request may be redirected to other node, which may introduce 
> latency spikes.
>  
> Actually it's preventing spikes, since if it won't read locally that means 
> the local replica is in worse shape than the rest (compacting, repairing, 
> etc.)
>  
> -Brandon 

Reply via email to