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

Ariel Weisberg commented on CASSANDRA-14459:
--------------------------------------------

I'll try to summarize what I think our current plan is.
 * Add active probing to the snitch
 * Remove resetting of reservoir
 * Deprecate reset interval
 * In the map in dynamic snitch wrap the reservoir so we can store extra 
boolean flags
 * Store a volatile boolean flag for whether a score has been requested, check 
if the flag is not set, and set it when checking the score
 * Store a volatile boolean flag for whether a real latency sample has been 
received, check if the flag is not set, and set it when receiving a real 
latency sample
 * Only send probes to nodes that have been involved in a query (score 
requested) since the last time a probe was sent && have not received a real 
latency sample since we last checked.
 * Add tunable configuring probes sent by each node per second (float?), or 
alternatively just seconds between probes. Probes could be sent by a dedicated 
thread blocking on a rate limiter vs. scheduling it with the dynamic snitch 
update thread. I wonder if the snitch thread ever becomes busy since it updates 
scores every 100 milliseconds which on a large cluster seems like it could be a 
lot.
 * Add EMA as a pluggable option, but keep the current default
 * Thread checking for active probes to send iterates the map and generates a 
list of all nodes needing a probe. As it iterates the list it unsets the flags. 
The list is then shuffled. It then sends the probes either rate limited or on a 
schedule executor.
 * Stop using echo message and gossip stage as samples.
 * When active probing send the probe over the large message channel and small 
message channel (response should be routed similarly) and the score will be the 
worse of the two for that active probe so we can detect if either socket has 
issues.

 

> DynamicEndpointSnitch should never prefer latent nodes
> ------------------------------------------------------
>
>                 Key: CASSANDRA-14459
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14459
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Coordination
>            Reporter: Joseph Lynch
>            Assignee: Joseph Lynch
>            Priority: Minor
>              Labels: 4.0-feature-freeze-review-requested
>             Fix For: 4.x
>
>
> The DynamicEndpointSnitch has two unfortunate behaviors that allow it to 
> provide latent hosts as replicas:
>  # Loses all latency information when Cassandra restarts
>  # Clears latency information entirely every ten minutes (by default), 
> allowing global queries to be routed to _other datacenters_ (and local 
> queries cross racks/azs)
> This means that the first few queries after restart/reset could be quite slow 
> compared to average latencies. I propose we solve this by resetting to the 
> minimum observed latency instead of completely clearing the samples and 
> extending the {{isLatencyForSnitch}} idea to a three state variable instead 
> of two, in particular {{YES}}, {{NO}}, {{MAYBE}}. This extension allows 
> {{EchoMessages}} and {{PingMessages}} to send {{MAYBE}} indicating that the 
> DS should use those measurements if it only has one or fewer samples for a 
> host. This fixes both problems because on process restart we send out 
> {{PingMessages}} / {{EchoMessages}} as part of startup, and we would reset to 
> effectively the RTT of the hosts (also at that point normal gossip 
> {{EchoMessages}} have an opportunity to add an additional latency 
> measurement).
> This strategy also nicely deals with the "a host got slow but now it's fine" 
> problem that the DS resets were (afaik) designed to stop because the 
> {{EchoMessage}} ping latency will count only after the reset for that host. 
> Ping latency is a more reasonable lower bound on host latency (as opposed to 
> status quo of zero).



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

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

Reply via email to