You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jason Brown (JIRA)" <ji...@apache.org> on 2018/06/05 13:40: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=16501791#comment-16501791 ] 

Jason Brown commented on CASSANDRA-14459:
-----------------------------------------

Took a quick look, and on the whole I like where this goes. The biggest problem I have is {{DynamicEndpointSnitch#reset(boolean)}} gets a snapshot of the existing {{reservoir}}. Unfortunately, generating that that snapshot is *very* heavyweight (copies a bunch of data, creates a bunch of garbage), just to get one value and then throw it all away. You should consider a different mechanism for getting the min value. Perhaps instead of {{DynamicEndpointSnitch#samples}} being defined as {{ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir>}}, maybe the value class is something like:

{code}
static class Holder
{
    private final ExponentiallyDecayingReservoir res;
    private volatile long minValue;

    void update(long val)
    {
        res.update(...)

        // It's probably ok if there's a race on minValue. better a small/irrelevant race than any real coordination
        if (val < minValue)
            minValue = val;
    }
}
{code}

Suit to taste, if you find this useful.

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