[ https://issues.apache.org/jira/browse/CASSANDRA-14459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16661557#comment-16661557 ]
Joseph Lynch commented on CASSANDRA-14459: ------------------------------------------ [~aweisberg] Ok I've pushed changes to the branch which I believe address all your comments (including recording only the maximum latencies). I also finished the pluggability work so that we can swap out snitches live as well as implementing the ~20 lines needed to provide a backwards compatible option. To make the snitch more or less equivalent to the old snitch behavior you can do either live reconfiguration: {noformat} $ jmxterm Welcome to JMX terminal. Type "help" for available commands. $>open localhost:7199 #Connection to localhost:7199 is opened $>bean org.apache.cassandra.db:type=StorageService #bean is set to org.apache.cassandra.db:type=StorageService $>run updateSnitch -t string,string,java.lang.Integer,java.lang.Integer,java.lang.Double null "DynamicEndpointSnitchLegacyHistogram" null 5000 null #calling operation updateSnitch of mbean org.apache.cassandra.db:type=StorageService with params [null, DynamicEndpointSnitchLegacyHistogram, null, 1000, null] #operation returns: null {noformat} Or this can be updated in {{cassandra.yaml}} with the (intentionally) undocumented {{dynamic_snitch_class_name}} option. If a user does set that to not the default probing Histogram, we log a warning that it is not supported. I imagine that this pluggability will allow us to rapidly experiment with different metrics and load balancing implementations in CASSANDRA-14817. Summary of the patch: # Makes the {{DynamicEndpointSnitch}} pluggable and live swappable (the underlying {{IEndpointSnitch}} already was) # Refactors {{DynamicEndpointSnitch}} to send latency probes to interesting nodes instead of resetting the samples and losing all latency information # Refactors {{DynamicEndpointSnitch}} to be a lot more testable so we can test our future changes. # Provide three implementations, the new default as {{DynamicEndpointSnitchHistogram}} old legacy as {{DynamicEndpointSnitchLegacyHistogram}} and an experimental EMA based snitch as {{DynamicEndpointSnitchEMA}}. All three are run through the dynamic snitching ranking test. # Adds a bunch of unit tests of all the implementations. > 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 > Labels: 4.0-feature-freeze-review-requested, > pull-request-available > Fix For: 4.x > > Time Spent: 8h 10m > 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-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org