You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Joseph Lynch (JIRA)" <ji...@apache.org> on 2019/02/23 04:03:00 UTC

[jira] [Commented] (CASSANDRA-14459) DynamicEndpointSnitch should never prefer latent nodes

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

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

Alright, finally got around to writing those messaging service tests you asked for :), although if you're ok with it I'd prefer to refactor the methods to be slightly more unit testable and not even bother doing an e2e on the latency probe components as in my opinion the tests of that are just gnarly (and while I think that they won't flake because I ran them 2k times locally to check for race conditions, those kinds of multi-threading tests just worry me).

Some notable changes in behavior since the last patch:
 * The StartupConnectivityChecker now sends a second ping that counts after all the first round ones come back. While I was testing this patch with CCM I noticed that the first Ping is not very good quality data (since it includes the handshake).
 * I refactored {{ScheduledExecutors}} so that we can't leak threads out of random STPE's (e.g. DES will not break the jvm dtests because it won't leak a thread). If you don't like this let me know and I can change the close method to be called from {{Instance.java}}
 * DES now has microsecond resolution instead of millisecond, I noticed that in my CCM clusters all the nodes have latency 0 (less than a millisecond) which also appears true on many of our production clusters.
 * When the DES lacks latency information about a peer we defer to the subsnitch instead of scoring it zero (which will essentially prefer it over everything), we then rely on the latency probes to find out where it belongs
 * Refactored and payed down a bunch of tech debt with a lot of tests

On my side I'd like to give this more testing using CCM and on a real cluster with real cross datacenter latency to ensure the probes are working as expected, and I'd like to refactor the unit tests of the probe logic so that we don't actually have to hook into the MessagingService (which leads to a bunch of semi-gross test code imo)

> DynamicEndpointSnitch should never prefer latent nodes
> ------------------------------------------------------
>
>                 Key: CASSANDRA-14459
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14459
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Legacy/Coordination
>            Reporter: Joseph Lynch
>            Assignee: Joseph Lynch
>            Priority: Minor
>              Labels: 4.0-feature-freeze-review-requested, pull-request-available
>             Fix For: 4.x
>
>          Time Spent: 22h 50m
>  Remaining Estimate: 0h
>
> 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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org