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

Joseph Lynch commented on CASSANDRA-14459:
------------------------------------------

I've uploaded a patch which I think goes in the right direction. The Gossiper 
only sends an extra {{PingMessage}} if the latency subscribers need latency 
information. The DES only needs latency information if it has 2 or fewer 
measurements in the preceding 10 minute period. This would mean a maximum of 
one additional message to a host per 10 minute period under the assumption of 
no traffic (may want this to be Echo if we want to support mixed mode clusters, 
I'm somewhat ambivalent if it's an {{EchoMessage}} coming in on 
{{REQUEST_RESPONSE}} or a {{PingMessage}} coming in on READ, but it sounds like 
you dislike the latter). This avoids extra traffic to seeds as well as most 
hosts under most conditions.

I think at this point I'm pretty convinced that we need some active 
healthchecks _somewhere_, and at the start of this I was hoping to piggy back 
on the existing ones in the Gossiper. If the {{Ack}} used {{sendReply}} then I 
could get away with no additional traffic by adding callback handling in 
{{GossipDigestAckVerbHandler}} but I don't see how to do that in a backwards 
compatible way (where old hosts are not using reply leading to a 10s 
measurement). I can also take a different approach and have a task that runs 
separately from gossiper and sends out {{PingMessages}} from a different thread 
all together. I could even unify that with the {{StartupConnectivityChecker}} 
to form a {{HealthcheckService}}. This would be consistent with how e.g. load 
balancers like HAProxy work (where they have background healthchecks that run 
all at once on startup and then periodically after that).

> 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
>             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