[ https://issues.apache.org/jira/browse/CASSANDRA-14459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16506243#comment-16506243 ]
Jason Brown commented on CASSANDRA-14459: ----------------------------------------- tbqh, I am -1 on sending an {{EchoMessage}} on every gossip round. This increases the gossip traffic by 66% [1], if not 100%, adds more processing demands to the single-threaded Gossip stage, and will not even give you realistic latency data (except, possibly, a rudimentary floor latency, but that assumes a small cluster that is rather quiescent). Seed nodes would also bear a lot of this additional traffic. If we don't have any latency data in DES for a host, it's because we have not communicated meaningfully with it (as far as latency numbers go). I am totally fine with that, and we don't need to goose the traffic to get latencies for a node which we don't talk to. Your original patch was probably good enough to start a proper review, as I believe this behavior is a worthwhile addition. [1] Currently there's 2-3 msgs per gossip round (Ack2 is optional), EchoMsg + response adds two more. > 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