cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Joseph Lynch (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-14459) DynamicEndpointSnitch should never prefer latent nodes
Date Tue, 05 Jun 2018 21:59:00 GMT

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

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

[~sumanth.pasupuleti]
Regarding lowest or latest latency, if I do additional bookkeeping either is viable, but without
additional bookkeeping it becomes slightly complex to do the latest value. Either strategy
would achieve allowing temporarily slow nodes to receive traffic again, but I worry that resetting
a host to a recent large value would temporarily remove it from replica consideration entirely
until the next reset/EchoMessage. I'll fix the enum documentation.

[~jasobrown] Thanks for taking a look! I figured since {{getSnapshot}} is called every 100ms
in {{updateScore}}, calling it once more every 10 minutes wouldn't be a huge deal? Separate
bookeeping is certainly possible if you prefer, I just figured it was better to minimize the
complexity of the change. While testing this I think that I need to hook in somewhere to the
3-way gossiping messaging since {{EchoMessages}} only get sent if the node is alive in gossip
but not in the local failure detector. I'll work on a test that a round of gossip has latency
numbers added and then fix it :-)



> 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


Mime
View raw message