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] [Comment Edited] (CASSANDRA-14459) DynamicEndpointSnitch should never prefer latent nodes
Date Fri, 08 Jun 2018 23:37:00 GMT

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

Joseph Lynch edited comment on CASSANDRA-14459 at 6/8/18 11:36 PM:
-------------------------------------------------------------------

I think that we need some way to get latency measurements for hosts that have been excluded
from traffic due to high minimums. For example if during the initial {{PingMessages}} a local
DC host gets a very high measurement (e.g. 100ms) we will never send traffic to it ever.
My understanding is that's why we reset in the first place.

I'll try to come up with a solution that doesn't involve additional traffic.


was (Author: jolynch):
I think that we need some way to get latency measurements for hosts that have been excluded
from traffic due to high minimums. For example if during the initial {{PingMessages}} a local
DC host gets a very high measurement (e.g. 100ms) we will never send traffic to it ever.
My understanding is that's why we reset in the first place.

I'll work on a feedback mechanism for the {{DES}} to ask for latency probes (which I guess
would be best implemented as {{PingMessages}} since you're concerned about {{EchoMessages}}).
I see possible two designs: one where I send the probes directly from the {{DES}} or I can
have a method expressing the desire for probes that propagates up to e.g. the {{MessagingService}}.
Are there better options?

> 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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message